Skip to content

Commit

Permalink
Parquet coalesce file reader for local filesystems (#990)
Browse files Browse the repository at this point in the history
* Add back the small file consolidation for the parquet reader for non-cloud environments

Signed-off-by: Thomas Graves <tgraves@apache.org>

* make resolveURI local

Signed-off-by: Thomas Graves <tgraves@apache.org>

* debug

* fix debug

* Cleanup

* rework names

* Fix bug in footer psoition

* Add input file transition logic back and update tests

Signed-off-by: Thomas Graves <tgraves@nvidia.com>

* Update configs so can control multi file optmization, multi file read, and coalesce reader

* remove debug

Signed-off-by: Thomas Graves <tgraves@nvidia.com>

* Update tests for 3 parquet readers and small bug fix

* Update logging

* test fixes

* various fixes

* Update configs and fix parametsr to GpuParquetScan

Signed-off-by: Thomas Graves <tgraves@apache.org>

* remove unneeded function dbshim

Signed-off-by: Thomas Graves <tgraves@nvidia.com>

* remove debug log and update configs

Signed-off-by: Thomas Graves <tgraves@nvidia.com>

* cleanup and debug

* Update configs.md

* cleanup

Signed-off-by: Thomas Graves <tgraves@nvidia.com>

* create a common function for getting small file opts for fileSourceScan

* Fix extra line and update config text

* Update text

* change to use close on exception

Signed-off-by: Thomas Graves <tgraves@apache.org>

* update configs doc

* Fix missing imports

* Fix import order

Signed-off-by: Thomas Graves <tgraves@nvidia.com>

* Rework the parquet multi-file configs to have a single configuration and change the way they are passed around
for the InputFileName

Signed-off-by: Thomas Graves <tgraves@apache.org>

* make rapidsConf transient

Signed-off-by: Thomas Graves <tgraves@apache.org>

* fix typo

Signed-off-by: Thomas Graves <tgraves@apache.org>

* forward rapidsconf

Signed-off-by: Thomas Graves <tgraves@apache.org>

* update test and fix missed config check

* Add log statement for original per file reader

* Update text and fix test

* add space

Signed-off-by: Thomas Graves <tgraves@apache.org>

* update config.md

Signed-off-by: Thomas Graves <tgraves@nvidia.com>

* Fix parameter to spark 3.1.0 parquet san

* Update sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala

Co-authored-by: Jason Lowe <jlowe@nvidia.com>

* Update sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala

Co-authored-by: Jason Lowe <jlowe@nvidia.com>

* Update sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala

Co-authored-by: Jason Lowe <jlowe@nvidia.com>

* Update sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala

Co-authored-by: Jason Lowe <jlowe@nvidia.com>

* Update sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala

Co-authored-by: Jason Lowe <jlowe@nvidia.com>

* Fix scalastyle line length

Signed-off-by: Thomas Graves <tgraves@nvidia.com>

* Update docs and change tests to copy reader confs

* Update GpuColumnVector.from call to handle MapTypes

Signed-off-by: Thomas Graves <tgraves@nvidia.com>

Co-authored-by: Jason Lowe <jlowe@nvidia.com>
  • Loading branch information
tgravescs and jlowe authored Oct 27, 2020
1 parent d92032c commit 4c451ab
Show file tree
Hide file tree
Showing 12 changed files with 771 additions and 218 deletions.
7 changes: 4 additions & 3 deletions docs/configs.md
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@ scala> spark.conf.set("spark.rapids.sql.incompatibleOps.enabled", true)

Name | Description | Default Value
-----|-------------|--------------
<a name="cloudSchemes"></a>spark.rapids.cloudSchemes|Comma separated list of additional URI schemes that are to be considered cloud based filesystems. Schemes already included: dbfs, s3, s3a, s3n, wasbs, gs. Cloud based stores generally would be total separate from the executors and likely have a higher I/O read cost. Many times the cloud filesystems also get better throughput when you have multiple readers in parallel. This is used with spark.rapids.sql.format.parquet.reader.type|None
<a name="memory.gpu.allocFraction"></a>spark.rapids.memory.gpu.allocFraction|The fraction of total GPU memory that should be initially allocated for pooled memory. Extra memory will be allocated as needed, but it may result in more fragmentation. This must be less than or equal to the maximum limit configured via spark.rapids.memory.gpu.maxAllocFraction.|0.9
<a name="memory.gpu.debug"></a>spark.rapids.memory.gpu.debug|Provides a log of GPU memory allocations and frees. If set to STDOUT or STDERR the logging will go there. Setting it to NONE disables logging. All other values are reserved for possible future expansion and in the mean time will disable logging.|NONE
<a name="memory.gpu.maxAllocFraction"></a>spark.rapids.memory.gpu.maxAllocFraction|The fraction of total GPU memory that limits the maximum size of the RMM pool. The value must be greater than or equal to the setting for spark.rapids.memory.gpu.allocFraction. Note that this limit will be reduced by the reserve memory configured in spark.rapids.memory.gpu.reserve.|1.0
Expand Down Expand Up @@ -61,10 +62,10 @@ Name | Description | Default Value
<a name="sql.format.orc.read.enabled"></a>spark.rapids.sql.format.orc.read.enabled|When set to false disables orc input acceleration|true
<a name="sql.format.orc.write.enabled"></a>spark.rapids.sql.format.orc.write.enabled|When set to false disables orc output acceleration|true
<a name="sql.format.parquet.enabled"></a>spark.rapids.sql.format.parquet.enabled|When set to false disables all parquet input and output acceleration|true
<a name="sql.format.parquet.multiThreadedRead.enabled"></a>spark.rapids.sql.format.parquet.multiThreadedRead.enabled|When set to true, reads multiple small files within a partition more efficiently by reading each file in a separate thread in parallel on the CPU side before sending to the GPU. Limited by spark.rapids.sql.format.parquet.multiThreadedRead.numThreads and spark.rapids.sql.format.parquet.multiThreadedRead.maxNumFilesParallel|true
<a name="sql.format.parquet.multiThreadedRead.maxNumFilesParallel"></a>spark.rapids.sql.format.parquet.multiThreadedRead.maxNumFilesParallel|A limit on the maximum number of files per task processed in parallel on the CPU side before the file is sent to the GPU. This affects the amount of host memory used when reading the files in parallel.|2147483647
<a name="sql.format.parquet.multiThreadedRead.numThreads"></a>spark.rapids.sql.format.parquet.multiThreadedRead.numThreads|The maximum number of threads, on the executor, to use for reading small parquet files in parallel. This can not be changed at runtime after the executor has started.|20
<a name="sql.format.parquet.multiThreadedRead.maxNumFilesParallel"></a>spark.rapids.sql.format.parquet.multiThreadedRead.maxNumFilesParallel|A limit on the maximum number of files per task processed in parallel on the CPU side before the file is sent to the GPU. This affects the amount of host memory used when reading the files in parallel. Used with MULTITHREADED reader, see spark.rapids.sql.format.parquet.reader.type|2147483647
<a name="sql.format.parquet.multiThreadedRead.numThreads"></a>spark.rapids.sql.format.parquet.multiThreadedRead.numThreads|The maximum number of threads, on the executor, to use for reading small parquet files in parallel. This can not be changed at runtime after the executor has started. Used with COALESCING and MULTITHREADED reader, see spark.rapids.sql.format.parquet.reader.type.|20
<a name="sql.format.parquet.read.enabled"></a>spark.rapids.sql.format.parquet.read.enabled|When set to false disables parquet input acceleration|true
<a name="sql.format.parquet.reader.type"></a>spark.rapids.sql.format.parquet.reader.type|Sets the parquet reader type. We support different types that are optimized for different environments. The original Spark style reader can be selected by setting this to PERFILE which individually reads and copies files to the GPU. Loading many small files individually has high overhead, and using either COALESCING or MULTITHREADED is recommended instead. The COALESCING reader is good when using a local file system where the executors are on the same nodes or close to the nodes the data is being read on. This reader coalesces all the files assigned to a task into a single host buffer before sending it down to the GPU. It copies blocks from a single file into a host buffer in separate threads in parallel, see spark.rapids.sql.format.parquet.multiThreadedRead.numThreads. MULTITHREADED is good for cloud environments where you are reading from a blobstore that is totally separate and likely has a higher I/O read cost. Many times the cloud environments also get better throughput when you have multiple readers in parallel. This reader uses multiple threads to read each file in parallel and each file is sent to the GPU separately. This allows the CPU to keep reading while GPU is also doing work. See spark.rapids.sql.format.parquet.multiThreadedRead.numThreads and spark.rapids.sql.format.parquet.multiThreadedRead.maxNumFilesParallel to control the number of threads and amount of memory used. By default this is set to AUTO so we select the reader we think is best. This will either be the COALESCING or the MULTITHREADED based on whether we think the file is in the cloud. See spark.rapids.cloudSchemes.|AUTO
<a name="sql.format.parquet.write.enabled"></a>spark.rapids.sql.format.parquet.write.enabled|When set to false disables parquet output acceleration|true
<a name="sql.hasNans"></a>spark.rapids.sql.hasNans|Config to indicate if your data has NaN's. Cudf doesn't currently support NaN's properly so you can get corrupt data if you have NaN's in your data and it runs on the GPU.|true
<a name="sql.hashOptimizeSort.enabled"></a>spark.rapids.sql.hashOptimizeSort.enabled|Whether sorts should be inserted after some hashed operations to improve output ordering. This can improve output file sizes when saving to columnar formats.|false
Expand Down
Loading

0 comments on commit 4c451ab

Please sign in to comment.