diff --git a/integration_tests/src/main/python/sort_test.py b/integration_tests/src/main/python/sort_test.py index 9d90a0dd66d..4f08673944d 100644 --- a/integration_tests/src/main/python/sort_test.py +++ b/integration_tests/src/main/python/sort_test.py @@ -36,11 +36,11 @@ def test_single_orderby(data_gen, order): @pytest.mark.parametrize('shuffle_parts', [ pytest.param(1), - pytest.param(200, marks=pytest.mark.xfail(reason="https://github.com/NVIDIA/spark-rapids/issues/1607")) + pytest.param(200) ]) @pytest.mark.parametrize('stable_sort', [ pytest.param(True), - pytest.param(False, marks=pytest.mark.xfail(reason="https://github.com/NVIDIA/spark-rapids/issues/1607")) + pytest.param(False) ]) @pytest.mark.parametrize('data_gen', [ pytest.param(all_basic_struct_gen), @@ -64,7 +64,6 @@ def test_single_orderby(data_gen, order): def test_single_nested_orderby_plain(data_gen, order, shuffle_parts, stable_sort): assert_gpu_and_cpu_are_equal_collect( lambda spark : unary_op_df(spark, data_gen).orderBy(order), - # TODO no interference with range partition once implemented conf = { **allow_negative_scale_of_decimal_conf, **{ diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala index a39647a0e02..a0cdfd7c7d0 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala @@ -2531,14 +2531,6 @@ object GpuOverrides { override val childExprs: Seq[BaseExprMeta[_]] = rp.ordering.map(GpuOverrides.wrapExpr(_, conf, Some(this))) - override def tagPartForGpu() { - val numPartitions = rp.numPartitions - if (numPartitions > 1 && rp.ordering.exists(so => isStructType(so.dataType))) { - willNotWorkOnGpu("only single partition sort is supported for nested types, " + - s"actual partitions: $numPartitions") - } - } - override def convertToGpu(): GpuPartitioning = { if (rp.numPartitions > 1) { val gpuOrdering = childExprs.map(_.convertToGpu()).asInstanceOf[Seq[SortOrder]] @@ -2793,14 +2785,7 @@ object GpuOverrides { // The types below are allowed as inputs and outputs. ExecChecks(pluginSupportedOrderableSig + (TypeSig.ARRAY + TypeSig.STRUCT).nested(), TypeSig.all), - (sort, conf, p, r) => new GpuSortMeta(sort, conf, p, r) { - override def tagPlanForGpu() { - if (!conf.stableSort && sort.sortOrder.exists(so => isStructType(so.dataType))) { - willNotWorkOnGpu("it's disabled for nested types " + - s"unless ${RapidsConf.STABLE_SORT.key} is true") - } - } - }), + (sort, conf, p, r) => new GpuSortMeta(sort, conf, p, r)), exec[ExpandExec]( "The backend for the expand operator", ExecChecks(TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL, TypeSig.all), diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuTransitionOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuTransitionOverrides.scala index 07f5492af33..6b0674f35d9 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuTransitionOverrides.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuTransitionOverrides.scala @@ -424,8 +424,7 @@ class GpuTransitionOverrides extends Rule[SparkPlan] { case _: GpuColumnarToRowExecParent => () // Ignored case _: ExecutedCommandExec => () // Ignored case _: RDDScanExec => () // Ignored - case shuffleExchange: ShuffleExchangeExec if conf.cpuRangePartitioningPermitted - || !shuffleExchange.outputPartitioning.isInstanceOf[RangePartitioning] => { + case _: ShuffleExchangeExec if conf.cpuRangePartitioningPermitted => { // Ignored for now, we don't force it to the GPU if // children are not on the gpu }