Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[BUG] split function+ repartition result in "ai.rapids.cudf.CudaException: device-side assert triggered" #2048

Closed
viadea opened this issue Mar 30, 2021 · 1 comment · Fixed by #2145
Assignees
Labels
bug Something isn't working P0 Must have for release

Comments

@viadea
Copy link
Collaborator

viadea commented Mar 30, 2021

Describe the bug
A clear and concise description of what the bug is.

If we use split function and then repartition afterwards, it will trigger an error:
ai.rapids.cudf.CudaException: device-side assert triggered

Full stacktrace is:

Caused by: ai.rapids.cudf.CudaException: device-side assert triggered
	at ai.rapids.cudf.Cuda.memcpyOnStream(Native Method)
	at ai.rapids.cudf.Cuda.memcpy(Cuda.java:436)
	at ai.rapids.cudf.Cuda.memcpy(Cuda.java:279)
	at ai.rapids.cudf.BaseDeviceMemoryBuffer.copyFromHostBuffer(BaseDeviceMemoryBuffer.java:43)
	at ai.rapids.cudf.BaseDeviceMemoryBuffer.copyFromHostBuffer(BaseDeviceMemoryBuffer.java:105)
	at ai.rapids.cudf.HostColumnVector.copyToDevice(HostColumnVector.java:197)
	at ai.rapids.cudf.HostColumnVector$ColumnBuilder.buildAndPutOnDevice(HostColumnVector.java:1197)
	at com.nvidia.spark.rapids.GpuColumnVector$GpuColumnarBatchBuilder.buildAndPutOnDevice(GpuColumnVector.java:344)
	at com.nvidia.spark.rapids.GpuColumnVector$GpuColumnarBatchBuilderBase.build(GpuColumnVector.java:216)
	at com.nvidia.spark.rapids.RowToColumnarIterator.$anonfun$buildBatch$2(GpuRowToColumnarExec.scala:589)
	at com.nvidia.spark.rapids.Arm.withResource(Arm.scala:28)
	at com.nvidia.spark.rapids.Arm.withResource$(Arm.scala:26)
	at com.nvidia.spark.rapids.RowToColumnarIterator.withResource(GpuRowToColumnarExec.scala:525)
	at com.nvidia.spark.rapids.RowToColumnarIterator.buildBatch(GpuRowToColumnarExec.scala:588)
	at com.nvidia.spark.rapids.RowToColumnarIterator.next(GpuRowToColumnarExec.scala:546)
	at com.nvidia.spark.rapids.RowToColumnarIterator.next(GpuRowToColumnarExec.scala:525)
	at scala.collection.Iterator$$anon$10.next(Iterator.scala:459)
	at com.nvidia.spark.rapids.GpuColumnarBatchSorter$$anon$1.loadNextBatch(GpuSortExec.scala:151)
	at com.nvidia.spark.rapids.GpuColumnarBatchSorter$$anon$1.hasNext(GpuSortExec.scala:202)
	at org.apache.spark.sql.rapids.execution.GpuShuffleExchangeExec$$anon$1.partNextBatch(GpuShuffleExchangeExec.scala:205)
	at org.apache.spark.sql.rapids.execution.GpuShuffleExchangeExec$$anon$1.hasNext(GpuShuffleExchangeExec.scala:222)
	at org.apache.spark.shuffle.sort.BypassMergeSortShuffleWriter.write(BypassMergeSortShuffleWriter.java:132)
	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:131)
	at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:497)
	at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1439)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:500)
	at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
	at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
	... 1 more

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.

Minimum reproduce in pyspark:

from pyspark.sql import SparkSession
from pyspark.sql.functions import col, split

data = [("aaa", "123 456 789"), ("bbb", "444 555 666"), ("ccc", "777 888 999")]
columns = ["a","b"]
df = spark.createDataFrame(data).toDF(*columns)
newdf = df.withColumn('newb', split(col('b'),' ')).repartition(3)
newdf.show()

Expected behavior
A clear and concise description of what you expected to happen.

It should work fine as CPU mode.

Environment details (please complete the following information)

  • Environment location: [Standalone, YARN, Kubernetes, Cloud(specify cloud provider)]
  • Spark configuration settings related to the issue

Rapids for Spark: 0.4.1
Apache Spark 3.1.1

Additional context
Add any other context about the problem here.

@viadea viadea added bug Something isn't working ? - Needs Triage Need team to review and classify labels Mar 30, 2021
@sameerz sameerz added P0 Must have for release and removed ? - Needs Triage Need team to review and classify labels Mar 30, 2021
@sameerz sameerz added this to the Mar 30 - Apr 9 milestone Mar 30, 2021
@jlowe
Copy link
Member

jlowe commented Apr 5, 2021

Note that this fails slightly differently with 0.5 and cudf 0.19:

>>> from pyspark.sql import SparkSession
>>> from pyspark.sql.functions import col, split
>>> 
>>> data = [("aaa", "123 456 789"), ("bbb", "444 555 666"), ("ccc", "777 888 999")]
>>> columns = ["a","b"]
>>> df = spark.createDataFrame(data).toDF(*columns)
>>> newdf = df.withColumn('newb', split(col('b'),' ')).repartition(3)
>>> newdf.show()
21/04/05 17:17:28 ERROR Executor: Exception in task 0.0 in stage 0.0 (TID 0)/ 1]
ai.rapids.cudf.CudfException: cuDF failure at: /home/jenkins/agent/workspace/jenkins-cudf_nightly-dev-github-242-cuda10.1/cpp/include/cudf/table/row_operators.cuh:363: Attempted to compare elements of uncomparable types.
	at ai.rapids.cudf.Table.sortOrder(Native Method)
	at ai.rapids.cudf.Table.sortOrder(Table.java:1455)
	at com.nvidia.spark.rapids.GpuSorter.$anonfun$computeSortOrder$2(SortUtils.scala:224)
	at com.nvidia.spark.rapids.Arm.withResource(Arm.scala:28)
	at com.nvidia.spark.rapids.Arm.withResource$(Arm.scala:26)
	at com.nvidia.spark.rapids.GpuSorter.withResource(SortUtils.scala:65)
	at com.nvidia.spark.rapids.GpuSorter.$anonfun$computeSortOrder$1(SortUtils.scala:223)
	at com.nvidia.spark.rapids.Arm.withResource(Arm.scala:28)
	at com.nvidia.spark.rapids.Arm.withResource$(Arm.scala:26)
	at com.nvidia.spark.rapids.GpuSorter.withResource(SortUtils.scala:65)
	at com.nvidia.spark.rapids.GpuSorter.computeSortOrder(SortUtils.scala:222)
	at com.nvidia.spark.rapids.GpuSorter.$anonfun$fullySortBatch$1(SortUtils.scala:274)
	at com.nvidia.spark.rapids.Arm.withResource(Arm.scala:28)
	at com.nvidia.spark.rapids.Arm.withResource$(Arm.scala:26)
	at com.nvidia.spark.rapids.GpuSorter.withResource(SortUtils.scala:65)
	at com.nvidia.spark.rapids.GpuSorter.fullySortBatch(SortUtils.scala:269)
	at com.nvidia.spark.rapids.GpuSortEachBatchIterator.$anonfun$next$2(GpuSortExec.scala:144)
	at com.nvidia.spark.rapids.Arm.withResource(Arm.scala:28)
	at com.nvidia.spark.rapids.Arm.withResource$(Arm.scala:26)
	at com.nvidia.spark.rapids.GpuSortEachBatchIterator.withResource(GpuSortExec.scala:131)
	at com.nvidia.spark.rapids.GpuSortEachBatchIterator.$anonfun$next$1(GpuSortExec.scala:143)
	at com.nvidia.spark.rapids.Arm.withResource(Arm.scala:28)
	at com.nvidia.spark.rapids.Arm.withResource$(Arm.scala:26)
	at com.nvidia.spark.rapids.GpuSortEachBatchIterator.withResource(GpuSortExec.scala:131)
	at com.nvidia.spark.rapids.GpuSortEachBatchIterator.next(GpuSortExec.scala:142)
	at com.nvidia.spark.rapids.GpuSortEachBatchIterator.next(GpuSortExec.scala:131)
	at org.apache.spark.sql.rapids.execution.GpuShuffleExchangeExec$$anon$1.partNextBatch(GpuShuffleExchangeExec.scala:209)
	at org.apache.spark.sql.rapids.execution.GpuShuffleExchangeExec$$anon$1.hasNext(GpuShuffleExchangeExec.scala:225)
	at org.apache.spark.shuffle.sort.BypassMergeSortShuffleWriter.write(BypassMergeSortShuffleWriter.java:132)
	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:127)
	at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:462)
	at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1377)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:465)
	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:748)

It looks like the repartition is causing a partition on all columns. The partitioning strategy added late in 0.4 to match Spark CPU hashing required sorting on the partition keys, which means sorting on all columns for this use-case. In this case, one of the columns is a list-of-strings which cudf does not support as a sort-key column. The RAPIDS Accelerator should not allow this repartition to occur since it requires partitioning on list-of-string and we cannot do that if that means sorting on it as well.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
bug Something isn't working P0 Must have for release
Projects
None yet
Development

Successfully merging a pull request may close this issue.

3 participants