Open LIN-Yu-Ting opened 4 months ago
Thanks for filing the issue. Do you have any details on how we could reproduce this? I'm wondering if this failure is specific to the chunked reader.
You could try re-running with spark.rapids.sql.reader.chunked=false to see if the failure is specific to the chunked reader.
We observe through our nvidia-smi saying that we are using CUDA 12.2. Would it be possible caused by CUDA version mismatch ?
The RAPIDS Spark plugin will work on the driver versions that come with CUDA 11 and CUDA 12, so the CUDA version should not be an issue.
parallel_for failed: cudaErrorInvalidDevice: invalid device ordinal
This likely indicates that we're trying to run a kernel with 0 blocks. @nvdbaranec is there a way the input data could trigger the Parquet chunked reader to do this?
There shouldn't be, but there's certainly the possibility of a bug. Based on this trace, it looks like it's in one of the setup kernels that runs before the actual decoding - computing various bits of bookkeeping related to chunking.
Would it be possible to get a sample of the data here?
@nvdbaranec I tried to describe the problem more clearly. We encountered this error when executing the following SQL command.
SELECT 'sampleId' FROM (SELECT GenotypeDT.runName,
GenotypeDT.sampleId,
GenotypeDT.calls,
GenotypeDT.start,
GenotypeDT.referenceAllele,
GenotypeDT.alternateAlleles,
VariantsDT.SYMBOL
FROM femh_rapids.genotypesdt_src AS GenotypeDT
LEFT JOIN femh_rapids.variantdt_src AS VariantsDT ON VariantsDT.contigName = GenotypeDT.contigName
AND VariantsDT.start = GenotypeDT.start
AND VariantsDT.referenceAllele = GenotypeDT.referenceAllele
AND VariantsDT.alternateAlleles = GenotypeDT.alternateAlleles)
GROUP BY sampleId
LIMIT 1000
where we LEFT JOIN table genotypesdt_src by another table variantdt_src.
genotypesdt_src
23 | 2024-06-02 12:47:30 | null | null | REPLACE TABLE AS SELECT | {'description': None, 'partitionBy': '["runName"]', 'properties': '{}', 'isManaged': 'false'} | null | null | null | 22 | Serializable | False | {'numOutputRows': '9572955254', 'numOutputBytes': '9649503160', 'numFiles': '280'} | null | Apache-Spark/3.3.0 Delta-Lake/2.3.0
variantdt_src
15 | 2024-05-14 11:22:54 | null | null | MERGE | {'matchedPredicates': '[]', 'predicate': '((((spark_catalog.100412_did.FEMH_NCGM_VariantsTable_vep_delta_1946444894.contigName = updatedTable.contigName) AND (spark_catalog.100412_did.FEMH_NCGM_VariantsTable_vep_delta_1946444894.start = updatedTable.start)) AND (spark_catalog.100412_did.FEMH_NCGM_VariantsTable_vep_delta_1946444894.end = updatedTable.end)) AND (((spark_catalog.100412_did.FEMH_NCGM_VariantsTable_vep_delta_1946444894.names = updatedTable.names) AND (spark_catalog.100412_did.FEMH_NCGM_VariantsTable_vep_delta_1946444894.referenceAllele = updatedTable.referenceAllele)) AND (spark_catalog.100412_did.FEMH_NCGM_VariantsTable_vep_delta_1946444894.alternateAlleles = updatedTable.alternateAlleles)))', 'notMatchedBySourcePredicates': '[]', 'notMatchedPredicates': '[{"actionType":"insert"}]'} | null | null | null | 14 | Serializable | False | {'numOutputRows': '2', 'numTargetBytesAdded': '306823', 'numTargetRowsInserted': '2', 'numTargetFilesAdded': '1', 'numTargetRowsMatchedDeleted': '0', 'numTargetFilesRemoved': '0', 'numTargetRowsMatchedUpdated': '0', 'executionTimeMs': '346189', 'numTargetRowsCopied': '0', 'rewriteTimeMs': '346002', 'numTargetRowsUpdated': '0', 'numTargetRowsDeleted': '0', 'scanTimeMs': '0', 'numSourceRows': '1101276', 'numTargetChangeFilesAdded': '0', 'numTargetRowsNotMatchedBySourceUpdated': '0', 'numTargetRowsNotMatchedBySourceDeleted': '0', 'numTargetBytesRemoved': '0'} | null | Apache-Spark/3.3.0 Delta-Lake/2.3.0
Exception does not appear when only LEFT JOIN is executed. Request can be finished in half a minute.
However, once we add GROUP BY sampleId then exception starts to appear.
This is the logical plan of our query
== Physical Plan ==
AdaptiveSparkPlan (14)
+- == Current Plan ==
CollectLimit (7)
+- HashAggregate (6)
+- ShuffleQueryStage (5)
+- GpuColumnarExchange (4)
+- GpuHashAggregate (3)
+- GpuProject (2)
+- GpuScan parquet femh_rapids.genotypesdt_src (1)
+- == Initial Plan ==
CollectLimit (13)
+- HashAggregate (12)
+- Exchange (11)
+- HashAggregate (10)
+- Project (9)
+- Scan parquet femh_rapids.genotypesdt_src (8)
(1) GpuScan parquet femh_rapids.genotypesdt_src
Output [1]: [sampleId#1606]
Batched: true
Location: PreparedDeltaFileIndex [abfss://******/FEMH-NCGM_GenotypeTable.delta]
ReadSchema: struct<sampleId:string>
(2) GpuProject
Input [1]: [sampleId#1606]
Arguments: [sampleId#1606], [loreId=5]
(3) GpuHashAggregate
Input [1]: [sampleId#1606]
Keys [1]: [sampleId#1606]
Functions: []
Aggregate Attributes: []
Results [1]: [sampleId#1606]
Lore: [loreId=6]
(4) GpuColumnarExchange
Input [1]: [sampleId#1606]
Arguments: gpuhashpartitioning(sampleId#1606, 200), ENSURE_REQUIREMENTS, [id=#1059], [loreId=7]
(5) ShuffleQueryStage
Output [1]: [sampleId#1606]
Arguments: 0
(6) HashAggregate
Input [1]: [sampleId#1606]
Keys [1]: [sampleId#1606]
Functions: []
Aggregate Attributes: []
Results [1]: [sampleId#1606]
(7) CollectLimit
Input [1]: [sampleId#1606]
Arguments: 1001
(8) Scan parquet femh_rapids.genotypesdt_src
Output [2]: [sampleId#1606, runName#1599]
Batched: true
Location: PreparedDeltaFileIndex [abfss://******/FEMH-NCGM_GenotypeTable.delta]
ReadSchema: struct<sampleId:string>
(9) Project
Output [1]: [sampleId#1606]
Input [2]: [sampleId#1606, runName#1599]
(10) HashAggregate
Input [1]: [sampleId#1606]
Keys [1]: [sampleId#1606]
Functions: []
Aggregate Attributes: []
Results [1]: [sampleId#1606]
(11) Exchange
Input [1]: [sampleId#1606]
Arguments: hashpartitioning(sampleId#1606, 200), ENSURE_REQUIREMENTS, [id=#970]
(12) HashAggregate
Input [1]: [sampleId#1606]
Keys [1]: [sampleId#1606]
Functions: []
Aggregate Attributes: []
Results [1]: [sampleId#1606]
(13) CollectLimit
Input [1]: [sampleId#1606]
Arguments: 1001
(14) AdaptiveSparkPlan
Output [1]: [sampleId#1606]
Arguments: isFinalPlan=false
Moreover, after we had this exception, we observe that session will not stop and spark cluster continuously occupies an executor.
Note: No exception will happen if we group by runName or SYMBOL.
Thanks for adding some details, @LIN-Yu-Ting! I suspect this is not related to the GROUPBY but instead is related to which columns are or are not being loaded from the tables. For reference, here's the working query: Failing:
SELECT GenotypeDT.runName,
GenotypeDT.sampleId,
GenotypeDT.calls,
GenotypeDT.start,
GenotypeDT.referenceAllele,
GenotypeDT.alternateAlleles,
VariantsDT.SYMBOL
FROM femh_rapids.genotypesdt_src AS GenotypeDT
LEFT JOIN femh_rapids.variantdt_src AS VariantsDT ON VariantsDT.contigName = GenotypeDT.contigName
AND VariantsDT.start = GenotypeDT.start
AND VariantsDT.referenceAllele = GenotypeDT.referenceAllele
AND VariantsDT.alternateAlleles = GenotypeDT.alternateAlleles
LIMIT 1000
Note that the number of columns being loaded during the scans between this and the failing query listed above is significantly different due to Spark's column pruning. In the original query, the SELECT sampleId FROM
that wraps the join will cause Spark to avoid loading any columns not necessary to perform the join and select the sampleId column. That means Spark will avoid loading GenotypeDT.runName, GenotypeDT.calls, and VariantsDT.SYMBOL since they aren't referenced in the join condition. Therefore I would expect this query, which does not perform an aggregation, to also fail:
SELECT 'sampleId' FROM (SELECT GenotypeDT.runName,
GenotypeDT.sampleId,
GenotypeDT.calls,
GenotypeDT.start,
GenotypeDT.referenceAllele,
GenotypeDT.alternateAlleles,
VariantsDT.SYMBOL
FROM femh_rapids.genotypesdt_src AS GenotypeDT
LEFT JOIN femh_rapids.variantdt_src AS VariantsDT ON VariantsDT.contigName = GenotypeDT.contigName
AND VariantsDT.start = GenotypeDT.start
AND VariantsDT.referenceAllele = GenotypeDT.referenceAllele
AND VariantsDT.alternateAlleles = GenotypeDT.alternateAlleles)
LIMIT 1000
There's a chance it may not fail if enough sampleID values are produced from the join and hits the limit before a task gets around to scanning the problematic file, since limits can cause it to early out of the query processing. Increasing or removing the limit should cause it to fail as with the GROUPBY (assuming the driver has enough memory to hold the result, which may be infeasible for this scenario).
So my guess is it's the absence of the few columns I listed above that triggers the error. It would be interesting to see if we can trigger this without a join at all, e.g. I would expect one of these queries to fail, depending on whether the problematic file is in the GenotypeDT or VariantsDT table. Limits or writes to temporary tables may need to be added if the driver cannot hold the results, with the caveat that limits may early-out the query before it fails.
SELECT GenotypeDT.sampleId,
GenotypeDT.start,
GenotypeDT.referenceAllele,
GenotypeDT.alternateAlleles,
GenotypeDT.contigName
FROM femh_rapids.genotypesdt_src AS GenotypeDT
WHERE GenotypeDT.start IS NOT NULL
AND GenotypeDT.referenceAllele IS NOT NULL
AND GenotypeDT.alternateAlleles IS NOT NULL
AND GenotypeDT.contigName IS NOT NULL
or
SELECT VariantsDT.contigName,
VariantsDT.start,
VariantsDT.referenceAlias,
VariantsDT.alternateAlias
FROM femh_rapids.variantdt_src AS VariantsDT
WHERE VariantsDT.contigName IS NOT NULL
AND VariantsDT.start IS NOT NULL
AND VariantsDT.referenceAlias IS NOT NULL
AND VariantsDT.alternateAlias IS NOT NULL
We also could check which stage is failing in the query and map that back to which table is being scanned to know which table must be the problematic one.
@nvdbaranec given that this provides clues that adding columns to the scan avoids the invalid device ordinal crash, I'm wondering if there's an issue where a large cluster of nulls in a column (or some other corner case) could cause one of the kernels being used to try to launch with zero blocks?
@jlowe Thanks for your comments. It seems that there is no exception while executing two SQL queries you recommended. I think it is because Table genotypesdt_src contains more than 9 billion records. I then tried with following SQL and got
SELECT DISTINCT(sampleId) FROM genotypesdt_src
23:51:55.776 ERROR RapidsExecutorPlugin - Stopping the Executor based on exception being a fatal CUDA error: java.io.IOException: Error when processing path: abfss://***/FEMH-NCGM_GenotypeTable.delta/runName=NCGM-1092-100277/part-00081-d0f234d8-b1fc-44f6-a7d9-019dcdb1f9b7.c000.snappy.parquet, range: 0-35611345, partition values: [empty row]; path: abfss://***/FEMH-NCGM_GenotypeTable.delta/runName=NCGM-1092-100277/part-00095-089ea8f3-f9f5-4b68-86f7-0efbca1e1297.c000.snappy.parquet, range: 0-35403176, partition values: [empty row]; path: abfss://***/FEMH-NCGM_GenotypeTable.delta/runName=100412_DID/part-00089-f173da8e-2e18-4c00-baf8-f1844b7b01e6.c000.snappy.parquet, range: 0-35395772, partition values: [empty row]
at com.nvidia.spark.rapids.ParquetTableReader.$anonfun$next$1(GpuParquetScan.scala:2701)
at com.nvidia.spark.rapids.Arm$.withResource(Arm.scala:30)
at com.nvidia.spark.rapids.ParquetTableReader.next(GpuParquetScan.scala:2692)
at com.nvidia.spark.rapids.ParquetTableReader.next(GpuParquetScan.scala:2664)
at com.nvidia.spark.rapids.CachedGpuBatchIterator$.$anonfun$apply$1(GpuDataProducer.scala:159)
at com.nvidia.spark.rapids.Arm$.withResource(Arm.scala:30)
at com.nvidia.spark.rapids.CachedGpuBatchIterator$.apply(GpuDataProducer.scala:156)
at com.nvidia.spark.rapids.MultiFileCloudParquetPartitionReader.$anonfun$readBufferToBatches$3(GpuParquetScan.scala:2573)
at com.nvidia.spark.rapids.RmmRapidsRetryIterator$AutoCloseableAttemptSpliterator.next(RmmRapidsRetryIterator.scala:477)
at com.nvidia.spark.rapids.RmmRapidsRetryIterator$RmmRapidsRetryIterator.next(RmmRapidsRetryIterator.scala:613)
at com.nvidia.spark.rapids.RmmRapidsRetryIterator$RmmRapidsRetryAutoCloseableIterator.next(RmmRapidsRetryIterator.scala:517)
at com.nvidia.spark.rapids.RmmRapidsRetryIterator$.drainSingleWithVerification(RmmRapidsRetryIterator.scala:291)
at com.nvidia.spark.rapids.RmmRapidsRetryIterator$.withRetryNoSplit(RmmRapidsRetryIterator.scala:132)
at com.nvidia.spark.rapids.MultiFileCloudParquetPartitionReader.readBufferToBatches(GpuParquetScan.scala:2560)
at com.nvidia.spark.rapids.MultiFileCloudParquetPartitionReader.readBatches(GpuParquetScan.scala:2530)
at com.nvidia.spark.rapids.MultiFileCloudPartitionReaderBase.liftedTree1$1(GpuMultiFileReader.scala:483)
at com.nvidia.spark.rapids.MultiFileCloudPartitionReaderBase.readBuffersToBatch(GpuMultiFileReader.scala:482)
at com.nvidia.spark.rapids.MultiFileCloudPartitionReaderBase.$anonfun$next$1(GpuMultiFileReader.scala:675)
at com.nvidia.spark.rapids.MultiFileCloudPartitionReaderBase.$anonfun$next$1$adapted(GpuMultiFileReader.scala:630)
at com.nvidia.spark.rapids.Arm$.withResource(Arm.scala:30)
at com.nvidia.spark.rapids.MultiFileCloudPartitionReaderBase.next(GpuMultiFileReader.scala:630)
at com.nvidia.spark.rapids.PartitionIterator.hasNext(dataSourceUtil.scala:29)
at com.nvidia.spark.rapids.MetricsBatchIterator.hasNext(dataSourceUtil.scala:46)
at com.nvidia.spark.rapids.shims.GpuDataSourceRDD$$anon$1.$anonfun$hasNext$1(GpuDataSourceRDD.scala:73)
at com.nvidia.spark.rapids.shims.GpuDataSourceRDD$$anon$1.$anonfun$hasNext$1$adapted(GpuDataSourceRDD.scala:73)
at scala.Option.exists(Option.scala:376)
at com.nvidia.spark.rapids.shims.GpuDataSourceRDD$$anon$1.hasNext(GpuDataSourceRDD.scala:73)
at com.nvidia.spark.rapids.shims.GpuDataSourceRDD$$anon$1.advanceToNextIter(GpuDataSourceRDD.scala:97)
at com.nvidia.spark.rapids.shims.GpuDataSourceRDD$$anon$1.hasNext(GpuDataSourceRDD.scala:73)
at org.apache.spark.InterruptibleIterator.hasNext(InterruptibleIterator.scala:37)
at org.apache.spark.sql.rapids.GpuFileSourceScanExec$$anon$1.hasNext(GpuFileSourceScanExec.scala:477)
at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:460)
at com.nvidia.spark.rapids.DynamicGpuPartialSortAggregateIterator.$anonfun$hasNext$4(GpuAggregateExec.scala:2005)
at scala.runtime.java8.JFunction0$mcZ$sp.apply(JFunction0$mcZ$sp.java:23)
at scala.Option.getOrElse(Option.scala:189)
at com.nvidia.spark.rapids.DynamicGpuPartialSortAggregateIterator.hasNext(GpuAggregateExec.scala:2005)
at org.apache.spark.sql.rapids.execution.GpuShuffleExchangeExecBase$$anon$1.partNextBatch(GpuShuffleExchangeExecBase.scala:332)
at org.apache.spark.sql.rapids.execution.GpuShuffleExchangeExecBase$$anon$1.hasNext(GpuShuffleExchangeExecBase.scala:355)
at org.apache.spark.shuffle.sort.BypassMergeSortShuffleWriter.write(BypassMergeSortShuffleWriter.java:140)
at org.apache.spark.shuffle.ShuffleWriteProcessor.write(ShuffleWriteProcessor.scala:59)
at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:99)
at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:52)
at org.apache.spark.scheduler.Task.run(Task.scala:136)
at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:548)
at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1504)
at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:551)
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
at java.lang.Thread.run(Thread.java:750)
Caused by: ai.rapids.cudf.CudaFatalException: std::bad_alloc: CUDA error at: /home/jenkins/agent/workspace/jenkins-spark-rapids-jni_nightly-dev-808-cuda12/target/libcudf/cmake-build/_deps/rmm-src/include/rmm/mr/device/cuda_async_view_memory_resource.hpp:111: cudaErrorIllegalAddress an illegal memory access was encountered
at ai.rapids.cudf.ParquetChunkedReader.readChunk(Native Method)
at ai.rapids.cudf.ParquetChunkedReader.readChunk(ParquetChunkedReader.java:170)
at com.nvidia.spark.rapids.ParquetTableReader.$anonfun$next$1(GpuParquetScan.scala:2694)
... 48 more
23:51:55.879 WARN RapidsExecutorPlugin - nvidia-smi:
Sat Jul 27 23:51:55 2024
+---------------------------------------------------------------------------------------+
| NVIDIA-SMI 535.183.01 Driver Version: 535.183.01 CUDA Version: 12.2 |
|-----------------------------------------+----------------------+----------------------+
| GPU Name Persistence-M | Bus-Id Disp.A | Volatile Uncorr. ECC |
| Fan Temp Perf Pwr:Usage/Cap | Memory-Usage | GPU-Util Compute M. |
| | | MIG M. |
|=========================================+======================+======================|
| 0 Tesla T4 Off | 00000001:00:00.0 Off | Off |
| N/A 45C P0 37W / 70W | 15303MiB / 16384MiB | 97% Default |
| | | N/A |
+-----------------------------------------+----------------------+----------------------+
+---------------------------------------------------------------------------------------+
| Processes: |
| GPU GI CI PID Type Process name GPU Memory |
| ID ID Usage |
|=======================================================================================|
+---------------------------------------------------------------------------------------+
23:51:55.901 INFO RapidsBufferCatalog - Closing storage
23:51:57.209 INFO AwsStorageExecutorPlugin - Shutting down S3 Plugin ...
23:51:57.209 ERROR Executor - Exception in task 84.0 in stage 10.0 (TID 237)
java.io.IOException: Error when processing path: abfss://***/FEMH-NCGM_GenotypeTable.delta/runName=100412_DID/part-00082-730d6f11-dbe3-48c9-a5c0-275023c48ae7.c000.snappy.parquet, range: 0-29427533, partition values: [empty row]; path: abfss://***/FEMH-NCGM_GenotypeTable.delta/runName=100325_DID/part-00121-fa29ba19-489f-4ec0-b446-677824a27ad2.c000.snappy.parquet, range: 0-29169410, partition values: [empty row]; path: abfss://***/FEMH-NCGM_GenotypeTable.delta/runName=NCGM-1092-100277/part-00093-8a196586-2fd5-4fdd-8085-3a9791b2fec1.c000.snappy.parquet, range: 0-28902069, partition values: [empty row]; path: abfss://***/FEMH-NCGM_GenotypeTable.delta/runName=NCGM-1092-100094/part-00132-f9bbff37-1ff9-46a0-8c0e-992d9dc4873a.c000.snappy.parquet, range: 0-28713272, partition values: [empty row]
at com.nvidia.spark.rapids.ParquetTableReader.$anonfun$next$1(GpuParquetScan.scala:2701) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.Arm$.withResource(Arm.scala:30) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.ParquetTableReader.next(GpuParquetScan.scala:2692) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.ParquetTableReader.next(GpuParquetScan.scala:2664) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.CachedGpuBatchIterator$.$anonfun$apply$1(GpuDataProducer.scala:159) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.Arm$.withResource(Arm.scala:30) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.CachedGpuBatchIterator$.apply(GpuDataProducer.scala:156) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.MultiFileCloudParquetPartitionReader.$anonfun$readBufferToBatches$3(GpuParquetScan.scala:2573) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.RmmRapidsRetryIterator$AutoCloseableAttemptSpliterator.next(RmmRapidsRetryIterator.scala:477) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.RmmRapidsRetryIterator$RmmRapidsRetryIterator.next(RmmRapidsRetryIterator.scala:613) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.RmmRapidsRetryIterator$RmmRapidsRetryAutoCloseableIterator.next(RmmRapidsRetryIterator.scala:517) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.RmmRapidsRetryIterator$.drainSingleWithVerification(RmmRapidsRetryIterator.scala:291) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.RmmRapidsRetryIterator$.withRetryNoSplit(RmmRapidsRetryIterator.scala:132) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.MultiFileCloudParquetPartitionReader.readBufferToBatches(GpuParquetScan.scala:2560) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.MultiFileCloudParquetPartitionReader.readBatches(GpuParquetScan.scala:2530) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.MultiFileCloudPartitionReaderBase.liftedTree1$1(GpuMultiFileReader.scala:483) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.MultiFileCloudPartitionReaderBase.readBuffersToBatch(GpuMultiFileReader.scala:482) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.MultiFileCloudPartitionReaderBase.$anonfun$next$1(GpuMultiFileReader.scala:675) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.MultiFileCloudPartitionReaderBase.$anonfun$next$1$adapted(GpuMultiFileReader.scala:630) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.Arm$.withResource(Arm.scala:30) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.MultiFileCloudPartitionReaderBase.next(GpuMultiFileReader.scala:630) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.PartitionIterator.hasNext(dataSourceUtil.scala:29) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.MetricsBatchIterator.hasNext(dataSourceUtil.scala:46) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.shims.GpuDataSourceRDD$$anon$1.$anonfun$hasNext$1(GpuDataSourceRDD.scala:73) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.shims.GpuDataSourceRDD$$anon$1.$anonfun$hasNext$1$adapted(GpuDataSourceRDD.scala:73) ~[rapids-4-spark_2.12-cuda12.jar:?]
at scala.Option.exists(Option.scala:376) ~[scala-library-2.12.15.jar:?]
at com.nvidia.spark.rapids.shims.GpuDataSourceRDD$$anon$1.hasNext(GpuDataSourceRDD.scala:73) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.shims.GpuDataSourceRDD$$anon$1.advanceToNextIter(GpuDataSourceRDD.scala:97) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.shims.GpuDataSourceRDD$$anon$1.hasNext(GpuDataSourceRDD.scala:73) ~[rapids-4-spark_2.12-cuda12.jar:?]
at org.apache.spark.InterruptibleIterator.hasNext(InterruptibleIterator.scala:37) ~[spark-core_2.12-3.3.0.jar:3.3.0]
at org.apache.spark.sql.rapids.GpuFileSourceScanExec$$anon$1.hasNext(GpuFileSourceScanExec.scala:477) ~[rapids-4-spark_2.12-cuda12.jar:?]
at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:460) ~[scala-library-2.12.15.jar:?]
at com.nvidia.spark.rapids.DynamicGpuPartialSortAggregateIterator.$anonfun$hasNext$4(GpuAggregateExec.scala:2005) ~[rapids-4-spark_2.12-cuda12.jar:?]
at scala.runtime.java8.JFunction0$mcZ$sp.apply(JFunction0$mcZ$sp.java:23) ~[scala-library-2.12.15.jar:?]
at scala.Option.getOrElse(Option.scala:189) ~[scala-library-2.12.15.jar:?]
at com.nvidia.spark.rapids.DynamicGpuPartialSortAggregateIterator.hasNext(GpuAggregateExec.scala:2005) ~[rapids-4-spark_2.12-cuda12.jar:?]
at org.apache.spark.sql.rapids.execution.GpuShuffleExchangeExecBase$$anon$1.partNextBatch(GpuShuffleExchangeExecBase.scala:332) ~[rapids-4-spark_2.12-cuda12.jar:?]
at org.apache.spark.sql.rapids.execution.GpuShuffleExchangeExecBase$$anon$1.hasNext(GpuShuffleExchangeExecBase.scala:355) ~[rapids-4-spark_2.12-cuda12.jar:?]
at org.apache.spark.shuffle.sort.BypassMergeSortShuffleWriter.write(BypassMergeSortShuffleWriter.java:140) ~[spark-core_2.12-3.3.0.jar:3.3.0]
at org.apache.spark.shuffle.ShuffleWriteProcessor.write(ShuffleWriteProcessor.scala:59) ~[spark-core_2.12-3.3.0.jar:3.3.0]
at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:99) ~[spark-core_2.12-3.3.0.jar:3.3.0]
at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:52) ~[spark-core_2.12-3.3.0.jar:3.3.0]
at org.apache.spark.scheduler.Task.run(Task.scala:136) ~[spark-core_2.12-3.3.0.jar:3.3.0]
at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:548) ~[spark-core_2.12-3.3.0.jar:3.3.0]
at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1504) ~[spark-core_2.12-3.3.0.jar:3.3.0]
at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:551) ~[spark-core_2.12-3.3.0.jar:3.3.0]
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) ~[?:1.8.0_412]
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) ~[?:1.8.0_412]
at java.lang.Thread.run(Thread.java:750) ~[?:1.8.0_412]
Caused by: ai.rapids.cudf.CudaFatalException: std::bad_alloc: CUDA error at: /home/jenkins/agent/workspace/jenkins-spark-rapids-jni_nightly-dev-808-cuda12/target/libcudf-install/include/rmm/mr/device/cuda_memory_resource.hpp:60: cudaErrorIllegalAddress an illegal memory access was encountered
at ai.rapids.cudf.ParquetChunkedReader.readChunk(Native Method) ~[rapids-4-spark_2.12-cuda12.jar:?]
at ai.rapids.cudf.ParquetChunkedReader.readChunk(ParquetChunkedReader.java:170) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.ParquetTableReader.$anonfun$next$1(GpuParquetScan.scala:2694) ~[rapids-4-spark_2.12-cuda12.jar:?]
... 48 more
23:51:57.209 ERROR Executor - Exception in task 93.0 in stage 10.0 (TID 246)
java.io.IOException: Error when processing path: abfss://***/FEMH-NCGM_GenotypeTable.delta/runName=100412_DID/part-00081-bdc90355-8abe-4634-9bce-6c0a34be24fc.c000.snappy.parquet, range: 0-23850813, partition values: [empty row]; path: abfss://***/FEMH-NCGM_GenotypeTable.delta/runName=NCGM-1092-100238/part-00071-0fcc47d9-5142-4564-86b4-d8efb8cf9cde.c000.snappy.parquet, range: 0-23558609, partition values: [empty row]; path: abfss://***/FEMH-NCGM_GenotypeTable.delta/runName=NCGM-1092-100138/part-00018-770c9457-bcb9-4a9f-a63f-1e5f6fb7f370.c000.snappy.parquet, range: 0-23485835, partition values: [empty row]; path: abfss://***/FEMH-NCGM_GenotypeTable.delta/runName=NCGM-1092-100335/part-00062-2e573ac6-f9d2-497a-8c5a-90a2e358ee25.c000.snappy.parquet, range: 0-23463201, partition values: [empty row]
at com.nvidia.spark.rapids.ParquetTableReader.$anonfun$next$1(GpuParquetScan.scala:2701) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.Arm$.withResource(Arm.scala:30) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.ParquetTableReader.next(GpuParquetScan.scala:2692) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.ParquetTableReader.next(GpuParquetScan.scala:2664) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.CachedGpuBatchIterator$.$anonfun$apply$1(GpuDataProducer.scala:159) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.Arm$.withResource(Arm.scala:30) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.CachedGpuBatchIterator$.apply(GpuDataProducer.scala:156) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.MultiFileCloudParquetPartitionReader.$anonfun$readBufferToBatches$3(GpuParquetScan.scala:2573) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.RmmRapidsRetryIterator$AutoCloseableAttemptSpliterator.next(RmmRapidsRetryIterator.scala:477) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.RmmRapidsRetryIterator$RmmRapidsRetryIterator.next(RmmRapidsRetryIterator.scala:613) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.RmmRapidsRetryIterator$RmmRapidsRetryAutoCloseableIterator.next(RmmRapidsRetryIterator.scala:517) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.RmmRapidsRetryIterator$.drainSingleWithVerification(RmmRapidsRetryIterator.scala:291) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.RmmRapidsRetryIterator$.withRetryNoSplit(RmmRapidsRetryIterator.scala:132) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.MultiFileCloudParquetPartitionReader.readBufferToBatches(GpuParquetScan.scala:2560) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.MultiFileCloudParquetPartitionReader.readBatches(GpuParquetScan.scala:2530) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.MultiFileCloudPartitionReaderBase.liftedTree1$1(GpuMultiFileReader.scala:483) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.MultiFileCloudPartitionReaderBase.readBuffersToBatch(GpuMultiFileReader.scala:482) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.MultiFileCloudPartitionReaderBase.$anonfun$next$1(GpuMultiFileReader.scala:675) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.MultiFileCloudPartitionReaderBase.$anonfun$next$1$adapted(GpuMultiFileReader.scala:630) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.Arm$.withResource(Arm.scala:30) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.MultiFileCloudPartitionReaderBase.next(GpuMultiFileReader.scala:630) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.PartitionIterator.hasNext(dataSourceUtil.scala:29) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.MetricsBatchIterator.hasNext(dataSourceUtil.scala:46) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.shims.GpuDataSourceRDD$$anon$1.$anonfun$hasNext$1(GpuDataSourceRDD.scala:73) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.shims.GpuDataSourceRDD$$anon$1.$anonfun$hasNext$1$adapted(GpuDataSourceRDD.scala:73) ~[rapids-4-spark_2.12-cuda12.jar:?]
at scala.Option.exists(Option.scala:376) ~[scala-library-2.12.15.jar:?]
at com.nvidia.spark.rapids.shims.GpuDataSourceRDD$$anon$1.hasNext(GpuDataSourceRDD.scala:73) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.shims.GpuDataSourceRDD$$anon$1.advanceToNextIter(GpuDataSourceRDD.scala:97) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.shims.GpuDataSourceRDD$$anon$1.hasNext(GpuDataSourceRDD.scala:73) ~[rapids-4-spark_2.12-cuda12.jar:?]
at org.apache.spark.InterruptibleIterator.hasNext(InterruptibleIterator.scala:37) ~[spark-core_2.12-3.3.0.jar:3.3.0]
at org.apache.spark.sql.rapids.GpuFileSourceScanExec$$anon$1.hasNext(GpuFileSourceScanExec.scala:477) ~[rapids-4-spark_2.12-cuda12.jar:?]
at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:460) ~[scala-library-2.12.15.jar:?]
at com.nvidia.spark.rapids.DynamicGpuPartialSortAggregateIterator.$anonfun$hasNext$4(GpuAggregateExec.scala:2005) ~[rapids-4-spark_2.12-cuda12.jar:?]
at scala.runtime.java8.JFunction0$mcZ$sp.apply(JFunction0$mcZ$sp.java:23) ~[scala-library-2.12.15.jar:?]
at scala.Option.getOrElse(Option.scala:189) ~[scala-library-2.12.15.jar:?]
at com.nvidia.spark.rapids.DynamicGpuPartialSortAggregateIterator.hasNext(GpuAggregateExec.scala:2005) ~[rapids-4-spark_2.12-cuda12.jar:?]
at org.apache.spark.sql.rapids.execution.GpuShuffleExchangeExecBase$$anon$1.partNextBatch(GpuShuffleExchangeExecBase.scala:332) ~[rapids-4-spark_2.12-cuda12.jar:?]
at org.apache.spark.sql.rapids.execution.GpuShuffleExchangeExecBase$$anon$1.hasNext(GpuShuffleExchangeExecBase.scala:355) ~[rapids-4-spark_2.12-cuda12.jar:?]
at org.apache.spark.shuffle.sort.BypassMergeSortShuffleWriter.write(BypassMergeSortShuffleWriter.java:140) ~[spark-core_2.12-3.3.0.jar:3.3.0]
at org.apache.spark.shuffle.ShuffleWriteProcessor.write(ShuffleWriteProcessor.scala:59) ~[spark-core_2.12-3.3.0.jar:3.3.0]
at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:99) ~[spark-core_2.12-3.3.0.jar:3.3.0]
at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:52) ~[spark-core_2.12-3.3.0.jar:3.3.0]
at org.apache.spark.scheduler.Task.run(Task.scala:136) ~[spark-core_2.12-3.3.0.jar:3.3.0]
at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:548) ~[spark-core_2.12-3.3.0.jar:3.3.0]
at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1504) ~[spark-core_2.12-3.3.0.jar:3.3.0]
at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:551) ~[spark-core_2.12-3.3.0.jar:3.3.0]
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) ~[?:1.8.0_412]
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) ~[?:1.8.0_412]
at java.lang.Thread.run(Thread.java:750) ~[?:1.8.0_412]
Caused by: ai.rapids.cudf.CudaFatalException: std::bad_alloc: CUDA error at: /home/jenkins/agent/workspace/jenkins-spark-rapids-jni_nightly-dev-808-cuda12/target/libcudf-install/include/rmm/mr/device/cuda_memory_resource.hpp:60: cudaErrorIllegalAddress an illegal memory access was encountered
at ai.rapids.cudf.ParquetChunkedReader.readChunk(Native Method) ~[rapids-4-spark_2.12-cuda12.jar:?]
at ai.rapids.cudf.ParquetChunkedReader.readChunk(ParquetChunkedReader.java:170) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.ParquetTableReader.$anonfun$next$1(GpuParquetScan.scala:2694) ~[rapids-4-spark_2.12-cuda12.jar:?]
... 48 more
Then by executing again the same query, we reproduced the original exception.
SELECT DISTINCT(sampleId) FROM genotypesdt_src
00:38:35.964 ERROR Executor - Exception in task 59.0 in stage 36.0 (TID 651)
ai.rapids.cudf.CudfException: parallel_for failed: cudaErrorInvalidDevice: invalid device ordinal
at ai.rapids.cudf.ParquetChunkedReader.hasNext(Native Method) ~[rapids-4-spark_2.12-cuda12.jar:?]
at ai.rapids.cudf.ParquetChunkedReader.hasNext(ParquetChunkedReader.java:155) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.ParquetTableReader.hasNext(GpuParquetScan.scala:2688) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.GpuDataProducer.foreach(GpuDataProducer.scala:56) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.GpuDataProducer.foreach$(GpuDataProducer.scala:55) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.ParquetTableReader.foreach(GpuParquetScan.scala:2664) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.CachedGpuBatchIterator$.$anonfun$apply$2(GpuDataProducer.scala:168) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.Arm$.closeOnExcept(Arm.scala:98) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.CachedGpuBatchIterator$.$anonfun$apply$1(GpuDataProducer.scala:159) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.Arm$.withResource(Arm.scala:30) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.CachedGpuBatchIterator$.apply(GpuDataProducer.scala:156) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.MultiFileCloudParquetPartitionReader.$anonfun$readBufferToBatches$3(GpuParquetScan.scala:2573) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.RmmRapidsRetryIterator$AutoCloseableAttemptSpliterator.next(RmmRapidsRetryIterator.scala:477) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.RmmRapidsRetryIterator$RmmRapidsRetryIterator.next(RmmRapidsRetryIterator.scala:613) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.RmmRapidsRetryIterator$RmmRapidsRetryAutoCloseableIterator.next(RmmRapidsRetryIterator.scala:517) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.RmmRapidsRetryIterator$.drainSingleWithVerification(RmmRapidsRetryIterator.scala:291) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.RmmRapidsRetryIterator$.withRetryNoSplit(RmmRapidsRetryIterator.scala:132) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.MultiFileCloudParquetPartitionReader.readBufferToBatches(GpuParquetScan.scala:2560) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.MultiFileCloudParquetPartitionReader.readBatches(GpuParquetScan.scala:2530) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.MultiFileCloudPartitionReaderBase.liftedTree1$1(GpuMultiFileReader.scala:483) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.MultiFileCloudPartitionReaderBase.readBuffersToBatch(GpuMultiFileReader.scala:482) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.MultiFileCloudPartitionReaderBase.$anonfun$next$1(GpuMultiFileReader.scala:675) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.MultiFileCloudPartitionReaderBase.$anonfun$next$1$adapted(GpuMultiFileReader.scala:630) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.Arm$.withResource(Arm.scala:30) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.MultiFileCloudPartitionReaderBase.next(GpuMultiFileReader.scala:630) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.PartitionIterator.hasNext(dataSourceUtil.scala:29) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.MetricsBatchIterator.hasNext(dataSourceUtil.scala:46) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.shims.GpuDataSourceRDD$$anon$1.$anonfun$hasNext$1(GpuDataSourceRDD.scala:73) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.shims.GpuDataSourceRDD$$anon$1.$anonfun$hasNext$1$adapted(GpuDataSourceRDD.scala:73) ~[rapids-4-spark_2.12-cuda12.jar:?]
at scala.Option.exists(Option.scala:376) ~[scala-library-2.12.15.jar:?]
at com.nvidia.spark.rapids.shims.GpuDataSourceRDD$$anon$1.hasNext(GpuDataSourceRDD.scala:73) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.shims.GpuDataSourceRDD$$anon$1.advanceToNextIter(GpuDataSourceRDD.scala:97) ~[rapids-4-spark_2.12-cuda12.jar:?]
at com.nvidia.spark.rapids.shims.GpuDataSourceRDD$$anon$1.hasNext(GpuDataSourceRDD.scala:73) ~[rapids-4-spark_2.12-cuda12.jar:?]
at org.apache.spark.InterruptibleIterator.hasNext(InterruptibleIterator.scala:37) ~[spark-core_2.12-3.3.0.jar:3.3.0]
at org.apache.spark.sql.rapids.GpuFileSourceScanExec$$anon$1.hasNext(GpuFileSourceScanExec.scala:477) ~[rapids-4-spark_2.12-cuda12.jar:?]
at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:460) ~[scala-library-2.12.15.jar:?]
at com.nvidia.spark.rapids.DynamicGpuPartialSortAggregateIterator.$anonfun$hasNext$4(GpuAggregateExec.scala:2005) ~[rapids-4-spark_2.12-cuda12.jar:?]
at scala.runtime.java8.JFunction0$mcZ$sp.apply(JFunction0$mcZ$sp.java:23) ~[scala-library-2.12.15.jar:?]
at scala.Option.getOrElse(Option.scala:189) ~[scala-library-2.12.15.jar:?]
at com.nvidia.spark.rapids.DynamicGpuPartialSortAggregateIterator.hasNext(GpuAggregateExec.scala:2005) ~[rapids-4-spark_2.12-cuda12.jar:?]
at org.apache.spark.sql.rapids.execution.GpuShuffleExchangeExecBase$$anon$1.partNextBatch(GpuShuffleExchangeExecBase.scala:332) ~[rapids-4-spark_2.12-cuda12.jar:?]
at org.apache.spark.sql.rapids.execution.GpuShuffleExchangeExecBase$$anon$1.hasNext(GpuShuffleExchangeExecBase.scala:355) ~[rapids-4-spark_2.12-cuda12.jar:?]
at org.apache.spark.shuffle.sort.BypassMergeSortShuffleWriter.write(BypassMergeSortShuffleWriter.java:140) ~[spark-core_2.12-3.3.0.jar:3.3.0]
at org.apache.spark.shuffle.ShuffleWriteProcessor.write(ShuffleWriteProcessor.scala:59) ~[spark-core_2.12-3.3.0.jar:3.3.0]
at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:99) ~[spark-core_2.12-3.3.0.jar:3.3.0]
at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:52) ~[spark-core_2.12-3.3.0.jar:3.3.0]
at org.apache.spark.scheduler.Task.run(Task.scala:136) ~[spark-core_2.12-3.3.0.jar:3.3.0]
at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:548) ~[spark-core_2.12-3.3.0.jar:3.3.0]
at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1504) ~[spark-core_2.12-3.3.0.jar:3.3.0]
at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:551) ~[spark-core_2.12-3.3.0.jar:3.3.0]
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) ~[?:1.8.0_412]
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) ~[?:1.8.0_412]
at java.lang.Thread.run(Thread.java:750) ~[?:1.8.0_412]
OK, those new queries and exception messages help. That shows us that the problem can be reproduced by loading just the sampleId column from the genotypesdt_src and specifically that the issue can be reproduced when trying to load the concatenated data from these ranges, assuming this is the first occurrence of the error:
abfss://***/FEMH-NCGM_GenotypeTable.delta/runName=NCGM-1092-100277/part-00081-d0f234d8-b1fc-44f6-a7d9-019dcdb1f9b7.c000.snappy.parquet, range: 0-35611345, partition values: [empty row];
path: abfss://***/FEMH-NCGM_GenotypeTable.delta/runName=NCGM-1092-100277/part-00095-089ea8f3-f9f5-4b68-86f7-0efbca1e1297.c000.snappy.parquet, range: 0-35403176, partition values: [empty row];
path: abfss://***/FEMH-NCGM_GenotypeTable.delta/runName=100412_DID/part-00089-f173da8e-2e18-4c00-baf8-f1844b7b01e6.c000.snappy.parquet, range: 0-35395772, partition values: [empty row]
There are some other file range groups reporting an error, but it's hard to tell if these occurred before or after the one reported above. They seemed to occur afterwards, but there could be a race between threads for which one reports the error first. Once the error occurs, all reads and any other GPU operations will report the error since a GPU illegal address exception is unrecoverable for a CUDA process once it occurs.
path: abfss://***/FEMH-NCGM_GenotypeTable.delta/runName=100412_DID/part-00082-730d6f11-dbe3-48c9-a5c0-275023c48ae7.c000.snappy.parquet, range: 0-29427533, partition values: [empty row];
path: abfss://***/FEMH-NCGM_GenotypeTable.delta/runName=100325_DID/part-00121-fa29ba19-489f-4ec0-b446-677824a27ad2.c000.snappy.parquet, range: 0-29169410, partition values: [empty row];
path: abfss://***/FEMH-NCGM_GenotypeTable.delta/runName=NCGM-1092-100277/part-00093-8a196586-2fd5-4fdd-8085-3a9791b2fec1.c000.snappy.parquet, range: 0-28902069, partition values: [empty row];
path: abfss://***/FEMH-NCGM_GenotypeTable.delta/runName=NCGM-1092-100094/part-00132-f9bbff37-1ff9-46a0-8c0e-992d9dc4873a.c000.snappy.parquet, range: 0-28713272, partition values: [empty row]
path: abfss://***/FEMH-NCGM_GenotypeTable.delta/runName=100412_DID/part-00081-bdc90355-8abe-4634-9bce-6c0a34be24fc.c000.snappy.parquet, range: 0-23850813, partition values: [empty row];
path: abfss://***/FEMH-NCGM_GenotypeTable.delta/runName=NCGM-1092-100238/part-00071-0fcc47d9-5142-4564-86b4-d8efb8cf9cde.c000.snappy.parquet, range: 0-23558609, partition values: [empty row];
path: abfss://***/FEMH-NCGM_GenotypeTable.delta/runName=NCGM-1092-100138/part-00018-770c9457-bcb9-4a9f-a63f-1e5f6fb7f370.c000.snappy.parquet, range: 0-23485835, partition values: [empty row];
path: abfss://***/FEMH-NCGM_GenotypeTable.delta/runName=NCGM-1092-100335/part-00062-2e573ac6-f9d2-497a-8c5a-90a2e358ee25.c000.snappy.parquet, range: 0-23463201, partition values: [empty row]
@LIN-Yu-Ting it would be great if you have some time to investigate if anything "interesting" is going on with sampleId values in these files (e.g.: long sequences of NULL values, etc.) that might trigger a corner-case in the Parquet reader code. It would also be interesting if we could reproduce the error just by performing the query on one or more tables consisting of just the files in each group separately (e.g.: running Spark in local mode with only 1 executor with only 1 core and a large setting for spark.sql.files.maxPartitionBytes so that a single task tries to load all of the data in one task).
@LIN-Yu-Ting If you want to share some sample file which can repro this issue or if you want to discuss further with more details with sensitive information, you can use spark-rapids-support spark-rapids-support@nvidia.com and we will keep this discussion internal with you.
@viadea Are there any updates from your sides ?
Describe the bug We are using Spark Rapids 24.08-SNAPSHOT with delta table 2.3.0 and then we encounter the following exception while executing a LEFT JOIN sql query.
Steps/Code to reproduce bug Please provide a list of steps or a code sample to reproduce the issue. Avoid posting private or sensitive data.
Expected behavior A clear and concise description of what you expected to happen.
Environment details (please complete the following information)
Additional context We observe through our nvidia-smi saying that we are using CUDA 12.2. Would it be possible caused by CUDA version mismatch ?