You signed in with another tab or window. Reload to refresh your session.You signed out in another tab or window. Reload to refresh your session.You switched accounts on another tab or window. Reload to refresh your session.Dismiss alert
Caused by: java.lang.IllegalArgumentException: requirement failed: Illegal sequence boundaries: step > 0 but start > stop
at scala.Predef$.require(Predef.scala:281)
at org.apache.spark.sql.rapids.GpuSequenceUtil$.$anonfun$checkSequenceInputs$5(collectionOperations.scala:558)
at org.apache.spark.sql.rapids.GpuSequenceUtil$.$anonfun$checkSequenceInputs$5$adapted(collectionOperations.scala:557)
at com.nvidia.spark.rapids.Arm.withResource(Arm.scala:28)
at com.nvidia.spark.rapids.Arm.withResource$(Arm.scala:26)
at org.apache.spark.sql.rapids.GpuSequenceUtil$.withResource(collectionOperations.scala:533)
at org.apache.spark.sql.rapids.GpuSequenceUtil$.$anonfun$checkSequenceInputs$2(collectionOperations.scala:557)
at org.apache.spark.sql.rapids.GpuSequenceUtil$.$anonfun$checkSequenceInputs$2$adapted(collectionOperations.scala:549)
at com.nvidia.spark.rapids.Arm.withResource(Arm.scala:28)
at com.nvidia.spark.rapids.Arm.withResource$(Arm.scala:26)
at org.apache.spark.sql.rapids.GpuSequenceUtil$.withResource(collectionOperations.scala:533)
at org.apache.spark.sql.rapids.GpuSequenceUtil$.$anonfun$checkSequenceInputs$1(collectionOperations.scala:549)
at org.apache.spark.sql.rapids.GpuSequenceUtil$.$anonfun$checkSequenceInputs$1$adapted(collectionOperations.scala:541)
at com.nvidia.spark.rapids.Arm.withResource(Arm.scala:28)
at com.nvidia.spark.rapids.Arm.withResource$(Arm.scala:26)
at org.apache.spark.sql.rapids.GpuSequenceUtil$.withResource(collectionOperations.scala:533)
at org.apache.spark.sql.rapids.GpuSequenceUtil$.checkSequenceInputs(collectionOperations.scala:541)
at org.apache.spark.sql.rapids.GpuSequenceUtil$.computeSequenceSizes(collectionOperations.scala:599)
at org.apache.spark.sql.rapids.GpuSequence.$anonfun$columnarEval$7(collectionOperations.scala:681)
at com.nvidia.spark.rapids.Arm.closeOnExcept(Arm.scala:87)
at com.nvidia.spark.rapids.Arm.closeOnExcept$(Arm.scala:85)
at org.apache.spark.sql.rapids.GpuSequence.closeOnExcept(collectionOperations.scala:653)
at org.apache.spark.sql.rapids.GpuSequence.$anonfun$columnarEval$4(collectionOperations.scala:680)
at com.nvidia.spark.rapids.Arm.withResource(Arm.scala:28)
at com.nvidia.spark.rapids.Arm.withResource$(Arm.scala:26)
at org.apache.spark.sql.rapids.GpuSequence.withResource(collectionOperations.scala:653)
at org.apache.spark.sql.rapids.GpuSequence.$anonfun$columnarEval$3(collectionOperations.scala:676)
at com.nvidia.spark.rapids.Arm.withResource(Arm.scala:37)
at com.nvidia.spark.rapids.Arm.withResource$(Arm.scala:35)
at org.apache.spark.sql.rapids.GpuSequence.withResource(collectionOperations.scala:653)
at org.apache.spark.sql.rapids.GpuSequence.$anonfun$columnarEval$1(collectionOperations.scala:672)
at com.nvidia.spark.rapids.Arm.withResource(Arm.scala:28)
at com.nvidia.spark.rapids.Arm.withResource$(Arm.scala:26)
at org.apache.spark.sql.rapids.GpuSequence.withResource(collectionOperations.scala:653)
at org.apache.spark.sql.rapids.GpuSequence.columnarEval(collectionOperations.scala:671)
at com.nvidia.spark.rapids.RapidsPluginImplicits$ReallyAGpuExpression.columnarEval(implicits.scala:34)
at com.nvidia.spark.rapids.GpuConditionalExpression.$anonfun$computeIfElse$2(conditionalExpressions.scala:159)
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.GpuCaseWhen.withResource(conditionalExpressions.scala:309)
at com.nvidia.spark.rapids.GpuConditionalExpression.$anonfun$computeIfElse$1(conditionalExpressions.scala:158)
at com.nvidia.spark.rapids.Arm.withResourceIfAllowed(Arm.scala:73)
at com.nvidia.spark.rapids.Arm.withResourceIfAllowed$(Arm.scala:71)
at com.nvidia.spark.rapids.GpuCaseWhen.withResourceIfAllowed(conditionalExpressions.scala:309)
at com.nvidia.spark.rapids.GpuConditionalExpression.computeIfElse(conditionalExpressions.scala:157)
at com.nvidia.spark.rapids.GpuConditionalExpression.computeIfElse$(conditionalExpressions.scala:152)
at com.nvidia.spark.rapids.GpuCaseWhen.computeIfElse(conditionalExpressions.scala:309)
at com.nvidia.spark.rapids.GpuCaseWhen.$anonfun$columnarEval$6(conditionalExpressions.scala:362)
at scala.collection.IndexedSeqOptimized.foldRight(IndexedSeqOptimized.scala:65)
at scala.collection.IndexedSeqOptimized.foldRight$(IndexedSeqOptimized.scala:72)
at scala.collection.mutable.ArrayBuffer.foldRight(ArrayBuffer.scala:49)
at com.nvidia.spark.rapids.GpuCaseWhen.columnarEval(conditionalExpressions.scala:360)
at com.nvidia.spark.rapids.RapidsPluginImplicits$ReallyAGpuExpression.columnarEval(implicits.scala:34)
at com.nvidia.spark.rapids.GpuExpressionsUtils$.columnarEvalToColumn(GpuExpressions.scala:93)
at com.nvidia.spark.rapids.GpuUnaryExpression.columnarEval(GpuExpressions.scala:202)
at com.nvidia.spark.rapids.RapidsPluginImplicits$ReallyAGpuExpression.columnarEval(implicits.scala:34)
at com.nvidia.spark.rapids.GpuAlias.columnarEval(namedExpressions.scala:109)
at com.nvidia.spark.rapids.RapidsPluginImplicits$ReallyAGpuExpression.columnarEval(implicits.scala:34)
at com.nvidia.spark.rapids.GpuExpressionsUtils$.columnarEvalToColumn(GpuExpressions.scala:93)
at com.nvidia.spark.rapids.GpuProjectExec$.projectSingle(basicPhysicalOperators.scala:102)
at com.nvidia.spark.rapids.GpuProjectExec$.$anonfun$project$1(basicPhysicalOperators.scala:109)
at com.nvidia.spark.rapids.RapidsPluginImplicits$MapsSafely.$anonfun$safeMap$1(implicits.scala:162)
at com.nvidia.spark.rapids.RapidsPluginImplicits$MapsSafely.$anonfun$safeMap$1$adapted(implicits.scala:159)
at scala.collection.immutable.List.foreach(List.scala:392)
at com.nvidia.spark.rapids.RapidsPluginImplicits$MapsSafely.safeMap(implicits.scala:159)
at com.nvidia.spark.rapids.RapidsPluginImplicits$AutoCloseableProducingSeq.safeMap(implicits.scala:194)
at com.nvidia.spark.rapids.GpuProjectExec$.project(basicPhysicalOperators.scala:109)
at com.nvidia.spark.rapids.GpuProjectExec$.projectAndClose(basicPhysicalOperators.scala:73)
at com.nvidia.spark.rapids.GpuProjectExec.$anonfun$doExecuteColumnar$1(basicPhysicalOperators.scala:149)
at scala.collection.Iterator$$anon$10.next(Iterator.scala:459)
at com.nvidia.spark.rapids.GpuBaseLimitExec$$anon$1.next(limit.scala:71)
at com.nvidia.spark.rapids.GpuBaseLimitExec$$anon$1.next(limit.scala:65)
at org.apache.spark.sql.rapids.execution.GpuShuffleExchangeExecBase$$anon$1.partNextBatch(GpuShuffleExchangeExecBase.scala:288)
at org.apache.spark.sql.rapids.execution.GpuShuffleExchangeExecBase$$anon$1.hasNext(GpuShuffleExchangeExecBase.scala:304)
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.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
... 1 more
CPU Spark works fine.
Basically the column "languagesAtWork" has some empty array value, seems even if "when clause" is FALSE, we are evaluating the expression(which contains sequence).
The text was updated successfully, but these errors were encountered:
Below is a minimum repro in pyspark:
Env:
22.02GA jars
or
22.04Snapshot jars(20220322)
Reproduced on Databricks 9.1ML GPU and also local workstation.
Repro
Error:
Full stacktrace:
CPU Spark works fine.
Basically the column "languagesAtWork" has some empty array value, seems even if "when clause" is FALSE, we are evaluating the expression(which contains sequence).
The text was updated successfully, but these errors were encountered: