diff --git a/backends-velox/src/main/java/org/apache/gluten/utils/VeloxBatchAppender.java b/backends-velox/src/main/java/org/apache/gluten/utils/VeloxBatchResizer.java similarity index 82% rename from backends-velox/src/main/java/org/apache/gluten/utils/VeloxBatchAppender.java rename to backends-velox/src/main/java/org/apache/gluten/utils/VeloxBatchResizer.java index 32b2289471f9..e2f11cd3510b 100644 --- a/backends-velox/src/main/java/org/apache/gluten/utils/VeloxBatchAppender.java +++ b/backends-velox/src/main/java/org/apache/gluten/utils/VeloxBatchResizer.java @@ -25,13 +25,13 @@ import java.util.Iterator; -public final class VeloxBatchAppender { +public final class VeloxBatchResizer { public static ColumnarBatchOutIterator create( - int minOutputBatchSize, Iterator in) { - final Runtime runtime = Runtimes.contextInstance("VeloxBatchAppender"); + int minOutputBatchSize, int maxOutputBatchSize, Iterator in) { + final Runtime runtime = Runtimes.contextInstance("VeloxBatchResizer"); long outHandle = - VeloxBatchAppenderJniWrapper.create(runtime) - .create(minOutputBatchSize, new ColumnarBatchInIterator(in)); + VeloxBatchResizerJniWrapper.create(runtime) + .create(minOutputBatchSize, maxOutputBatchSize, new ColumnarBatchInIterator(in)); return new ColumnarBatchOutIterator(runtime, outHandle); } } diff --git a/backends-velox/src/main/java/org/apache/gluten/utils/VeloxBatchAppenderJniWrapper.java b/backends-velox/src/main/java/org/apache/gluten/utils/VeloxBatchResizerJniWrapper.java similarity index 75% rename from backends-velox/src/main/java/org/apache/gluten/utils/VeloxBatchAppenderJniWrapper.java rename to backends-velox/src/main/java/org/apache/gluten/utils/VeloxBatchResizerJniWrapper.java index 231e65553981..3011ced2ab37 100644 --- a/backends-velox/src/main/java/org/apache/gluten/utils/VeloxBatchAppenderJniWrapper.java +++ b/backends-velox/src/main/java/org/apache/gluten/utils/VeloxBatchResizerJniWrapper.java @@ -20,15 +20,15 @@ import org.apache.gluten.exec.RuntimeAware; import org.apache.gluten.vectorized.ColumnarBatchInIterator; -public class VeloxBatchAppenderJniWrapper implements RuntimeAware { +public class VeloxBatchResizerJniWrapper implements RuntimeAware { private final Runtime runtime; - private VeloxBatchAppenderJniWrapper(Runtime runtime) { + private VeloxBatchResizerJniWrapper(Runtime runtime) { this.runtime = runtime; } - public static VeloxBatchAppenderJniWrapper create(Runtime runtime) { - return new VeloxBatchAppenderJniWrapper(runtime); + public static VeloxBatchResizerJniWrapper create(Runtime runtime) { + return new VeloxBatchResizerJniWrapper(runtime); } @Override @@ -36,5 +36,6 @@ public long handle() { return runtime.getHandle(); } - public native long create(int minOutputBatchSize, ColumnarBatchInIterator itr); + public native long create( + int minOutputBatchSize, int maxOutputBatchSize, ColumnarBatchInIterator itr); } diff --git a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxSparkPlanExecApi.scala b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxSparkPlanExecApi.scala index a32b00196aeb..9e0d85e25a36 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxSparkPlanExecApi.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxSparkPlanExecApi.scala @@ -352,9 +352,10 @@ class VeloxSparkPlanExecApi extends SparkPlanExecApi { plan match { case shuffle: ColumnarShuffleExchangeExec if !shuffle.useSortBasedShuffle && - GlutenConfig.getConf.veloxCoalesceBatchesBeforeShuffle => + GlutenConfig.getConf.veloxResizeBatchesShuffleInput => + val range = GlutenConfig.getConf.veloxResizeBatchesShuffleInputRange val appendBatches = - VeloxAppendBatchesExec(shuffle.child, GlutenConfig.getConf.veloxMinBatchSizeForShuffle) + VeloxResizeBatchesExec(shuffle.child, range.min, range.max) shuffle.withNewChildren(Seq(appendBatches)) case _ => plan } diff --git a/backends-velox/src/main/scala/org/apache/gluten/execution/VeloxAppendBatchesExec.scala b/backends-velox/src/main/scala/org/apache/gluten/execution/VeloxResizeBatchesExec.scala similarity index 86% rename from backends-velox/src/main/scala/org/apache/gluten/execution/VeloxAppendBatchesExec.scala rename to backends-velox/src/main/scala/org/apache/gluten/execution/VeloxResizeBatchesExec.scala index 4b4db703de7a..d05e68f857db 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/execution/VeloxAppendBatchesExec.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/execution/VeloxResizeBatchesExec.scala @@ -17,7 +17,7 @@ package org.apache.gluten.execution import org.apache.gluten.extension.GlutenPlan -import org.apache.gluten.utils.VeloxBatchAppender +import org.apache.gluten.utils.VeloxBatchResizer import org.apache.gluten.utils.iterator.Iterators import org.apache.spark.rdd.RDD @@ -33,10 +33,13 @@ import java.util.concurrent.atomic.AtomicLong import scala.collection.JavaConverters._ /** - * An operator to coalesce input batches by appending the later batches to the one that comes - * earlier. + * An operator to resize input batches by appending the later batches to the one that comes earlier, + * or splitting one batch to smaller ones. */ -case class VeloxAppendBatchesExec(override val child: SparkPlan, minOutputBatchSize: Int) +case class VeloxResizeBatchesExec( + override val child: SparkPlan, + minOutputBatchSize: Int, + maxOutputBatchSize: Int) extends GlutenPlan with UnaryExecNode { @@ -45,7 +48,7 @@ case class VeloxAppendBatchesExec(override val child: SparkPlan, minOutputBatchS "numInputBatches" -> SQLMetrics.createMetric(sparkContext, "number of input batches"), "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), "numOutputBatches" -> SQLMetrics.createMetric(sparkContext, "number of output batches"), - "appendTime" -> SQLMetrics.createTimingMetric(sparkContext, "time to append batches") + "selfTime" -> SQLMetrics.createTimingMetric(sparkContext, "time to append / split batches") ) override def supportsColumnar: Boolean = true @@ -56,15 +59,15 @@ case class VeloxAppendBatchesExec(override val child: SparkPlan, minOutputBatchS val numInputBatches = longMetric("numInputBatches") val numOutputRows = longMetric("numOutputRows") val numOutputBatches = longMetric("numOutputBatches") - val appendTime = longMetric("appendTime") + val selfTime = longMetric("selfTime") child.executeColumnar().mapPartitions { in => // Append millis = Out millis - In millis. val appendMillis = new AtomicLong(0L) - - val appender = VeloxBatchAppender.create( + val appender = VeloxBatchResizer.create( minOutputBatchSize, + maxOutputBatchSize, Iterators .wrap(in) .collectReadMillis(inMillis => appendMillis.getAndAdd(-inMillis)) @@ -84,7 +87,7 @@ case class VeloxAppendBatchesExec(override val child: SparkPlan, minOutputBatchS .recyclePayload(_.close()) .recycleIterator { appender.close() - appendTime += appendMillis.get() + selfTime += appendMillis.get() } .create() .map { diff --git a/backends-velox/src/main/scala/org/apache/gluten/extension/FlushableHashAggregateRule.scala b/backends-velox/src/main/scala/org/apache/gluten/extension/FlushableHashAggregateRule.scala index 777bf553856e..3137d6e6aef5 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/extension/FlushableHashAggregateRule.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/extension/FlushableHashAggregateRule.scala @@ -77,7 +77,7 @@ case class FlushableHashAggregateRule(session: SparkSession) extends Rule[SparkP private def canPropagate(plan: SparkPlan): Boolean = plan match { case _: ProjectExecTransformer => true - case _: VeloxAppendBatchesExec => true + case _: VeloxResizeBatchesExec => true case _ => false } } diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/1.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/1.txt index 53edb933c1fb..deb09c6c1c0f 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/1.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/1.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (30) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ RegularHashAggregateExecTransformer (12) +- ^ InputIteratorTransformer (11) +- ShuffleQueryStage (9) +- ColumnarExchange (8) - +- VeloxAppendBatches (7) + +- VeloxResizeBatches (7) +- ^ ProjectExecTransformer (5) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) @@ -58,9 +58,9 @@ Input [17]: [l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] Arguments: false -(7) VeloxAppendBatches +(7) VeloxResizeBatches Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] -Arguments: X +Arguments: X, X (8) ColumnarExchange Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] @@ -87,9 +87,9 @@ Results [10]: [l_returnflag#X, l_linestatus#X, sum(l_quantity#X)#X AS sum_qty#X, Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/10.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/10.txt index 29f28ac26a6b..2b4b7266faa6 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/10.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/10.txt @@ -8,7 +8,7 @@ AdaptiveSparkPlan (68) +- ^ InputIteratorTransformer (39) +- ShuffleQueryStage (37) +- ColumnarExchange (36) - +- VeloxAppendBatches (35) + +- VeloxResizeBatches (35) +- ^ ProjectExecTransformer (33) +- ^ FlushableHashAggregateExecTransformer (32) +- ^ ProjectExecTransformer (31) @@ -210,9 +210,9 @@ Input [9]: [c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] Arguments: false -(35) VeloxAppendBatches +(35) VeloxResizeBatches Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (36) ColumnarExchange Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/11.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/11.txt index b8f466eb9456..356e7b09de8e 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/11.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/11.txt @@ -6,13 +6,13 @@ AdaptiveSparkPlan (60) +- ^ InputIteratorTransformer (37) +- ShuffleQueryStage (35) +- ColumnarExchange (34) - +- VeloxAppendBatches (33) + +- VeloxResizeBatches (33) +- ^ FilterExecTransformer (31) +- ^ RegularHashAggregateExecTransformer (30) +- ^ InputIteratorTransformer (29) +- ShuffleQueryStage (27) +- ColumnarExchange (26) - +- VeloxAppendBatches (25) + +- VeloxResizeBatches (25) +- ^ ProjectExecTransformer (23) +- ^ FlushableHashAggregateExecTransformer (22) +- ^ ProjectExecTransformer (21) @@ -160,9 +160,9 @@ Input [3]: [ps_partkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] Arguments: false -(25) VeloxAppendBatches +(25) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (26) ColumnarExchange Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] @@ -193,9 +193,9 @@ Arguments: (isnotnull(value#X) AND (cast(value#X as decimal(38,6)) > Subquery su Input [2]: [ps_partkey#X, value#X] Arguments: false -(33) VeloxAppendBatches +(33) VeloxResizeBatches Input [2]: [ps_partkey#X, value#X] -Arguments: X +Arguments: X, X (34) ColumnarExchange Input [2]: [ps_partkey#X, value#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/12.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/12.txt index 1e60a93910a7..824d4e33aeb7 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/12.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/12.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (44) +- ^ InputIteratorTransformer (27) +- ShuffleQueryStage (25) +- ColumnarExchange (24) - +- VeloxAppendBatches (23) + +- VeloxResizeBatches (23) +- ^ RegularHashAggregateExecTransformer (21) +- ^ InputIteratorTransformer (20) +- ShuffleQueryStage (18) +- ColumnarExchange (17) - +- VeloxAppendBatches (16) + +- VeloxResizeBatches (16) +- ^ ProjectExecTransformer (14) +- ^ FlushableHashAggregateExecTransformer (13) +- ^ ProjectExecTransformer (12) @@ -108,9 +108,9 @@ Input [3]: [l_shipmode#X, sum#X, sum#X] Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] Arguments: false -(16) VeloxAppendBatches +(16) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] -Arguments: X +Arguments: X, X (17) ColumnarExchange Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] @@ -137,9 +137,9 @@ Results [3]: [l_shipmode#X, sum(CASE WHEN ((o_orderpriority#X = 1-URGENT) OR (o_ Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] Arguments: false -(23) VeloxAppendBatches +(23) VeloxResizeBatches Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] -Arguments: X +Arguments: X, X (24) ColumnarExchange Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/13.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/13.txt index c525944e1262..0387f9bdbd1c 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/13.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/13.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (55) +- ^ InputIteratorTransformer (36) +- ShuffleQueryStage (34) +- ColumnarExchange (33) - +- VeloxAppendBatches (32) + +- VeloxResizeBatches (32) +- ^ RegularHashAggregateExecTransformer (30) +- ^ InputIteratorTransformer (29) +- ShuffleQueryStage (27) +- ColumnarExchange (26) - +- VeloxAppendBatches (25) + +- VeloxResizeBatches (25) +- ^ ProjectExecTransformer (23) +- ^ FlushableHashAggregateExecTransformer (22) +- ^ ProjectExecTransformer (21) @@ -19,7 +19,7 @@ AdaptiveSparkPlan (55) +- ^ InputIteratorTransformer (19) +- ShuffleQueryStage (17) +- ColumnarExchange (16) - +- VeloxAppendBatches (15) + +- VeloxResizeBatches (15) +- ^ ProjectExecTransformer (13) +- ^ FlushableHashAggregateExecTransformer (12) +- ^ ProjectExecTransformer (11) @@ -112,9 +112,9 @@ Input [2]: [c_custkey#X, count#X] Input [3]: [hash_partition_key#X, c_custkey#X, count#X] Arguments: false -(15) VeloxAppendBatches +(15) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_custkey#X, count#X] -Arguments: X +Arguments: X, X (16) ColumnarExchange Input [3]: [hash_partition_key#X, c_custkey#X, count#X] @@ -156,9 +156,9 @@ Input [2]: [c_count#X, count#X] Input [3]: [hash_partition_key#X, c_count#X, count#X] Arguments: false -(25) VeloxAppendBatches +(25) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_count#X, count#X] -Arguments: X +Arguments: X, X (26) ColumnarExchange Input [3]: [hash_partition_key#X, c_count#X, count#X] @@ -185,9 +185,9 @@ Results [2]: [c_count#X, count(1)#X AS custdist#X] Input [2]: [c_count#X, custdist#X] Arguments: false -(32) VeloxAppendBatches +(32) VeloxResizeBatches Input [2]: [c_count#X, custdist#X] -Arguments: X +Arguments: X, X (33) ColumnarExchange Input [2]: [c_count#X, custdist#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/14.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/14.txt index fbfcf07030eb..354637343992 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/14.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/14.txt @@ -7,7 +7,7 @@ AdaptiveSparkPlan (35) +- ^ InputIteratorTransformer (19) +- ShuffleQueryStage (17) +- ColumnarExchange (16) - +- VeloxAppendBatches (15) + +- VeloxResizeBatches (15) +- ^ FlushableHashAggregateExecTransformer (13) +- ^ ProjectExecTransformer (12) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (11) @@ -97,9 +97,9 @@ Results [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] Arguments: false -(15) VeloxAppendBatches +(15) VeloxResizeBatches Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (16) ColumnarExchange Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/15.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/15.txt index b5073901c620..2e51e95f0761 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/15.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/15.txt @@ -6,7 +6,7 @@ AdaptiveSparkPlan (46) +- ^ InputIteratorTransformer (28) +- ShuffleQueryStage (26) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ ProjectExecTransformer (22) +- ^ BroadcastHashJoinExecTransformer Inner BuildLeft (21) :- ^ InputIteratorTransformer (7) @@ -19,7 +19,7 @@ AdaptiveSparkPlan (46) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FlushableHashAggregateExecTransformer (11) +- ^ ProjectExecTransformer (10) @@ -101,9 +101,9 @@ Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] @@ -143,9 +143,9 @@ Input [6]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, supplier_no#X, total_ Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/16.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/16.txt index f3cef927551c..5f255fdaee88 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/16.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/16.txt @@ -6,19 +6,19 @@ AdaptiveSparkPlan (59) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33) +- ColumnarExchange (32) - +- VeloxAppendBatches (31) + +- VeloxResizeBatches (31) +- ^ RegularHashAggregateExecTransformer (29) +- ^ InputIteratorTransformer (28) +- ShuffleQueryStage (26) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ ProjectExecTransformer (22) +- ^ FlushableHashAggregateExecTransformer (21) +- ^ RegularHashAggregateExecTransformer (20) +- ^ InputIteratorTransformer (19) +- ShuffleQueryStage (17) +- ColumnarExchange (16) - +- VeloxAppendBatches (15) + +- VeloxResizeBatches (15) +- ^ ProjectExecTransformer (13) +- ^ FlushableHashAggregateExecTransformer (12) +- ^ ProjectExecTransformer (11) @@ -117,9 +117,9 @@ Input [4]: [p_brand#X, p_type#X, p_size#X, ps_suppkey#X] Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] Arguments: false -(15) VeloxAppendBatches +(15) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] -Arguments: X +Arguments: X, X (16) ColumnarExchange Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] @@ -157,9 +157,9 @@ Input [4]: [p_brand#X, p_type#X, p_size#X, count#X] Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] @@ -186,9 +186,9 @@ Results [4]: [p_brand#X, p_type#X, p_size#X, count(ps_suppkey#X)#X AS supplier_c Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/17.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/17.txt index 7c5359849d4e..7f4e4e666c0d 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/17.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/17.txt @@ -7,7 +7,7 @@ AdaptiveSparkPlan (36) +- ^ InputIteratorTransformer (11) +- ShuffleQueryStage (9) +- ColumnarExchange (8) - +- VeloxAppendBatches (7) + +- VeloxResizeBatches (7) +- ^ FlushableHashAggregateExecTransformer (5) +- ^ InputIteratorTransformer (4) +- RowToVeloxColumnar (2) @@ -59,9 +59,9 @@ Results [2]: [sum#X, isEmpty#X] Input [2]: [sum#X, isEmpty#X] Arguments: false -(7) VeloxAppendBatches +(7) VeloxResizeBatches Input [2]: [sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (8) ColumnarExchange Input [2]: [sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/18.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/18.txt index a0a6e26a6ac3..6ee709d6b5a6 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/18.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/18.txt @@ -7,7 +7,7 @@ AdaptiveSparkPlan (88) +- ^ InputIteratorTransformer (51) +- ShuffleQueryStage (49) +- ColumnarExchange (48) - +- VeloxAppendBatches (47) + +- VeloxResizeBatches (47) +- ^ ProjectExecTransformer (45) +- ^ FlushableHashAggregateExecTransformer (44) +- ^ ProjectExecTransformer (43) @@ -31,7 +31,7 @@ AdaptiveSparkPlan (88) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16) : +- ColumnarExchange (15) - : +- VeloxAppendBatches (14) + : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ FlushableHashAggregateExecTransformer (11) : +- ^ Scan parquet (10) @@ -140,9 +140,9 @@ Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] @@ -276,9 +276,9 @@ Input [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, Input [8]: [hash_partition_key#X, c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] Arguments: false -(47) VeloxAppendBatches +(47) VeloxResizeBatches Input [8]: [hash_partition_key#X, c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (48) ColumnarExchange Input [8]: [hash_partition_key#X, c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/19.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/19.txt index d2ab9979f8a3..0555720ef082 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/19.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/19.txt @@ -6,7 +6,7 @@ AdaptiveSparkPlan (34) +- ^ InputIteratorTransformer (19) +- ShuffleQueryStage (17) +- ColumnarExchange (16) - +- VeloxAppendBatches (15) + +- VeloxResizeBatches (15) +- ^ FlushableHashAggregateExecTransformer (13) +- ^ ProjectExecTransformer (12) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (11) @@ -96,9 +96,9 @@ Results [2]: [sum#X, isEmpty#X] Input [2]: [sum#X, isEmpty#X] Arguments: false -(15) VeloxAppendBatches +(15) VeloxResizeBatches Input [2]: [sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (16) ColumnarExchange Input [2]: [sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/20.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/20.txt index a1f1bb51cb98..fef19a90199b 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/20.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/20.txt @@ -6,7 +6,7 @@ AdaptiveSparkPlan (112) +- ^ InputIteratorTransformer (70) +- ShuffleQueryStage (68) +- ColumnarExchange (67) - +- VeloxAppendBatches (66) + +- VeloxResizeBatches (66) +- ^ ProjectExecTransformer (64) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (63) :- ^ ProjectExecTransformer (54) @@ -15,7 +15,7 @@ AdaptiveSparkPlan (112) : : +- AQEShuffleRead (8) : : +- ShuffleQueryStage (7) : : +- ColumnarExchange (6) - : : +- VeloxAppendBatches (5) + : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ NoopFilter (2) : : +- ^ Scan parquet (1) @@ -42,7 +42,7 @@ AdaptiveSparkPlan (112) : +- ^ InputIteratorTransformer (42) : +- ShuffleQueryStage (40) : +- ColumnarExchange (39) - : +- VeloxAppendBatches (38) + : +- VeloxResizeBatches (38) : +- ^ ProjectExecTransformer (36) : +- ^ FlushableHashAggregateExecTransformer (35) : +- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (34) @@ -118,9 +118,9 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] @@ -255,9 +255,9 @@ Input [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] Input [5]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] Arguments: false -(38) VeloxAppendBatches +(38) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (39) ColumnarExchange Input [5]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] @@ -370,9 +370,9 @@ Input [4]: [s_name#X, s_address#X, s_nationkey#X, n_nationkey#X] Input [2]: [s_name#X, s_address#X] Arguments: false -(66) VeloxAppendBatches +(66) VeloxResizeBatches Input [2]: [s_name#X, s_address#X] -Arguments: X +Arguments: X, X (67) ColumnarExchange Input [2]: [s_name#X, s_address#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/21.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/21.txt index eb3f64d3efcc..b4158e426d5c 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/21.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/21.txt @@ -7,7 +7,7 @@ AdaptiveSparkPlan (93) +- ^ InputIteratorTransformer (56) +- ShuffleQueryStage (54) +- ColumnarExchange (53) - +- VeloxAppendBatches (52) + +- VeloxResizeBatches (52) +- ^ ProjectExecTransformer (50) +- ^ FlushableHashAggregateExecTransformer (49) +- ^ ProjectExecTransformer (48) @@ -302,9 +302,9 @@ Input [2]: [s_name#X, count#X] Input [3]: [hash_partition_key#X, s_name#X, count#X] Arguments: false -(52) VeloxAppendBatches +(52) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_name#X, count#X] -Arguments: X +Arguments: X, X (53) ColumnarExchange Input [3]: [hash_partition_key#X, s_name#X, count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/22.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/22.txt index 4c149d9bb43e..9db514620fa3 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/22.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/22.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (40) +- ^ InputIteratorTransformer (25) +- ShuffleQueryStage (23) +- ColumnarExchange (22) - +- VeloxAppendBatches (21) + +- VeloxResizeBatches (21) +- ^ RegularHashAggregateExecTransformer (19) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FlushableHashAggregateExecTransformer (11) +- ^ ProjectExecTransformer (10) @@ -95,9 +95,9 @@ Input [4]: [cntrycode#X, count#X, sum#X, isEmpty#X] Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] @@ -124,9 +124,9 @@ Results [3]: [cntrycode#X, count(1)#X AS numcust#X, sum(c_acctbal#X)#X AS totacc Input [3]: [cntrycode#X, numcust#X, totacctbal#X] Arguments: false -(21) VeloxAppendBatches +(21) VeloxResizeBatches Input [3]: [cntrycode#X, numcust#X, totacctbal#X] -Arguments: X +Arguments: X, X (22) ColumnarExchange Input [3]: [cntrycode#X, numcust#X, totacctbal#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/3.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/3.txt index df2a6de8d184..e9e3670db842 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/3.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/3.txt @@ -8,7 +8,7 @@ AdaptiveSparkPlan (54) +- ^ InputIteratorTransformer (30) +- ShuffleQueryStage (28) +- ColumnarExchange (27) - +- VeloxAppendBatches (26) + +- VeloxResizeBatches (26) +- ^ ProjectExecTransformer (24) +- ^ FlushableHashAggregateExecTransformer (23) +- ^ ProjectExecTransformer (22) @@ -160,9 +160,9 @@ Input [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] Input [6]: [hash_partition_key#X, l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] Arguments: false -(26) VeloxAppendBatches +(26) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (27) ColumnarExchange Input [6]: [hash_partition_key#X, l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/4.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/4.txt index 34cb6467cbc2..0195ac272159 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/4.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/4.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (46) +- ^ InputIteratorTransformer (28) +- ShuffleQueryStage (26) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ RegularHashAggregateExecTransformer (22) +- ^ InputIteratorTransformer (21) +- ShuffleQueryStage (19) +- ColumnarExchange (18) - +- VeloxAppendBatches (17) + +- VeloxResizeBatches (17) +- ^ ProjectExecTransformer (15) +- ^ FlushableHashAggregateExecTransformer (14) +- ^ ProjectExecTransformer (13) @@ -114,9 +114,9 @@ Input [2]: [o_orderpriority#X, count#X] Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] Arguments: false -(17) VeloxAppendBatches +(17) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] -Arguments: X +Arguments: X, X (18) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] @@ -143,9 +143,9 @@ Results [2]: [o_orderpriority#X, count(1)#X AS order_count#X] Input [2]: [o_orderpriority#X, order_count#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [2]: [o_orderpriority#X, order_count#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [2]: [o_orderpriority#X, order_count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/5.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/5.txt index 2c9ba4c938d4..277d203a869c 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/5.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/5.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (102) +- ^ InputIteratorTransformer (64) +- ShuffleQueryStage (62) +- ColumnarExchange (61) - +- VeloxAppendBatches (60) + +- VeloxResizeBatches (60) +- ^ RegularHashAggregateExecTransformer (58) +- ^ InputIteratorTransformer (57) +- ShuffleQueryStage (55) +- ColumnarExchange (54) - +- VeloxAppendBatches (53) + +- VeloxResizeBatches (53) +- ^ ProjectExecTransformer (51) +- ^ FlushableHashAggregateExecTransformer (50) +- ^ ProjectExecTransformer (49) @@ -314,9 +314,9 @@ Input [3]: [n_name#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] Arguments: false -(53) VeloxAppendBatches +(53) VeloxResizeBatches Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (54) ColumnarExchange Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] @@ -343,9 +343,9 @@ Results [2]: [n_name#X, sum(CheckOverflow((promote_precision(cast(l_extendedpric Input [2]: [n_name#X, revenue#X] Arguments: false -(60) VeloxAppendBatches +(60) VeloxResizeBatches Input [2]: [n_name#X, revenue#X] -Arguments: X +Arguments: X, X (61) ColumnarExchange Input [2]: [n_name#X, revenue#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/6.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/6.txt index b6f876d48e5a..f30bcb0da801 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/6.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/6.txt @@ -6,7 +6,7 @@ AdaptiveSparkPlan (20) +- ^ InputIteratorTransformer (10) +- ShuffleQueryStage (8) +- ColumnarExchange (7) - +- VeloxAppendBatches (6) + +- VeloxResizeBatches (6) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ NoopFilter (2) @@ -46,9 +46,9 @@ Results [2]: [sum#X, isEmpty#X] Input [2]: [sum#X, isEmpty#X] Arguments: false -(6) VeloxAppendBatches +(6) VeloxResizeBatches Input [2]: [sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (7) ColumnarExchange Input [2]: [sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/7.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/7.txt index 8cf20f7694c9..35de09c78d48 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/7.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/7.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (95) +- ^ InputIteratorTransformer (59) +- ShuffleQueryStage (57) +- ColumnarExchange (56) - +- VeloxAppendBatches (55) + +- VeloxResizeBatches (55) +- ^ RegularHashAggregateExecTransformer (53) +- ^ InputIteratorTransformer (52) +- ShuffleQueryStage (50) +- ColumnarExchange (49) - +- VeloxAppendBatches (48) + +- VeloxResizeBatches (48) +- ^ ProjectExecTransformer (46) +- ^ FlushableHashAggregateExecTransformer (45) +- ^ ProjectExecTransformer (44) @@ -284,9 +284,9 @@ Input [5]: [supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] @@ -313,9 +313,9 @@ Results [4]: [supp_nation#X, cust_nation#X, l_year#X, sum(volume#X)#X AS revenue Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] Arguments: false -(55) VeloxAppendBatches +(55) VeloxResizeBatches Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] -Arguments: X +Arguments: X, X (56) ColumnarExchange Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/8.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/8.txt index 045e283bd036..e7fd9789b9a3 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/8.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/8.txt @@ -6,13 +6,13 @@ AdaptiveSparkPlan (131) +- ^ InputIteratorTransformer (83) +- ShuffleQueryStage (81) +- ColumnarExchange (80) - +- VeloxAppendBatches (79) + +- VeloxResizeBatches (79) +- ^ ProjectExecTransformer (77) +- ^ RegularHashAggregateExecTransformer (76) +- ^ InputIteratorTransformer (75) +- ShuffleQueryStage (73) +- ColumnarExchange (72) - +- VeloxAppendBatches (71) + +- VeloxResizeBatches (71) +- ^ ProjectExecTransformer (69) +- ^ FlushableHashAggregateExecTransformer (68) +- ^ ProjectExecTransformer (67) @@ -415,9 +415,9 @@ Input [5]: [o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] Arguments: false -(71) VeloxAppendBatches +(71) VeloxResizeBatches Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (72) ColumnarExchange Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] @@ -448,9 +448,9 @@ Input [3]: [o_year#X, sum(CASE WHEN (nation#X = BRAZIL) THEN volume#X ELSE 0.000 Input [2]: [o_year#X, mkt_share#X] Arguments: false -(79) VeloxAppendBatches +(79) VeloxResizeBatches Input [2]: [o_year#X, mkt_share#X] -Arguments: X +Arguments: X, X (80) ColumnarExchange Input [2]: [o_year#X, mkt_share#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/9.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/9.txt index abdd1d7b9369..813cf616f41c 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/9.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/9.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (100) +- ^ InputIteratorTransformer (63) +- ShuffleQueryStage (61) +- ColumnarExchange (60) - +- VeloxAppendBatches (59) + +- VeloxResizeBatches (59) +- ^ RegularHashAggregateExecTransformer (57) +- ^ InputIteratorTransformer (56) +- ShuffleQueryStage (54) +- ColumnarExchange (53) - +- VeloxAppendBatches (52) + +- VeloxResizeBatches (52) +- ^ ProjectExecTransformer (50) +- ^ FlushableHashAggregateExecTransformer (49) +- ^ ProjectExecTransformer (48) @@ -308,9 +308,9 @@ Input [4]: [nation#X, o_year#X, sum#X, isEmpty#X] Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] Arguments: false -(52) VeloxAppendBatches +(52) VeloxResizeBatches Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (53) ColumnarExchange Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] @@ -337,9 +337,9 @@ Results [3]: [nation#X, o_year#X, sum(amount#X)#X AS sum_profit#X] Input [3]: [nation#X, o_year#X, sum_profit#X] Arguments: false -(59) VeloxAppendBatches +(59) VeloxResizeBatches Input [3]: [nation#X, o_year#X, sum_profit#X] -Arguments: X +Arguments: X, X (60) ColumnarExchange Input [3]: [nation#X, o_year#X, sum_profit#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/1.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/1.txt index afea15af53d2..159a1598c9bf 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/1.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/1.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (30) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ RegularHashAggregateExecTransformer (12) +- ^ InputIteratorTransformer (11) +- ShuffleQueryStage (9), Statistics(X) +- ColumnarExchange (8) - +- VeloxAppendBatches (7) + +- VeloxResizeBatches (7) +- ^ ProjectExecTransformer (5) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) @@ -58,9 +58,9 @@ Input [17]: [l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] Arguments: false -(7) VeloxAppendBatches +(7) VeloxResizeBatches Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] -Arguments: X +Arguments: X, X (8) ColumnarExchange Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] @@ -87,9 +87,9 @@ Results [10]: [l_returnflag#X, l_linestatus#X, sum(l_quantity#X)#X AS sum_qty#X, Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/10.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/10.txt index 7a509e435d07..2d91fcf6299e 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/10.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/10.txt @@ -8,7 +8,7 @@ AdaptiveSparkPlan (68) +- ^ InputIteratorTransformer (39) +- ShuffleQueryStage (37), Statistics(X) +- ColumnarExchange (36) - +- VeloxAppendBatches (35) + +- VeloxResizeBatches (35) +- ^ ProjectExecTransformer (33) +- ^ FlushableHashAggregateExecTransformer (32) +- ^ ProjectExecTransformer (31) @@ -210,9 +210,9 @@ Input [9]: [c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] Arguments: false -(35) VeloxAppendBatches +(35) VeloxResizeBatches Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (36) ColumnarExchange Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/11.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/11.txt index 1c4e102aa0b9..b901a5db5815 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/11.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/11.txt @@ -6,13 +6,13 @@ AdaptiveSparkPlan (60) +- ^ InputIteratorTransformer (37) +- ShuffleQueryStage (35), Statistics(X) +- ColumnarExchange (34) - +- VeloxAppendBatches (33) + +- VeloxResizeBatches (33) +- ^ FilterExecTransformer (31) +- ^ RegularHashAggregateExecTransformer (30) +- ^ InputIteratorTransformer (29) +- ShuffleQueryStage (27), Statistics(X) +- ColumnarExchange (26) - +- VeloxAppendBatches (25) + +- VeloxResizeBatches (25) +- ^ ProjectExecTransformer (23) +- ^ FlushableHashAggregateExecTransformer (22) +- ^ ProjectExecTransformer (21) @@ -160,9 +160,9 @@ Input [3]: [ps_partkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] Arguments: false -(25) VeloxAppendBatches +(25) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (26) ColumnarExchange Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] @@ -193,9 +193,9 @@ Arguments: (isnotnull(value#X) AND (cast(value#X as decimal(38,6)) > Subquery su Input [2]: [ps_partkey#X, value#X] Arguments: false -(33) VeloxAppendBatches +(33) VeloxResizeBatches Input [2]: [ps_partkey#X, value#X] -Arguments: X +Arguments: X, X (34) ColumnarExchange Input [2]: [ps_partkey#X, value#X] @@ -330,7 +330,7 @@ AdaptiveSparkPlan (102) +- ^ InputIteratorTransformer (81) +- ShuffleQueryStage (79), Statistics(X) +- ColumnarExchange (78) - +- VeloxAppendBatches (77) + +- VeloxResizeBatches (77) +- ^ FlushableHashAggregateExecTransformer (75) +- ^ ProjectExecTransformer (74) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (73) @@ -429,9 +429,9 @@ Results [2]: [sum#X, isEmpty#X] Input [2]: [sum#X, isEmpty#X] Arguments: false -(77) VeloxAppendBatches +(77) VeloxResizeBatches Input [2]: [sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (78) ColumnarExchange Input [2]: [sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/12.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/12.txt index 12d0f6f0ff95..456b95e5f5de 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/12.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/12.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (44) +- ^ InputIteratorTransformer (27) +- ShuffleQueryStage (25), Statistics(X) +- ColumnarExchange (24) - +- VeloxAppendBatches (23) + +- VeloxResizeBatches (23) +- ^ RegularHashAggregateExecTransformer (21) +- ^ InputIteratorTransformer (20) +- ShuffleQueryStage (18), Statistics(X) +- ColumnarExchange (17) - +- VeloxAppendBatches (16) + +- VeloxResizeBatches (16) +- ^ ProjectExecTransformer (14) +- ^ FlushableHashAggregateExecTransformer (13) +- ^ ProjectExecTransformer (12) @@ -108,9 +108,9 @@ Input [3]: [l_shipmode#X, sum#X, sum#X] Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] Arguments: false -(16) VeloxAppendBatches +(16) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] -Arguments: X +Arguments: X, X (17) ColumnarExchange Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] @@ -137,9 +137,9 @@ Results [3]: [l_shipmode#X, sum(CASE WHEN ((o_orderpriority#X = 1-URGENT) OR (o_ Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] Arguments: false -(23) VeloxAppendBatches +(23) VeloxResizeBatches Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] -Arguments: X +Arguments: X, X (24) ColumnarExchange Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/13.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/13.txt index 1dfb80d78050..6044e9fa526c 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/13.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/13.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (55) +- ^ InputIteratorTransformer (36) +- ShuffleQueryStage (34), Statistics(X) +- ColumnarExchange (33) - +- VeloxAppendBatches (32) + +- VeloxResizeBatches (32) +- ^ RegularHashAggregateExecTransformer (30) +- ^ InputIteratorTransformer (29) +- ShuffleQueryStage (27), Statistics(X) +- ColumnarExchange (26) - +- VeloxAppendBatches (25) + +- VeloxResizeBatches (25) +- ^ ProjectExecTransformer (23) +- ^ FlushableHashAggregateExecTransformer (22) +- ^ ProjectExecTransformer (21) @@ -19,7 +19,7 @@ AdaptiveSparkPlan (55) +- ^ InputIteratorTransformer (19) +- ShuffleQueryStage (17), Statistics(X) +- ColumnarExchange (16) - +- VeloxAppendBatches (15) + +- VeloxResizeBatches (15) +- ^ ProjectExecTransformer (13) +- ^ FlushableHashAggregateExecTransformer (12) +- ^ ProjectExecTransformer (11) @@ -112,9 +112,9 @@ Input [2]: [c_custkey#X, count#X] Input [3]: [hash_partition_key#X, c_custkey#X, count#X] Arguments: false -(15) VeloxAppendBatches +(15) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_custkey#X, count#X] -Arguments: X +Arguments: X, X (16) ColumnarExchange Input [3]: [hash_partition_key#X, c_custkey#X, count#X] @@ -156,9 +156,9 @@ Input [2]: [c_count#X, count#X] Input [3]: [hash_partition_key#X, c_count#X, count#X] Arguments: false -(25) VeloxAppendBatches +(25) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_count#X, count#X] -Arguments: X +Arguments: X, X (26) ColumnarExchange Input [3]: [hash_partition_key#X, c_count#X, count#X] @@ -185,9 +185,9 @@ Results [2]: [c_count#X, count(1)#X AS custdist#X] Input [2]: [c_count#X, custdist#X] Arguments: false -(32) VeloxAppendBatches +(32) VeloxResizeBatches Input [2]: [c_count#X, custdist#X] -Arguments: X +Arguments: X, X (33) ColumnarExchange Input [2]: [c_count#X, custdist#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/14.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/14.txt index 6f71859cba6d..0f8e52cb2056 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/14.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/14.txt @@ -7,7 +7,7 @@ AdaptiveSparkPlan (35) +- ^ InputIteratorTransformer (19) +- ShuffleQueryStage (17), Statistics(X) +- ColumnarExchange (16) - +- VeloxAppendBatches (15) + +- VeloxResizeBatches (15) +- ^ FlushableHashAggregateExecTransformer (13) +- ^ ProjectExecTransformer (12) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (11) @@ -97,9 +97,9 @@ Results [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] Arguments: false -(15) VeloxAppendBatches +(15) VeloxResizeBatches Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (16) ColumnarExchange Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/15.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/15.txt index 40b342510de2..dd611a9b29c8 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/15.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/15.txt @@ -5,7 +5,7 @@ AdaptiveSparkPlan (43) +- AQEShuffleRead (27) +- ShuffleQueryStage (26), Statistics(X) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ ProjectExecTransformer (22) +- ^ BroadcastHashJoinExecTransformer Inner BuildLeft (21) :- ^ InputIteratorTransformer (7) @@ -18,7 +18,7 @@ AdaptiveSparkPlan (43) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FlushableHashAggregateExecTransformer (11) +- ^ ProjectExecTransformer (10) @@ -100,9 +100,9 @@ Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] @@ -142,9 +142,9 @@ Input [6]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, supplier_no#X, total_ Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] @@ -247,7 +247,7 @@ AdaptiveSparkPlan (69) +- ^ InputIteratorTransformer (54) +- ShuffleQueryStage (52), Statistics(X) +- ColumnarExchange (51) - +- VeloxAppendBatches (50) + +- VeloxResizeBatches (50) +- ^ ProjectExecTransformer (48) +- ^ FlushableHashAggregateExecTransformer (47) +- ^ ProjectExecTransformer (46) @@ -294,9 +294,9 @@ Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] Arguments: false -(50) VeloxAppendBatches +(50) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (51) ColumnarExchange Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/16.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/16.txt index d579c1a704e5..13aa5f68019e 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/16.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/16.txt @@ -6,19 +6,19 @@ AdaptiveSparkPlan (59) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33), Statistics(X) +- ColumnarExchange (32) - +- VeloxAppendBatches (31) + +- VeloxResizeBatches (31) +- ^ RegularHashAggregateExecTransformer (29) +- ^ InputIteratorTransformer (28) +- ShuffleQueryStage (26), Statistics(X) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ ProjectExecTransformer (22) +- ^ FlushableHashAggregateExecTransformer (21) +- ^ RegularHashAggregateExecTransformer (20) +- ^ InputIteratorTransformer (19) +- ShuffleQueryStage (17), Statistics(X) +- ColumnarExchange (16) - +- VeloxAppendBatches (15) + +- VeloxResizeBatches (15) +- ^ ProjectExecTransformer (13) +- ^ FlushableHashAggregateExecTransformer (12) +- ^ ProjectExecTransformer (11) @@ -117,9 +117,9 @@ Input [4]: [p_brand#X, p_type#X, p_size#X, ps_suppkey#X] Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] Arguments: false -(15) VeloxAppendBatches +(15) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] -Arguments: X +Arguments: X, X (16) ColumnarExchange Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] @@ -157,9 +157,9 @@ Input [4]: [p_brand#X, p_type#X, p_size#X, count#X] Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] @@ -186,9 +186,9 @@ Results [4]: [p_brand#X, p_type#X, p_size#X, count(ps_suppkey#X)#X AS supplier_c Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/17.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/17.txt index 1e2ed970aef4..38801e9f690a 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/17.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/17.txt @@ -7,7 +7,7 @@ AdaptiveSparkPlan (36) +- ^ InputIteratorTransformer (11) +- ShuffleQueryStage (9), Statistics(X) +- ColumnarExchange (8) - +- VeloxAppendBatches (7) + +- VeloxResizeBatches (7) +- ^ FlushableHashAggregateExecTransformer (5) +- ^ InputIteratorTransformer (4) +- RowToVeloxColumnar (2) @@ -59,9 +59,9 @@ Results [2]: [sum#X, isEmpty#X] Input [2]: [sum#X, isEmpty#X] Arguments: false -(7) VeloxAppendBatches +(7) VeloxResizeBatches Input [2]: [sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (8) ColumnarExchange Input [2]: [sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/18.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/18.txt index 2e7ce455aebd..62a2f7b0b48a 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/18.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/18.txt @@ -7,7 +7,7 @@ AdaptiveSparkPlan (88) +- ^ InputIteratorTransformer (51) +- ShuffleQueryStage (49), Statistics(X) +- ColumnarExchange (48) - +- VeloxAppendBatches (47) + +- VeloxResizeBatches (47) +- ^ ProjectExecTransformer (45) +- ^ FlushableHashAggregateExecTransformer (44) +- ^ ProjectExecTransformer (43) @@ -31,7 +31,7 @@ AdaptiveSparkPlan (88) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16), Statistics(X) : +- ColumnarExchange (15) - : +- VeloxAppendBatches (14) + : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ FlushableHashAggregateExecTransformer (11) : +- ^ Scan parquet (10) @@ -140,9 +140,9 @@ Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] @@ -276,9 +276,9 @@ Input [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, Input [8]: [hash_partition_key#X, c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] Arguments: false -(47) VeloxAppendBatches +(47) VeloxResizeBatches Input [8]: [hash_partition_key#X, c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (48) ColumnarExchange Input [8]: [hash_partition_key#X, c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/19.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/19.txt index 992c5328adba..178fa021fb70 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/19.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/19.txt @@ -6,7 +6,7 @@ AdaptiveSparkPlan (34) +- ^ InputIteratorTransformer (19) +- ShuffleQueryStage (17), Statistics(X) +- ColumnarExchange (16) - +- VeloxAppendBatches (15) + +- VeloxResizeBatches (15) +- ^ FlushableHashAggregateExecTransformer (13) +- ^ ProjectExecTransformer (12) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (11) @@ -96,9 +96,9 @@ Results [2]: [sum#X, isEmpty#X] Input [2]: [sum#X, isEmpty#X] Arguments: false -(15) VeloxAppendBatches +(15) VeloxResizeBatches Input [2]: [sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (16) ColumnarExchange Input [2]: [sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/20.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/20.txt index 4a899ae239be..62a0f1a317b1 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/20.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/20.txt @@ -5,7 +5,7 @@ AdaptiveSparkPlan (109) +- AQEShuffleRead (69) +- ShuffleQueryStage (68), Statistics(X) +- ColumnarExchange (67) - +- VeloxAppendBatches (66) + +- VeloxResizeBatches (66) +- ^ ProjectExecTransformer (64) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (63) :- ^ ProjectExecTransformer (54) @@ -14,7 +14,7 @@ AdaptiveSparkPlan (109) : : +- AQEShuffleRead (8) : : +- ShuffleQueryStage (7), Statistics(X) : : +- ColumnarExchange (6) - : : +- VeloxAppendBatches (5) + : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ NoopFilter (2) : : +- ^ Scan parquet (1) @@ -41,7 +41,7 @@ AdaptiveSparkPlan (109) : +- ^ InputIteratorTransformer (42) : +- ShuffleQueryStage (40), Statistics(X) : +- ColumnarExchange (39) - : +- VeloxAppendBatches (38) + : +- VeloxResizeBatches (38) : +- ^ ProjectExecTransformer (36) : +- ^ FlushableHashAggregateExecTransformer (35) : +- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (34) @@ -117,9 +117,9 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] @@ -254,9 +254,9 @@ Input [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] Input [5]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] Arguments: false -(38) VeloxAppendBatches +(38) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (39) ColumnarExchange Input [5]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] @@ -369,9 +369,9 @@ Input [4]: [s_name#X, s_address#X, s_nationkey#X, n_nationkey#X] Input [2]: [s_name#X, s_address#X] Arguments: false -(66) VeloxAppendBatches +(66) VeloxResizeBatches Input [2]: [s_name#X, s_address#X] -Arguments: X +Arguments: X, X (67) ColumnarExchange Input [2]: [s_name#X, s_address#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/21.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/21.txt index 7627c7f4f147..045b4a38e3a3 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/21.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/21.txt @@ -6,7 +6,7 @@ AdaptiveSparkPlan (92) +- ^ InputIteratorTransformer (56) +- ShuffleQueryStage (54), Statistics(X) +- ColumnarExchange (53) - +- VeloxAppendBatches (52) + +- VeloxResizeBatches (52) +- ^ ProjectExecTransformer (50) +- ^ FlushableHashAggregateExecTransformer (49) +- ^ ProjectExecTransformer (48) @@ -301,9 +301,9 @@ Input [2]: [s_name#X, count#X] Input [3]: [hash_partition_key#X, s_name#X, count#X] Arguments: false -(52) VeloxAppendBatches +(52) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_name#X, count#X] -Arguments: X +Arguments: X, X (53) ColumnarExchange Input [3]: [hash_partition_key#X, s_name#X, count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/22.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/22.txt index 9a0475d25d78..829be008d843 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/22.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/22.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (40) +- ^ InputIteratorTransformer (25) +- ShuffleQueryStage (23), Statistics(X) +- ColumnarExchange (22) - +- VeloxAppendBatches (21) + +- VeloxResizeBatches (21) +- ^ RegularHashAggregateExecTransformer (19) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FlushableHashAggregateExecTransformer (11) +- ^ ProjectExecTransformer (10) @@ -95,9 +95,9 @@ Input [4]: [cntrycode#X, count#X, sum#X, isEmpty#X] Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] @@ -124,9 +124,9 @@ Results [3]: [cntrycode#X, count(1)#X AS numcust#X, sum(c_acctbal#X)#X AS totacc Input [3]: [cntrycode#X, numcust#X, totacctbal#X] Arguments: false -(21) VeloxAppendBatches +(21) VeloxResizeBatches Input [3]: [cntrycode#X, numcust#X, totacctbal#X] -Arguments: X +Arguments: X, X (22) ColumnarExchange Input [3]: [cntrycode#X, numcust#X, totacctbal#X] @@ -223,7 +223,7 @@ AdaptiveSparkPlan (60) +- ^ InputIteratorTransformer (50) +- ShuffleQueryStage (48), Statistics(X) +- ColumnarExchange (47) - +- VeloxAppendBatches (46) + +- VeloxResizeBatches (46) +- ^ FlushableHashAggregateExecTransformer (44) +- ^ ProjectExecTransformer (43) +- ^ NoopFilter (42) @@ -263,9 +263,9 @@ Results [2]: [sum#X, count#X] Input [2]: [sum#X, count#X] Arguments: false -(46) VeloxAppendBatches +(46) VeloxResizeBatches Input [2]: [sum#X, count#X] -Arguments: X +Arguments: X, X (47) ColumnarExchange Input [2]: [sum#X, count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/3.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/3.txt index 60843c0991ad..5ee5820d32d6 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/3.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/3.txt @@ -8,7 +8,7 @@ AdaptiveSparkPlan (54) +- ^ InputIteratorTransformer (30) +- ShuffleQueryStage (28), Statistics(X) +- ColumnarExchange (27) - +- VeloxAppendBatches (26) + +- VeloxResizeBatches (26) +- ^ ProjectExecTransformer (24) +- ^ FlushableHashAggregateExecTransformer (23) +- ^ ProjectExecTransformer (22) @@ -160,9 +160,9 @@ Input [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] Input [6]: [hash_partition_key#X, l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] Arguments: false -(26) VeloxAppendBatches +(26) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (27) ColumnarExchange Input [6]: [hash_partition_key#X, l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/4.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/4.txt index b142ee1d8f33..b9eb997a0cbb 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/4.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/4.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (46) +- ^ InputIteratorTransformer (28) +- ShuffleQueryStage (26), Statistics(X) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ RegularHashAggregateExecTransformer (22) +- ^ InputIteratorTransformer (21) +- ShuffleQueryStage (19), Statistics(X) +- ColumnarExchange (18) - +- VeloxAppendBatches (17) + +- VeloxResizeBatches (17) +- ^ ProjectExecTransformer (15) +- ^ FlushableHashAggregateExecTransformer (14) +- ^ ProjectExecTransformer (13) @@ -114,9 +114,9 @@ Input [2]: [o_orderpriority#X, count#X] Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] Arguments: false -(17) VeloxAppendBatches +(17) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] -Arguments: X +Arguments: X, X (18) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] @@ -143,9 +143,9 @@ Results [2]: [o_orderpriority#X, count(1)#X AS order_count#X] Input [2]: [o_orderpriority#X, order_count#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [2]: [o_orderpriority#X, order_count#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [2]: [o_orderpriority#X, order_count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/5.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/5.txt index c8a62c3aca1f..a0d3ca93f02b 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/5.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/5.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (102) +- ^ InputIteratorTransformer (64) +- ShuffleQueryStage (62), Statistics(X) +- ColumnarExchange (61) - +- VeloxAppendBatches (60) + +- VeloxResizeBatches (60) +- ^ RegularHashAggregateExecTransformer (58) +- ^ InputIteratorTransformer (57) +- ShuffleQueryStage (55), Statistics(X) +- ColumnarExchange (54) - +- VeloxAppendBatches (53) + +- VeloxResizeBatches (53) +- ^ ProjectExecTransformer (51) +- ^ FlushableHashAggregateExecTransformer (50) +- ^ ProjectExecTransformer (49) @@ -314,9 +314,9 @@ Input [3]: [n_name#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] Arguments: false -(53) VeloxAppendBatches +(53) VeloxResizeBatches Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (54) ColumnarExchange Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] @@ -343,9 +343,9 @@ Results [2]: [n_name#X, sum(CheckOverflow((promote_precision(cast(l_extendedpric Input [2]: [n_name#X, revenue#X] Arguments: false -(60) VeloxAppendBatches +(60) VeloxResizeBatches Input [2]: [n_name#X, revenue#X] -Arguments: X +Arguments: X, X (61) ColumnarExchange Input [2]: [n_name#X, revenue#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/6.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/6.txt index fa9c936a0ca1..1aaed506d7e0 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/6.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/6.txt @@ -6,7 +6,7 @@ AdaptiveSparkPlan (20) +- ^ InputIteratorTransformer (10) +- ShuffleQueryStage (8), Statistics(X) +- ColumnarExchange (7) - +- VeloxAppendBatches (6) + +- VeloxResizeBatches (6) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ NoopFilter (2) @@ -46,9 +46,9 @@ Results [2]: [sum#X, isEmpty#X] Input [2]: [sum#X, isEmpty#X] Arguments: false -(6) VeloxAppendBatches +(6) VeloxResizeBatches Input [2]: [sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (7) ColumnarExchange Input [2]: [sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/7.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/7.txt index f098b9ba837f..8d95ec5e6f74 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/7.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/7.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (95) +- ^ InputIteratorTransformer (59) +- ShuffleQueryStage (57), Statistics(X) +- ColumnarExchange (56) - +- VeloxAppendBatches (55) + +- VeloxResizeBatches (55) +- ^ RegularHashAggregateExecTransformer (53) +- ^ InputIteratorTransformer (52) +- ShuffleQueryStage (50), Statistics(X) +- ColumnarExchange (49) - +- VeloxAppendBatches (48) + +- VeloxResizeBatches (48) +- ^ ProjectExecTransformer (46) +- ^ FlushableHashAggregateExecTransformer (45) +- ^ ProjectExecTransformer (44) @@ -284,9 +284,9 @@ Input [5]: [supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] @@ -313,9 +313,9 @@ Results [4]: [supp_nation#X, cust_nation#X, l_year#X, sum(volume#X)#X AS revenue Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] Arguments: false -(55) VeloxAppendBatches +(55) VeloxResizeBatches Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] -Arguments: X +Arguments: X, X (56) ColumnarExchange Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/8.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/8.txt index becfc12a1b86..b74dc65358d2 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/8.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/8.txt @@ -6,13 +6,13 @@ AdaptiveSparkPlan (131) +- ^ InputIteratorTransformer (83) +- ShuffleQueryStage (81), Statistics(X) +- ColumnarExchange (80) - +- VeloxAppendBatches (79) + +- VeloxResizeBatches (79) +- ^ ProjectExecTransformer (77) +- ^ RegularHashAggregateExecTransformer (76) +- ^ InputIteratorTransformer (75) +- ShuffleQueryStage (73), Statistics(X) +- ColumnarExchange (72) - +- VeloxAppendBatches (71) + +- VeloxResizeBatches (71) +- ^ ProjectExecTransformer (69) +- ^ FlushableHashAggregateExecTransformer (68) +- ^ ProjectExecTransformer (67) @@ -415,9 +415,9 @@ Input [5]: [o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] Arguments: false -(71) VeloxAppendBatches +(71) VeloxResizeBatches Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (72) ColumnarExchange Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] @@ -448,9 +448,9 @@ Input [3]: [o_year#X, sum(CASE WHEN (nation#X = BRAZIL) THEN volume#X ELSE 0.000 Input [2]: [o_year#X, mkt_share#X] Arguments: false -(79) VeloxAppendBatches +(79) VeloxResizeBatches Input [2]: [o_year#X, mkt_share#X] -Arguments: X +Arguments: X, X (80) ColumnarExchange Input [2]: [o_year#X, mkt_share#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/9.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/9.txt index a486fc65677b..400fb12e1dea 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/9.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/9.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (100) +- ^ InputIteratorTransformer (63) +- ShuffleQueryStage (61), Statistics(X) +- ColumnarExchange (60) - +- VeloxAppendBatches (59) + +- VeloxResizeBatches (59) +- ^ RegularHashAggregateExecTransformer (57) +- ^ InputIteratorTransformer (56) +- ShuffleQueryStage (54), Statistics(X) +- ColumnarExchange (53) - +- VeloxAppendBatches (52) + +- VeloxResizeBatches (52) +- ^ ProjectExecTransformer (50) +- ^ FlushableHashAggregateExecTransformer (49) +- ^ ProjectExecTransformer (48) @@ -308,9 +308,9 @@ Input [4]: [nation#X, o_year#X, sum#X, isEmpty#X] Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] Arguments: false -(52) VeloxAppendBatches +(52) VeloxResizeBatches Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (53) ColumnarExchange Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] @@ -337,9 +337,9 @@ Results [3]: [nation#X, o_year#X, sum(amount#X)#X AS sum_profit#X] Input [3]: [nation#X, o_year#X, sum_profit#X] Arguments: false -(59) VeloxAppendBatches +(59) VeloxResizeBatches Input [3]: [nation#X, o_year#X, sum_profit#X] -Arguments: X +Arguments: X, X (60) ColumnarExchange Input [3]: [nation#X, o_year#X, sum_profit#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/1.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/1.txt index 545f2e7e086d..2db104cfeb12 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/1.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/1.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (30) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ RegularHashAggregateExecTransformer (12) +- ^ InputIteratorTransformer (11) +- ShuffleQueryStage (9), Statistics(X) +- ColumnarExchange (8) - +- VeloxAppendBatches (7) + +- VeloxResizeBatches (7) +- ^ ProjectExecTransformer (5) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) @@ -58,9 +58,9 @@ Input [17]: [l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] Arguments: false -(7) VeloxAppendBatches +(7) VeloxResizeBatches Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] -Arguments: X +Arguments: X, X (8) ColumnarExchange Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] @@ -87,9 +87,9 @@ Results [10]: [l_returnflag#X, l_linestatus#X, sum(l_quantity#X)#X AS sum_qty#X, Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/10.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/10.txt index 9325f007789b..4c263cb4a280 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/10.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/10.txt @@ -8,7 +8,7 @@ AdaptiveSparkPlan (68) +- ^ InputIteratorTransformer (39) +- ShuffleQueryStage (37), Statistics(X) +- ColumnarExchange (36) - +- VeloxAppendBatches (35) + +- VeloxResizeBatches (35) +- ^ ProjectExecTransformer (33) +- ^ FlushableHashAggregateExecTransformer (32) +- ^ ProjectExecTransformer (31) @@ -213,9 +213,9 @@ Input [9]: [c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] Arguments: false -(35) VeloxAppendBatches +(35) VeloxResizeBatches Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (36) ColumnarExchange Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/11.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/11.txt index c593374b9a00..83e60d925e40 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/11.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/11.txt @@ -6,13 +6,13 @@ AdaptiveSparkPlan (60) +- ^ InputIteratorTransformer (37) +- ShuffleQueryStage (35), Statistics(X) +- ColumnarExchange (34) - +- VeloxAppendBatches (33) + +- VeloxResizeBatches (33) +- ^ FilterExecTransformer (31) +- ^ RegularHashAggregateExecTransformer (30) +- ^ InputIteratorTransformer (29) +- ShuffleQueryStage (27), Statistics(X) +- ColumnarExchange (26) - +- VeloxAppendBatches (25) + +- VeloxResizeBatches (25) +- ^ ProjectExecTransformer (23) +- ^ FlushableHashAggregateExecTransformer (22) +- ^ ProjectExecTransformer (21) @@ -162,9 +162,9 @@ Input [3]: [ps_partkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] Arguments: false -(25) VeloxAppendBatches +(25) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (26) ColumnarExchange Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] @@ -195,9 +195,9 @@ Arguments: (isnotnull(value#X) AND (cast(value#X as decimal(38,6)) > Subquery su Input [2]: [ps_partkey#X, value#X] Arguments: false -(33) VeloxAppendBatches +(33) VeloxResizeBatches Input [2]: [ps_partkey#X, value#X] -Arguments: X +Arguments: X, X (34) ColumnarExchange Input [2]: [ps_partkey#X, value#X] @@ -334,7 +334,7 @@ AdaptiveSparkPlan (102) +- ^ InputIteratorTransformer (81) +- ShuffleQueryStage (79), Statistics(X) +- ColumnarExchange (78) - +- VeloxAppendBatches (77) + +- VeloxResizeBatches (77) +- ^ FlushableHashAggregateExecTransformer (75) +- ^ ProjectExecTransformer (74) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (73) @@ -435,9 +435,9 @@ Results [2]: [sum#X, isEmpty#X] Input [2]: [sum#X, isEmpty#X] Arguments: false -(77) VeloxAppendBatches +(77) VeloxResizeBatches Input [2]: [sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (78) ColumnarExchange Input [2]: [sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/12.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/12.txt index 93f3a4cf9ab7..354e97da2066 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/12.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/12.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (44) +- ^ InputIteratorTransformer (27) +- ShuffleQueryStage (25), Statistics(X) +- ColumnarExchange (24) - +- VeloxAppendBatches (23) + +- VeloxResizeBatches (23) +- ^ RegularHashAggregateExecTransformer (21) +- ^ InputIteratorTransformer (20) +- ShuffleQueryStage (18), Statistics(X) +- ColumnarExchange (17) - +- VeloxAppendBatches (16) + +- VeloxResizeBatches (16) +- ^ ProjectExecTransformer (14) +- ^ FlushableHashAggregateExecTransformer (13) +- ^ ProjectExecTransformer (12) @@ -109,9 +109,9 @@ Input [3]: [l_shipmode#X, sum#X, sum#X] Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] Arguments: false -(16) VeloxAppendBatches +(16) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] -Arguments: X +Arguments: X, X (17) ColumnarExchange Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] @@ -138,9 +138,9 @@ Results [3]: [l_shipmode#X, sum(CASE WHEN ((o_orderpriority#X = 1-URGENT) OR (o_ Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] Arguments: false -(23) VeloxAppendBatches +(23) VeloxResizeBatches Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] -Arguments: X +Arguments: X, X (24) ColumnarExchange Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/13.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/13.txt index e44b10042401..614098d876a4 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/13.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/13.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (55) +- ^ InputIteratorTransformer (36) +- ShuffleQueryStage (34), Statistics(X) +- ColumnarExchange (33) - +- VeloxAppendBatches (32) + +- VeloxResizeBatches (32) +- ^ RegularHashAggregateExecTransformer (30) +- ^ InputIteratorTransformer (29) +- ShuffleQueryStage (27), Statistics(X) +- ColumnarExchange (26) - +- VeloxAppendBatches (25) + +- VeloxResizeBatches (25) +- ^ ProjectExecTransformer (23) +- ^ FlushableHashAggregateExecTransformer (22) +- ^ ProjectExecTransformer (21) @@ -19,7 +19,7 @@ AdaptiveSparkPlan (55) +- ^ InputIteratorTransformer (19) +- ShuffleQueryStage (17), Statistics(X) +- ColumnarExchange (16) - +- VeloxAppendBatches (15) + +- VeloxResizeBatches (15) +- ^ ProjectExecTransformer (13) +- ^ FlushableHashAggregateExecTransformer (12) +- ^ ProjectExecTransformer (11) @@ -113,9 +113,9 @@ Input [2]: [c_custkey#X, count#X] Input [3]: [hash_partition_key#X, c_custkey#X, count#X] Arguments: false -(15) VeloxAppendBatches +(15) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_custkey#X, count#X] -Arguments: X +Arguments: X, X (16) ColumnarExchange Input [3]: [hash_partition_key#X, c_custkey#X, count#X] @@ -157,9 +157,9 @@ Input [2]: [c_count#X, count#X] Input [3]: [hash_partition_key#X, c_count#X, count#X] Arguments: false -(25) VeloxAppendBatches +(25) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_count#X, count#X] -Arguments: X +Arguments: X, X (26) ColumnarExchange Input [3]: [hash_partition_key#X, c_count#X, count#X] @@ -186,9 +186,9 @@ Results [2]: [c_count#X, count(1)#X AS custdist#X] Input [2]: [c_count#X, custdist#X] Arguments: false -(32) VeloxAppendBatches +(32) VeloxResizeBatches Input [2]: [c_count#X, custdist#X] -Arguments: X +Arguments: X, X (33) ColumnarExchange Input [2]: [c_count#X, custdist#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/14.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/14.txt index 8f5ae0f52e63..df48e66cf0ab 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/14.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/14.txt @@ -7,7 +7,7 @@ AdaptiveSparkPlan (35) +- ^ InputIteratorTransformer (19) +- ShuffleQueryStage (17), Statistics(X) +- ColumnarExchange (16) - +- VeloxAppendBatches (15) + +- VeloxResizeBatches (15) +- ^ FlushableHashAggregateExecTransformer (13) +- ^ ProjectExecTransformer (12) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (11) @@ -98,9 +98,9 @@ Results [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] Arguments: false -(15) VeloxAppendBatches +(15) VeloxResizeBatches Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (16) ColumnarExchange Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/15.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/15.txt index d616981e57d2..dfdcc6a1d8c7 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/15.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/15.txt @@ -5,7 +5,7 @@ AdaptiveSparkPlan (43) +- AQEShuffleRead (27) +- ShuffleQueryStage (26), Statistics(X) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ ProjectExecTransformer (22) +- ^ BroadcastHashJoinExecTransformer Inner BuildLeft (21) :- ^ InputIteratorTransformer (7) @@ -18,7 +18,7 @@ AdaptiveSparkPlan (43) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FlushableHashAggregateExecTransformer (11) +- ^ ProjectExecTransformer (10) @@ -100,9 +100,9 @@ Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] @@ -143,9 +143,9 @@ Input [6]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, supplier_no#X, total_ Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] @@ -249,7 +249,7 @@ AdaptiveSparkPlan (69) +- ^ InputIteratorTransformer (54) +- ShuffleQueryStage (52), Statistics(X) +- ColumnarExchange (51) - +- VeloxAppendBatches (50) + +- VeloxResizeBatches (50) +- ^ ProjectExecTransformer (48) +- ^ FlushableHashAggregateExecTransformer (47) +- ^ ProjectExecTransformer (46) @@ -296,9 +296,9 @@ Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] Arguments: false -(50) VeloxAppendBatches +(50) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (51) ColumnarExchange Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/16.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/16.txt index 3b4da47fc25f..73bf50f8f6f0 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/16.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/16.txt @@ -6,19 +6,19 @@ AdaptiveSparkPlan (59) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33), Statistics(X) +- ColumnarExchange (32) - +- VeloxAppendBatches (31) + +- VeloxResizeBatches (31) +- ^ RegularHashAggregateExecTransformer (29) +- ^ InputIteratorTransformer (28) +- ShuffleQueryStage (26), Statistics(X) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ ProjectExecTransformer (22) +- ^ FlushableHashAggregateExecTransformer (21) +- ^ RegularHashAggregateExecTransformer (20) +- ^ InputIteratorTransformer (19) +- ShuffleQueryStage (17), Statistics(X) +- ColumnarExchange (16) - +- VeloxAppendBatches (15) + +- VeloxResizeBatches (15) +- ^ ProjectExecTransformer (13) +- ^ FlushableHashAggregateExecTransformer (12) +- ^ ProjectExecTransformer (11) @@ -118,9 +118,9 @@ Input [4]: [p_brand#X, p_type#X, p_size#X, ps_suppkey#X] Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] Arguments: false -(15) VeloxAppendBatches +(15) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] -Arguments: X +Arguments: X, X (16) ColumnarExchange Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] @@ -158,9 +158,9 @@ Input [4]: [p_brand#X, p_type#X, p_size#X, count#X] Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] @@ -187,9 +187,9 @@ Results [4]: [p_brand#X, p_type#X, p_size#X, count(ps_suppkey#X)#X AS supplier_c Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/17.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/17.txt index 77733ad6f8a5..77c18798faa2 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/17.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/17.txt @@ -7,7 +7,7 @@ AdaptiveSparkPlan (36) +- ^ InputIteratorTransformer (11) +- ShuffleQueryStage (9), Statistics(X) +- ColumnarExchange (8) - +- VeloxAppendBatches (7) + +- VeloxResizeBatches (7) +- ^ FlushableHashAggregateExecTransformer (5) +- ^ InputIteratorTransformer (4) +- RowToVeloxColumnar (2) @@ -59,9 +59,9 @@ Results [2]: [sum#X, isEmpty#X] Input [2]: [sum#X, isEmpty#X] Arguments: false -(7) VeloxAppendBatches +(7) VeloxResizeBatches Input [2]: [sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (8) ColumnarExchange Input [2]: [sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/18.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/18.txt index 0f5e82ec54c1..db32a9be29f1 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/18.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/18.txt @@ -7,7 +7,7 @@ AdaptiveSparkPlan (88) +- ^ InputIteratorTransformer (51) +- ShuffleQueryStage (49), Statistics(X) +- ColumnarExchange (48) - +- VeloxAppendBatches (47) + +- VeloxResizeBatches (47) +- ^ ProjectExecTransformer (45) +- ^ FlushableHashAggregateExecTransformer (44) +- ^ ProjectExecTransformer (43) @@ -31,7 +31,7 @@ AdaptiveSparkPlan (88) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16), Statistics(X) : +- ColumnarExchange (15) - : +- VeloxAppendBatches (14) + : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ FlushableHashAggregateExecTransformer (11) : +- ^ Scan parquet (10) @@ -140,9 +140,9 @@ Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] @@ -280,9 +280,9 @@ Input [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, Input [8]: [hash_partition_key#X, c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] Arguments: false -(47) VeloxAppendBatches +(47) VeloxResizeBatches Input [8]: [hash_partition_key#X, c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (48) ColumnarExchange Input [8]: [hash_partition_key#X, c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/19.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/19.txt index 3bafdb994153..0a48f3fd8117 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/19.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/19.txt @@ -6,7 +6,7 @@ AdaptiveSparkPlan (34) +- ^ InputIteratorTransformer (19) +- ShuffleQueryStage (17), Statistics(X) +- ColumnarExchange (16) - +- VeloxAppendBatches (15) + +- VeloxResizeBatches (15) +- ^ FlushableHashAggregateExecTransformer (13) +- ^ ProjectExecTransformer (12) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (11) @@ -97,9 +97,9 @@ Results [2]: [sum#X, isEmpty#X] Input [2]: [sum#X, isEmpty#X] Arguments: false -(15) VeloxAppendBatches +(15) VeloxResizeBatches Input [2]: [sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (16) ColumnarExchange Input [2]: [sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/20.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/20.txt index 945cc79de1ac..d54ea7dc1c13 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/20.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/20.txt @@ -5,7 +5,7 @@ AdaptiveSparkPlan (98) +- AQEShuffleRead (61) +- ShuffleQueryStage (60), Statistics(X) +- ColumnarExchange (59) - +- VeloxAppendBatches (58) + +- VeloxResizeBatches (58) +- ^ ProjectExecTransformer (56) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (55) :- ^ ProjectExecTransformer (46) @@ -35,7 +35,7 @@ AdaptiveSparkPlan (98) : +- ^ InputIteratorTransformer (34) : +- ShuffleQueryStage (32), Statistics(X) : +- ColumnarExchange (31) - : +- VeloxAppendBatches (30) + : +- VeloxResizeBatches (30) : +- ^ ProjectExecTransformer (28) : +- ^ FlushableHashAggregateExecTransformer (27) : +- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (26) @@ -217,9 +217,9 @@ Input [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] Input [5]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] Arguments: false -(30) VeloxAppendBatches +(30) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (31) ColumnarExchange Input [5]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] @@ -335,9 +335,9 @@ Input [4]: [s_name#X, s_address#X, s_nationkey#X, n_nationkey#X] Input [2]: [s_name#X, s_address#X] Arguments: false -(58) VeloxAppendBatches +(58) VeloxResizeBatches Input [2]: [s_name#X, s_address#X] -Arguments: X +Arguments: X, X (59) ColumnarExchange Input [2]: [s_name#X, s_address#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/21.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/21.txt index a7d9f42063a0..162a7db4d212 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/21.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/21.txt @@ -6,7 +6,7 @@ AdaptiveSparkPlan (92) +- ^ InputIteratorTransformer (56) +- ShuffleQueryStage (54), Statistics(X) +- ColumnarExchange (53) - +- VeloxAppendBatches (52) + +- VeloxResizeBatches (52) +- ^ ProjectExecTransformer (50) +- ^ FlushableHashAggregateExecTransformer (49) +- ^ ProjectExecTransformer (48) @@ -306,9 +306,9 @@ Input [2]: [s_name#X, count#X] Input [3]: [hash_partition_key#X, s_name#X, count#X] Arguments: false -(52) VeloxAppendBatches +(52) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_name#X, count#X] -Arguments: X +Arguments: X, X (53) ColumnarExchange Input [3]: [hash_partition_key#X, s_name#X, count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/22.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/22.txt index 0d9139bcd990..b1ed2c89b6eb 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/22.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/22.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (40) +- ^ InputIteratorTransformer (25) +- ShuffleQueryStage (23), Statistics(X) +- ColumnarExchange (22) - +- VeloxAppendBatches (21) + +- VeloxResizeBatches (21) +- ^ RegularHashAggregateExecTransformer (19) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FlushableHashAggregateExecTransformer (11) +- ^ ProjectExecTransformer (10) @@ -96,9 +96,9 @@ Input [4]: [cntrycode#X, count#X, sum#X, isEmpty#X] Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] @@ -125,9 +125,9 @@ Results [3]: [cntrycode#X, count(1)#X AS numcust#X, sum(c_acctbal#X)#X AS totacc Input [3]: [cntrycode#X, numcust#X, totacctbal#X] Arguments: false -(21) VeloxAppendBatches +(21) VeloxResizeBatches Input [3]: [cntrycode#X, numcust#X, totacctbal#X] -Arguments: X +Arguments: X, X (22) ColumnarExchange Input [3]: [cntrycode#X, numcust#X, totacctbal#X] @@ -225,7 +225,7 @@ AdaptiveSparkPlan (60) +- ^ InputIteratorTransformer (50) +- ShuffleQueryStage (48), Statistics(X) +- ColumnarExchange (47) - +- VeloxAppendBatches (46) + +- VeloxResizeBatches (46) +- ^ FlushableHashAggregateExecTransformer (44) +- ^ ProjectExecTransformer (43) +- ^ NoopFilter (42) @@ -265,9 +265,9 @@ Results [2]: [sum#X, count#X] Input [2]: [sum#X, count#X] Arguments: false -(46) VeloxAppendBatches +(46) VeloxResizeBatches Input [2]: [sum#X, count#X] -Arguments: X +Arguments: X, X (47) ColumnarExchange Input [2]: [sum#X, count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/3.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/3.txt index 988bdc6c26ca..9673efca9bd6 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/3.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/3.txt @@ -8,7 +8,7 @@ AdaptiveSparkPlan (54) +- ^ InputIteratorTransformer (30) +- ShuffleQueryStage (28), Statistics(X) +- ColumnarExchange (27) - +- VeloxAppendBatches (26) + +- VeloxResizeBatches (26) +- ^ ProjectExecTransformer (24) +- ^ FlushableHashAggregateExecTransformer (23) +- ^ ProjectExecTransformer (22) @@ -162,9 +162,9 @@ Input [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] Input [6]: [hash_partition_key#X, l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] Arguments: false -(26) VeloxAppendBatches +(26) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (27) ColumnarExchange Input [6]: [hash_partition_key#X, l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/4.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/4.txt index 0b73608fbe4b..ce4180123f4c 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/4.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/4.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (46) +- ^ InputIteratorTransformer (28) +- ShuffleQueryStage (26), Statistics(X) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ RegularHashAggregateExecTransformer (22) +- ^ InputIteratorTransformer (21) +- ShuffleQueryStage (19), Statistics(X) +- ColumnarExchange (18) - +- VeloxAppendBatches (17) + +- VeloxResizeBatches (17) +- ^ ProjectExecTransformer (15) +- ^ FlushableHashAggregateExecTransformer (14) +- ^ ProjectExecTransformer (13) @@ -115,9 +115,9 @@ Input [2]: [o_orderpriority#X, count#X] Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] Arguments: false -(17) VeloxAppendBatches +(17) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] -Arguments: X +Arguments: X, X (18) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] @@ -144,9 +144,9 @@ Results [2]: [o_orderpriority#X, count(1)#X AS order_count#X] Input [2]: [o_orderpriority#X, order_count#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [2]: [o_orderpriority#X, order_count#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [2]: [o_orderpriority#X, order_count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/5.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/5.txt index 8891188e8a08..742f9db1a054 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/5.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/5.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (102) +- ^ InputIteratorTransformer (64) +- ShuffleQueryStage (62), Statistics(X) +- ColumnarExchange (61) - +- VeloxAppendBatches (60) + +- VeloxResizeBatches (60) +- ^ RegularHashAggregateExecTransformer (58) +- ^ InputIteratorTransformer (57) +- ShuffleQueryStage (55), Statistics(X) +- ColumnarExchange (54) - +- VeloxAppendBatches (53) + +- VeloxResizeBatches (53) +- ^ ProjectExecTransformer (51) +- ^ FlushableHashAggregateExecTransformer (50) +- ^ ProjectExecTransformer (49) @@ -319,9 +319,9 @@ Input [3]: [n_name#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] Arguments: false -(53) VeloxAppendBatches +(53) VeloxResizeBatches Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (54) ColumnarExchange Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] @@ -348,9 +348,9 @@ Results [2]: [n_name#X, sum((l_extendedprice#X * (1 - l_discount#X)))#X AS reven Input [2]: [n_name#X, revenue#X] Arguments: false -(60) VeloxAppendBatches +(60) VeloxResizeBatches Input [2]: [n_name#X, revenue#X] -Arguments: X +Arguments: X, X (61) ColumnarExchange Input [2]: [n_name#X, revenue#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/6.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/6.txt index 64624c791f72..89cf24d87474 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/6.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/6.txt @@ -6,7 +6,7 @@ AdaptiveSparkPlan (20) +- ^ InputIteratorTransformer (10) +- ShuffleQueryStage (8), Statistics(X) +- ColumnarExchange (7) - +- VeloxAppendBatches (6) + +- VeloxResizeBatches (6) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ NoopFilter (2) @@ -46,9 +46,9 @@ Results [2]: [sum#X, isEmpty#X] Input [2]: [sum#X, isEmpty#X] Arguments: false -(6) VeloxAppendBatches +(6) VeloxResizeBatches Input [2]: [sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (7) ColumnarExchange Input [2]: [sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/7.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/7.txt index ae1ab637f805..7b0e80cded47 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/7.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/7.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (95) +- ^ InputIteratorTransformer (59) +- ShuffleQueryStage (57), Statistics(X) +- ColumnarExchange (56) - +- VeloxAppendBatches (55) + +- VeloxResizeBatches (55) +- ^ RegularHashAggregateExecTransformer (53) +- ^ InputIteratorTransformer (52) +- ShuffleQueryStage (50), Statistics(X) +- ColumnarExchange (49) - +- VeloxAppendBatches (48) + +- VeloxResizeBatches (48) +- ^ ProjectExecTransformer (46) +- ^ FlushableHashAggregateExecTransformer (45) +- ^ ProjectExecTransformer (44) @@ -289,9 +289,9 @@ Input [5]: [supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] @@ -318,9 +318,9 @@ Results [4]: [supp_nation#X, cust_nation#X, l_year#X, sum(volume#X)#X AS revenue Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] Arguments: false -(55) VeloxAppendBatches +(55) VeloxResizeBatches Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] -Arguments: X +Arguments: X, X (56) ColumnarExchange Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/8.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/8.txt index 4bbf7967744c..394c10ccb012 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/8.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/8.txt @@ -6,13 +6,13 @@ AdaptiveSparkPlan (131) +- ^ InputIteratorTransformer (83) +- ShuffleQueryStage (81), Statistics(X) +- ColumnarExchange (80) - +- VeloxAppendBatches (79) + +- VeloxResizeBatches (79) +- ^ ProjectExecTransformer (77) +- ^ RegularHashAggregateExecTransformer (76) +- ^ InputIteratorTransformer (75) +- ShuffleQueryStage (73), Statistics(X) +- ColumnarExchange (72) - +- VeloxAppendBatches (71) + +- VeloxResizeBatches (71) +- ^ ProjectExecTransformer (69) +- ^ FlushableHashAggregateExecTransformer (68) +- ^ ProjectExecTransformer (67) @@ -422,9 +422,9 @@ Input [5]: [o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] Arguments: false -(71) VeloxAppendBatches +(71) VeloxResizeBatches Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (72) ColumnarExchange Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] @@ -455,9 +455,9 @@ Input [3]: [o_year#X, sum(CASE WHEN (nation#X = BRAZIL) THEN volume#X ELSE 0.000 Input [2]: [o_year#X, mkt_share#X] Arguments: false -(79) VeloxAppendBatches +(79) VeloxResizeBatches Input [2]: [o_year#X, mkt_share#X] -Arguments: X +Arguments: X, X (80) ColumnarExchange Input [2]: [o_year#X, mkt_share#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/9.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/9.txt index 301c001eb793..f9fe0dbc9ac1 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/9.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/9.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (100) +- ^ InputIteratorTransformer (63) +- ShuffleQueryStage (61), Statistics(X) +- ColumnarExchange (60) - +- VeloxAppendBatches (59) + +- VeloxResizeBatches (59) +- ^ RegularHashAggregateExecTransformer (57) +- ^ InputIteratorTransformer (56) +- ShuffleQueryStage (54), Statistics(X) +- ColumnarExchange (53) - +- VeloxAppendBatches (52) + +- VeloxResizeBatches (52) +- ^ ProjectExecTransformer (50) +- ^ FlushableHashAggregateExecTransformer (49) +- ^ ProjectExecTransformer (48) @@ -313,9 +313,9 @@ Input [4]: [nation#X, o_year#X, sum#X, isEmpty#X] Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] Arguments: false -(52) VeloxAppendBatches +(52) VeloxResizeBatches Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (53) ColumnarExchange Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] @@ -342,9 +342,9 @@ Results [3]: [nation#X, o_year#X, sum(amount#X)#X AS sum_profit#X] Input [3]: [nation#X, o_year#X, sum_profit#X] Arguments: false -(59) VeloxAppendBatches +(59) VeloxResizeBatches Input [3]: [nation#X, o_year#X, sum_profit#X] -Arguments: X +Arguments: X, X (60) ColumnarExchange Input [3]: [nation#X, o_year#X, sum_profit#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/1.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/1.txt index 63b7d317f3cf..5ceb73b301db 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/1.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/1.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (30) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ RegularHashAggregateExecTransformer (12) +- ^ InputIteratorTransformer (11) +- ShuffleQueryStage (9) +- ColumnarExchange (8) - +- VeloxAppendBatches (7) + +- VeloxResizeBatches (7) +- ^ ProjectExecTransformer (5) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) @@ -58,9 +58,9 @@ Input [17]: [l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] Arguments: false -(7) VeloxAppendBatches +(7) VeloxResizeBatches Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] -Arguments: X +Arguments: X, X (8) ColumnarExchange Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] @@ -87,9 +87,9 @@ Results [10]: [l_returnflag#X, l_linestatus#X, sum(l_quantity#X)#X AS sum_qty#X, Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/10.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/10.txt index db7c1cb79667..5aeddac49db5 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/10.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/10.txt @@ -8,7 +8,7 @@ AdaptiveSparkPlan (68) +- ^ InputIteratorTransformer (39) +- ShuffleQueryStage (37) +- ColumnarExchange (36) - +- VeloxAppendBatches (35) + +- VeloxResizeBatches (35) +- ^ ProjectExecTransformer (33) +- ^ FlushableHashAggregateExecTransformer (32) +- ^ ProjectExecTransformer (31) @@ -210,9 +210,9 @@ Input [9]: [c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] Arguments: false -(35) VeloxAppendBatches +(35) VeloxResizeBatches Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (36) ColumnarExchange Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/11.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/11.txt index 33c7971d3749..7d95202ba98f 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/11.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/11.txt @@ -6,13 +6,13 @@ AdaptiveSparkPlan (60) +- ^ InputIteratorTransformer (37) +- ShuffleQueryStage (35) +- ColumnarExchange (34) - +- VeloxAppendBatches (33) + +- VeloxResizeBatches (33) +- ^ FilterExecTransformer (31) +- ^ RegularHashAggregateExecTransformer (30) +- ^ InputIteratorTransformer (29) +- ShuffleQueryStage (27) +- ColumnarExchange (26) - +- VeloxAppendBatches (25) + +- VeloxResizeBatches (25) +- ^ ProjectExecTransformer (23) +- ^ FlushableHashAggregateExecTransformer (22) +- ^ ProjectExecTransformer (21) @@ -160,9 +160,9 @@ Input [3]: [ps_partkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] Arguments: false -(25) VeloxAppendBatches +(25) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (26) ColumnarExchange Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] @@ -193,9 +193,9 @@ Arguments: (isnotnull(value#X) AND (cast(value#X as decimal(38,6)) > Subquery su Input [2]: [ps_partkey#X, value#X] Arguments: false -(33) VeloxAppendBatches +(33) VeloxResizeBatches Input [2]: [ps_partkey#X, value#X] -Arguments: X +Arguments: X, X (34) ColumnarExchange Input [2]: [ps_partkey#X, value#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/12.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/12.txt index f1f3a9234354..39d1eb13754e 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/12.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/12.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (44) +- ^ InputIteratorTransformer (27) +- ShuffleQueryStage (25) +- ColumnarExchange (24) - +- VeloxAppendBatches (23) + +- VeloxResizeBatches (23) +- ^ RegularHashAggregateExecTransformer (21) +- ^ InputIteratorTransformer (20) +- ShuffleQueryStage (18) +- ColumnarExchange (17) - +- VeloxAppendBatches (16) + +- VeloxResizeBatches (16) +- ^ ProjectExecTransformer (14) +- ^ FlushableHashAggregateExecTransformer (13) +- ^ ProjectExecTransformer (12) @@ -108,9 +108,9 @@ Input [3]: [l_shipmode#X, sum#X, sum#X] Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] Arguments: false -(16) VeloxAppendBatches +(16) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] -Arguments: X +Arguments: X, X (17) ColumnarExchange Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] @@ -137,9 +137,9 @@ Results [3]: [l_shipmode#X, sum(CASE WHEN ((o_orderpriority#X = 1-URGENT) OR (o_ Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] Arguments: false -(23) VeloxAppendBatches +(23) VeloxResizeBatches Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] -Arguments: X +Arguments: X, X (24) ColumnarExchange Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/13.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/13.txt index 13ef25b681c0..1033f582f2b9 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/13.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/13.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (55) +- ^ InputIteratorTransformer (36) +- ShuffleQueryStage (34) +- ColumnarExchange (33) - +- VeloxAppendBatches (32) + +- VeloxResizeBatches (32) +- ^ RegularHashAggregateExecTransformer (30) +- ^ InputIteratorTransformer (29) +- ShuffleQueryStage (27) +- ColumnarExchange (26) - +- VeloxAppendBatches (25) + +- VeloxResizeBatches (25) +- ^ ProjectExecTransformer (23) +- ^ FlushableHashAggregateExecTransformer (22) +- ^ ProjectExecTransformer (21) @@ -19,7 +19,7 @@ AdaptiveSparkPlan (55) +- ^ InputIteratorTransformer (19) +- ShuffleQueryStage (17) +- ColumnarExchange (16) - +- VeloxAppendBatches (15) + +- VeloxResizeBatches (15) +- ^ ProjectExecTransformer (13) +- ^ FlushableHashAggregateExecTransformer (12) +- ^ ProjectExecTransformer (11) @@ -112,9 +112,9 @@ Input [2]: [c_custkey#X, count#X] Input [3]: [hash_partition_key#X, c_custkey#X, count#X] Arguments: false -(15) VeloxAppendBatches +(15) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_custkey#X, count#X] -Arguments: X +Arguments: X, X (16) ColumnarExchange Input [3]: [hash_partition_key#X, c_custkey#X, count#X] @@ -156,9 +156,9 @@ Input [2]: [c_count#X, count#X] Input [3]: [hash_partition_key#X, c_count#X, count#X] Arguments: false -(25) VeloxAppendBatches +(25) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_count#X, count#X] -Arguments: X +Arguments: X, X (26) ColumnarExchange Input [3]: [hash_partition_key#X, c_count#X, count#X] @@ -185,9 +185,9 @@ Results [2]: [c_count#X, count(1)#X AS custdist#X] Input [2]: [c_count#X, custdist#X] Arguments: false -(32) VeloxAppendBatches +(32) VeloxResizeBatches Input [2]: [c_count#X, custdist#X] -Arguments: X +Arguments: X, X (33) ColumnarExchange Input [2]: [c_count#X, custdist#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/14.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/14.txt index c86f2215cda5..b9231f934ea8 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/14.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/14.txt @@ -7,7 +7,7 @@ AdaptiveSparkPlan (35) +- ^ InputIteratorTransformer (19) +- ShuffleQueryStage (17) +- ColumnarExchange (16) - +- VeloxAppendBatches (15) + +- VeloxResizeBatches (15) +- ^ FlushableHashAggregateExecTransformer (13) +- ^ ProjectExecTransformer (12) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (11) @@ -97,9 +97,9 @@ Results [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] Arguments: false -(15) VeloxAppendBatches +(15) VeloxResizeBatches Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (16) ColumnarExchange Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/15.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/15.txt index 8edb179e592f..f3b74c2c5104 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/15.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/15.txt @@ -6,7 +6,7 @@ AdaptiveSparkPlan (46) +- ^ InputIteratorTransformer (28) +- ShuffleQueryStage (26) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ ProjectExecTransformer (22) +- ^ BroadcastHashJoinExecTransformer Inner BuildLeft (21) :- ^ InputIteratorTransformer (7) @@ -19,7 +19,7 @@ AdaptiveSparkPlan (46) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FlushableHashAggregateExecTransformer (11) +- ^ ProjectExecTransformer (10) @@ -101,9 +101,9 @@ Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] @@ -143,9 +143,9 @@ Input [6]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, supplier_no#X, total_ Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/16.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/16.txt index 4d4d52c7d2e6..440e56487b6f 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/16.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/16.txt @@ -6,19 +6,19 @@ AdaptiveSparkPlan (59) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33) +- ColumnarExchange (32) - +- VeloxAppendBatches (31) + +- VeloxResizeBatches (31) +- ^ RegularHashAggregateExecTransformer (29) +- ^ InputIteratorTransformer (28) +- ShuffleQueryStage (26) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ ProjectExecTransformer (22) +- ^ FlushableHashAggregateExecTransformer (21) +- ^ RegularHashAggregateExecTransformer (20) +- ^ InputIteratorTransformer (19) +- ShuffleQueryStage (17) +- ColumnarExchange (16) - +- VeloxAppendBatches (15) + +- VeloxResizeBatches (15) +- ^ ProjectExecTransformer (13) +- ^ FlushableHashAggregateExecTransformer (12) +- ^ ProjectExecTransformer (11) @@ -117,9 +117,9 @@ Input [4]: [p_brand#X, p_type#X, p_size#X, ps_suppkey#X] Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] Arguments: false -(15) VeloxAppendBatches +(15) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] -Arguments: X +Arguments: X, X (16) ColumnarExchange Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] @@ -157,9 +157,9 @@ Input [4]: [p_brand#X, p_type#X, p_size#X, count#X] Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] @@ -186,9 +186,9 @@ Results [4]: [p_brand#X, p_type#X, p_size#X, count(ps_suppkey#X)#X AS supplier_c Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/17.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/17.txt index 7c5359849d4e..7f4e4e666c0d 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/17.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/17.txt @@ -7,7 +7,7 @@ AdaptiveSparkPlan (36) +- ^ InputIteratorTransformer (11) +- ShuffleQueryStage (9) +- ColumnarExchange (8) - +- VeloxAppendBatches (7) + +- VeloxResizeBatches (7) +- ^ FlushableHashAggregateExecTransformer (5) +- ^ InputIteratorTransformer (4) +- RowToVeloxColumnar (2) @@ -59,9 +59,9 @@ Results [2]: [sum#X, isEmpty#X] Input [2]: [sum#X, isEmpty#X] Arguments: false -(7) VeloxAppendBatches +(7) VeloxResizeBatches Input [2]: [sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (8) ColumnarExchange Input [2]: [sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/18.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/18.txt index a4aaf08ff8bb..6da312afa5f9 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/18.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/18.txt @@ -7,7 +7,7 @@ AdaptiveSparkPlan (88) +- ^ InputIteratorTransformer (51) +- ShuffleQueryStage (49) +- ColumnarExchange (48) - +- VeloxAppendBatches (47) + +- VeloxResizeBatches (47) +- ^ ProjectExecTransformer (45) +- ^ FlushableHashAggregateExecTransformer (44) +- ^ ProjectExecTransformer (43) @@ -31,7 +31,7 @@ AdaptiveSparkPlan (88) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16) : +- ColumnarExchange (15) - : +- VeloxAppendBatches (14) + : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ FlushableHashAggregateExecTransformer (11) : +- ^ Scan parquet (10) @@ -140,9 +140,9 @@ Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] @@ -276,9 +276,9 @@ Input [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, Input [8]: [hash_partition_key#X, c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] Arguments: false -(47) VeloxAppendBatches +(47) VeloxResizeBatches Input [8]: [hash_partition_key#X, c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (48) ColumnarExchange Input [8]: [hash_partition_key#X, c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/19.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/19.txt index 87acf3c4b28e..835386f95fdc 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/19.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/19.txt @@ -6,7 +6,7 @@ AdaptiveSparkPlan (34) +- ^ InputIteratorTransformer (19) +- ShuffleQueryStage (17) +- ColumnarExchange (16) - +- VeloxAppendBatches (15) + +- VeloxResizeBatches (15) +- ^ FlushableHashAggregateExecTransformer (13) +- ^ ProjectExecTransformer (12) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (11) @@ -96,9 +96,9 @@ Results [2]: [sum#X, isEmpty#X] Input [2]: [sum#X, isEmpty#X] Arguments: false -(15) VeloxAppendBatches +(15) VeloxResizeBatches Input [2]: [sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (16) ColumnarExchange Input [2]: [sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/20.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/20.txt index 9e03d8319537..51eaaa2a51b9 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/20.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/20.txt @@ -6,7 +6,7 @@ AdaptiveSparkPlan (112) +- ^ InputIteratorTransformer (70) +- ShuffleQueryStage (68) +- ColumnarExchange (67) - +- VeloxAppendBatches (66) + +- VeloxResizeBatches (66) +- ^ ProjectExecTransformer (64) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (63) :- ^ ProjectExecTransformer (54) @@ -15,7 +15,7 @@ AdaptiveSparkPlan (112) : : +- AQEShuffleRead (8) : : +- ShuffleQueryStage (7) : : +- ColumnarExchange (6) - : : +- VeloxAppendBatches (5) + : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) : : +- ^ Scan parquet (1) @@ -42,7 +42,7 @@ AdaptiveSparkPlan (112) : +- ^ InputIteratorTransformer (42) : +- ShuffleQueryStage (40) : +- ColumnarExchange (39) - : +- VeloxAppendBatches (38) + : +- VeloxResizeBatches (38) : +- ^ ProjectExecTransformer (36) : +- ^ FlushableHashAggregateExecTransformer (35) : +- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (34) @@ -118,9 +118,9 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] @@ -255,9 +255,9 @@ Input [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] Input [5]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] Arguments: false -(38) VeloxAppendBatches +(38) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (39) ColumnarExchange Input [5]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] @@ -370,9 +370,9 @@ Input [4]: [s_name#X, s_address#X, s_nationkey#X, n_nationkey#X] Input [2]: [s_name#X, s_address#X] Arguments: false -(66) VeloxAppendBatches +(66) VeloxResizeBatches Input [2]: [s_name#X, s_address#X] -Arguments: X +Arguments: X, X (67) ColumnarExchange Input [2]: [s_name#X, s_address#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/21.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/21.txt index ee28d7b592e8..875f05406e85 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/21.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/21.txt @@ -7,7 +7,7 @@ AdaptiveSparkPlan (93) +- ^ InputIteratorTransformer (56) +- ShuffleQueryStage (54) +- ColumnarExchange (53) - +- VeloxAppendBatches (52) + +- VeloxResizeBatches (52) +- ^ ProjectExecTransformer (50) +- ^ FlushableHashAggregateExecTransformer (49) +- ^ ProjectExecTransformer (48) @@ -302,9 +302,9 @@ Input [2]: [s_name#X, count#X] Input [3]: [hash_partition_key#X, s_name#X, count#X] Arguments: false -(52) VeloxAppendBatches +(52) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_name#X, count#X] -Arguments: X +Arguments: X, X (53) ColumnarExchange Input [3]: [hash_partition_key#X, s_name#X, count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/22.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/22.txt index d578b43f3d6a..e0b3b12c7b96 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/22.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/22.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (40) +- ^ InputIteratorTransformer (25) +- ShuffleQueryStage (23) +- ColumnarExchange (22) - +- VeloxAppendBatches (21) + +- VeloxResizeBatches (21) +- ^ RegularHashAggregateExecTransformer (19) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FlushableHashAggregateExecTransformer (11) +- ^ ProjectExecTransformer (10) @@ -95,9 +95,9 @@ Input [4]: [cntrycode#X, count#X, sum#X, isEmpty#X] Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] @@ -124,9 +124,9 @@ Results [3]: [cntrycode#X, count(1)#X AS numcust#X, sum(c_acctbal#X)#X AS totacc Input [3]: [cntrycode#X, numcust#X, totacctbal#X] Arguments: false -(21) VeloxAppendBatches +(21) VeloxResizeBatches Input [3]: [cntrycode#X, numcust#X, totacctbal#X] -Arguments: X +Arguments: X, X (22) ColumnarExchange Input [3]: [cntrycode#X, numcust#X, totacctbal#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/3.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/3.txt index 2ffd75b92964..ed5f34e91889 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/3.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/3.txt @@ -8,7 +8,7 @@ AdaptiveSparkPlan (54) +- ^ InputIteratorTransformer (30) +- ShuffleQueryStage (28) +- ColumnarExchange (27) - +- VeloxAppendBatches (26) + +- VeloxResizeBatches (26) +- ^ ProjectExecTransformer (24) +- ^ FlushableHashAggregateExecTransformer (23) +- ^ ProjectExecTransformer (22) @@ -160,9 +160,9 @@ Input [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] Input [6]: [hash_partition_key#X, l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] Arguments: false -(26) VeloxAppendBatches +(26) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (27) ColumnarExchange Input [6]: [hash_partition_key#X, l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/4.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/4.txt index 36c6de5374ba..a24a32699b79 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/4.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/4.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (46) +- ^ InputIteratorTransformer (28) +- ShuffleQueryStage (26) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ RegularHashAggregateExecTransformer (22) +- ^ InputIteratorTransformer (21) +- ShuffleQueryStage (19) +- ColumnarExchange (18) - +- VeloxAppendBatches (17) + +- VeloxResizeBatches (17) +- ^ ProjectExecTransformer (15) +- ^ FlushableHashAggregateExecTransformer (14) +- ^ ProjectExecTransformer (13) @@ -114,9 +114,9 @@ Input [2]: [o_orderpriority#X, count#X] Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] Arguments: false -(17) VeloxAppendBatches +(17) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] -Arguments: X +Arguments: X, X (18) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] @@ -143,9 +143,9 @@ Results [2]: [o_orderpriority#X, count(1)#X AS order_count#X] Input [2]: [o_orderpriority#X, order_count#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [2]: [o_orderpriority#X, order_count#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [2]: [o_orderpriority#X, order_count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/5.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/5.txt index 49ee27485b4a..178ee7c7e2f3 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/5.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/5.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (102) +- ^ InputIteratorTransformer (64) +- ShuffleQueryStage (62) +- ColumnarExchange (61) - +- VeloxAppendBatches (60) + +- VeloxResizeBatches (60) +- ^ RegularHashAggregateExecTransformer (58) +- ^ InputIteratorTransformer (57) +- ShuffleQueryStage (55) +- ColumnarExchange (54) - +- VeloxAppendBatches (53) + +- VeloxResizeBatches (53) +- ^ ProjectExecTransformer (51) +- ^ FlushableHashAggregateExecTransformer (50) +- ^ ProjectExecTransformer (49) @@ -314,9 +314,9 @@ Input [3]: [n_name#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] Arguments: false -(53) VeloxAppendBatches +(53) VeloxResizeBatches Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (54) ColumnarExchange Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] @@ -343,9 +343,9 @@ Results [2]: [n_name#X, sum(CheckOverflow((promote_precision(cast(l_extendedpric Input [2]: [n_name#X, revenue#X] Arguments: false -(60) VeloxAppendBatches +(60) VeloxResizeBatches Input [2]: [n_name#X, revenue#X] -Arguments: X +Arguments: X, X (61) ColumnarExchange Input [2]: [n_name#X, revenue#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/6.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/6.txt index 786a89fe715a..ee8c494f56a3 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/6.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/6.txt @@ -6,7 +6,7 @@ AdaptiveSparkPlan (20) +- ^ InputIteratorTransformer (10) +- ShuffleQueryStage (8) +- ColumnarExchange (7) - +- VeloxAppendBatches (6) + +- VeloxResizeBatches (6) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ FilterExecTransformer (2) @@ -46,9 +46,9 @@ Results [2]: [sum#X, isEmpty#X] Input [2]: [sum#X, isEmpty#X] Arguments: false -(6) VeloxAppendBatches +(6) VeloxResizeBatches Input [2]: [sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (7) ColumnarExchange Input [2]: [sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/7.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/7.txt index 2ba42f806f3e..f0e7e37d3887 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/7.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/7.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (95) +- ^ InputIteratorTransformer (59) +- ShuffleQueryStage (57) +- ColumnarExchange (56) - +- VeloxAppendBatches (55) + +- VeloxResizeBatches (55) +- ^ RegularHashAggregateExecTransformer (53) +- ^ InputIteratorTransformer (52) +- ShuffleQueryStage (50) +- ColumnarExchange (49) - +- VeloxAppendBatches (48) + +- VeloxResizeBatches (48) +- ^ ProjectExecTransformer (46) +- ^ FlushableHashAggregateExecTransformer (45) +- ^ ProjectExecTransformer (44) @@ -284,9 +284,9 @@ Input [5]: [supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] @@ -313,9 +313,9 @@ Results [4]: [supp_nation#X, cust_nation#X, l_year#X, sum(volume#X)#X AS revenue Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] Arguments: false -(55) VeloxAppendBatches +(55) VeloxResizeBatches Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] -Arguments: X +Arguments: X, X (56) ColumnarExchange Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/8.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/8.txt index 64a40563c238..9d9abd6b04bf 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/8.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/8.txt @@ -6,13 +6,13 @@ AdaptiveSparkPlan (131) +- ^ InputIteratorTransformer (83) +- ShuffleQueryStage (81) +- ColumnarExchange (80) - +- VeloxAppendBatches (79) + +- VeloxResizeBatches (79) +- ^ ProjectExecTransformer (77) +- ^ RegularHashAggregateExecTransformer (76) +- ^ InputIteratorTransformer (75) +- ShuffleQueryStage (73) +- ColumnarExchange (72) - +- VeloxAppendBatches (71) + +- VeloxResizeBatches (71) +- ^ ProjectExecTransformer (69) +- ^ FlushableHashAggregateExecTransformer (68) +- ^ ProjectExecTransformer (67) @@ -415,9 +415,9 @@ Input [5]: [o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] Arguments: false -(71) VeloxAppendBatches +(71) VeloxResizeBatches Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (72) ColumnarExchange Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] @@ -448,9 +448,9 @@ Input [3]: [o_year#X, sum(CASE WHEN (nation#X = BRAZIL) THEN volume#X ELSE 0.000 Input [2]: [o_year#X, mkt_share#X] Arguments: false -(79) VeloxAppendBatches +(79) VeloxResizeBatches Input [2]: [o_year#X, mkt_share#X] -Arguments: X +Arguments: X, X (80) ColumnarExchange Input [2]: [o_year#X, mkt_share#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/9.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/9.txt index b5a21a0261d2..2b213b681a05 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/9.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/9.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (100) +- ^ InputIteratorTransformer (63) +- ShuffleQueryStage (61) +- ColumnarExchange (60) - +- VeloxAppendBatches (59) + +- VeloxResizeBatches (59) +- ^ RegularHashAggregateExecTransformer (57) +- ^ InputIteratorTransformer (56) +- ShuffleQueryStage (54) +- ColumnarExchange (53) - +- VeloxAppendBatches (52) + +- VeloxResizeBatches (52) +- ^ ProjectExecTransformer (50) +- ^ FlushableHashAggregateExecTransformer (49) +- ^ ProjectExecTransformer (48) @@ -308,9 +308,9 @@ Input [4]: [nation#X, o_year#X, sum#X, isEmpty#X] Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] Arguments: false -(52) VeloxAppendBatches +(52) VeloxResizeBatches Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (53) ColumnarExchange Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] @@ -337,9 +337,9 @@ Results [3]: [nation#X, o_year#X, sum(amount#X)#X AS sum_profit#X] Input [3]: [nation#X, o_year#X, sum_profit#X] Arguments: false -(59) VeloxAppendBatches +(59) VeloxResizeBatches Input [3]: [nation#X, o_year#X, sum_profit#X] -Arguments: X +Arguments: X, X (60) ColumnarExchange Input [3]: [nation#X, o_year#X, sum_profit#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/1.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/1.txt index 22dd5100c4fb..12c19c45e38d 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/1.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/1.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (30) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ RegularHashAggregateExecTransformer (12) +- ^ InputIteratorTransformer (11) +- ShuffleQueryStage (9), Statistics(X) +- ColumnarExchange (8) - +- VeloxAppendBatches (7) + +- VeloxResizeBatches (7) +- ^ ProjectExecTransformer (5) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) @@ -58,9 +58,9 @@ Input [17]: [l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] Arguments: false -(7) VeloxAppendBatches +(7) VeloxResizeBatches Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] -Arguments: X +Arguments: X, X (8) ColumnarExchange Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] @@ -87,9 +87,9 @@ Results [10]: [l_returnflag#X, l_linestatus#X, sum(l_quantity#X)#X AS sum_qty#X, Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/10.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/10.txt index 8681ecf5f93f..adaeaf49efee 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/10.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/10.txt @@ -8,7 +8,7 @@ AdaptiveSparkPlan (68) +- ^ InputIteratorTransformer (39) +- ShuffleQueryStage (37), Statistics(X) +- ColumnarExchange (36) - +- VeloxAppendBatches (35) + +- VeloxResizeBatches (35) +- ^ ProjectExecTransformer (33) +- ^ FlushableHashAggregateExecTransformer (32) +- ^ ProjectExecTransformer (31) @@ -210,9 +210,9 @@ Input [9]: [c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] Arguments: false -(35) VeloxAppendBatches +(35) VeloxResizeBatches Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (36) ColumnarExchange Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/11.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/11.txt index 6dfc1d00f4cc..25d3c4ac2f75 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/11.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/11.txt @@ -6,13 +6,13 @@ AdaptiveSparkPlan (60) +- ^ InputIteratorTransformer (37) +- ShuffleQueryStage (35), Statistics(X) +- ColumnarExchange (34) - +- VeloxAppendBatches (33) + +- VeloxResizeBatches (33) +- ^ FilterExecTransformer (31) +- ^ RegularHashAggregateExecTransformer (30) +- ^ InputIteratorTransformer (29) +- ShuffleQueryStage (27), Statistics(X) +- ColumnarExchange (26) - +- VeloxAppendBatches (25) + +- VeloxResizeBatches (25) +- ^ ProjectExecTransformer (23) +- ^ FlushableHashAggregateExecTransformer (22) +- ^ ProjectExecTransformer (21) @@ -160,9 +160,9 @@ Input [3]: [ps_partkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] Arguments: false -(25) VeloxAppendBatches +(25) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (26) ColumnarExchange Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] @@ -193,9 +193,9 @@ Arguments: (isnotnull(value#X) AND (cast(value#X as decimal(38,6)) > Subquery su Input [2]: [ps_partkey#X, value#X] Arguments: false -(33) VeloxAppendBatches +(33) VeloxResizeBatches Input [2]: [ps_partkey#X, value#X] -Arguments: X +Arguments: X, X (34) ColumnarExchange Input [2]: [ps_partkey#X, value#X] @@ -330,7 +330,7 @@ AdaptiveSparkPlan (102) +- ^ InputIteratorTransformer (81) +- ShuffleQueryStage (79), Statistics(X) +- ColumnarExchange (78) - +- VeloxAppendBatches (77) + +- VeloxResizeBatches (77) +- ^ FlushableHashAggregateExecTransformer (75) +- ^ ProjectExecTransformer (74) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (73) @@ -429,9 +429,9 @@ Results [2]: [sum#X, isEmpty#X] Input [2]: [sum#X, isEmpty#X] Arguments: false -(77) VeloxAppendBatches +(77) VeloxResizeBatches Input [2]: [sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (78) ColumnarExchange Input [2]: [sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/12.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/12.txt index 1e1ad6d497a4..21681feeacc1 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/12.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/12.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (44) +- ^ InputIteratorTransformer (27) +- ShuffleQueryStage (25), Statistics(X) +- ColumnarExchange (24) - +- VeloxAppendBatches (23) + +- VeloxResizeBatches (23) +- ^ RegularHashAggregateExecTransformer (21) +- ^ InputIteratorTransformer (20) +- ShuffleQueryStage (18), Statistics(X) +- ColumnarExchange (17) - +- VeloxAppendBatches (16) + +- VeloxResizeBatches (16) +- ^ ProjectExecTransformer (14) +- ^ FlushableHashAggregateExecTransformer (13) +- ^ ProjectExecTransformer (12) @@ -108,9 +108,9 @@ Input [3]: [l_shipmode#X, sum#X, sum#X] Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] Arguments: false -(16) VeloxAppendBatches +(16) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] -Arguments: X +Arguments: X, X (17) ColumnarExchange Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] @@ -137,9 +137,9 @@ Results [3]: [l_shipmode#X, sum(CASE WHEN ((o_orderpriority#X = 1-URGENT) OR (o_ Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] Arguments: false -(23) VeloxAppendBatches +(23) VeloxResizeBatches Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] -Arguments: X +Arguments: X, X (24) ColumnarExchange Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/13.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/13.txt index ff6532aa6579..808e8189cca4 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/13.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/13.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (55) +- ^ InputIteratorTransformer (36) +- ShuffleQueryStage (34), Statistics(X) +- ColumnarExchange (33) - +- VeloxAppendBatches (32) + +- VeloxResizeBatches (32) +- ^ RegularHashAggregateExecTransformer (30) +- ^ InputIteratorTransformer (29) +- ShuffleQueryStage (27), Statistics(X) +- ColumnarExchange (26) - +- VeloxAppendBatches (25) + +- VeloxResizeBatches (25) +- ^ ProjectExecTransformer (23) +- ^ FlushableHashAggregateExecTransformer (22) +- ^ ProjectExecTransformer (21) @@ -19,7 +19,7 @@ AdaptiveSparkPlan (55) +- ^ InputIteratorTransformer (19) +- ShuffleQueryStage (17), Statistics(X) +- ColumnarExchange (16) - +- VeloxAppendBatches (15) + +- VeloxResizeBatches (15) +- ^ ProjectExecTransformer (13) +- ^ FlushableHashAggregateExecTransformer (12) +- ^ ProjectExecTransformer (11) @@ -112,9 +112,9 @@ Input [2]: [c_custkey#X, count#X] Input [3]: [hash_partition_key#X, c_custkey#X, count#X] Arguments: false -(15) VeloxAppendBatches +(15) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_custkey#X, count#X] -Arguments: X +Arguments: X, X (16) ColumnarExchange Input [3]: [hash_partition_key#X, c_custkey#X, count#X] @@ -156,9 +156,9 @@ Input [2]: [c_count#X, count#X] Input [3]: [hash_partition_key#X, c_count#X, count#X] Arguments: false -(25) VeloxAppendBatches +(25) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_count#X, count#X] -Arguments: X +Arguments: X, X (26) ColumnarExchange Input [3]: [hash_partition_key#X, c_count#X, count#X] @@ -185,9 +185,9 @@ Results [2]: [c_count#X, count(1)#X AS custdist#X] Input [2]: [c_count#X, custdist#X] Arguments: false -(32) VeloxAppendBatches +(32) VeloxResizeBatches Input [2]: [c_count#X, custdist#X] -Arguments: X +Arguments: X, X (33) ColumnarExchange Input [2]: [c_count#X, custdist#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/14.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/14.txt index d144dd39ca1d..6ac8b1cd94d3 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/14.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/14.txt @@ -7,7 +7,7 @@ AdaptiveSparkPlan (35) +- ^ InputIteratorTransformer (19) +- ShuffleQueryStage (17), Statistics(X) +- ColumnarExchange (16) - +- VeloxAppendBatches (15) + +- VeloxResizeBatches (15) +- ^ FlushableHashAggregateExecTransformer (13) +- ^ ProjectExecTransformer (12) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (11) @@ -97,9 +97,9 @@ Results [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] Arguments: false -(15) VeloxAppendBatches +(15) VeloxResizeBatches Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (16) ColumnarExchange Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/15.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/15.txt index c4f825f5ca53..671000b2f28d 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/15.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/15.txt @@ -5,7 +5,7 @@ AdaptiveSparkPlan (43) +- AQEShuffleRead (27) +- ShuffleQueryStage (26), Statistics(X) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ ProjectExecTransformer (22) +- ^ BroadcastHashJoinExecTransformer Inner BuildLeft (21) :- ^ InputIteratorTransformer (7) @@ -18,7 +18,7 @@ AdaptiveSparkPlan (43) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FlushableHashAggregateExecTransformer (11) +- ^ ProjectExecTransformer (10) @@ -100,9 +100,9 @@ Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] @@ -142,9 +142,9 @@ Input [6]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, supplier_no#X, total_ Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] @@ -247,7 +247,7 @@ AdaptiveSparkPlan (69) +- ^ InputIteratorTransformer (54) +- ShuffleQueryStage (52), Statistics(X) +- ColumnarExchange (51) - +- VeloxAppendBatches (50) + +- VeloxResizeBatches (50) +- ^ ProjectExecTransformer (48) +- ^ FlushableHashAggregateExecTransformer (47) +- ^ ProjectExecTransformer (46) @@ -294,9 +294,9 @@ Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] Arguments: false -(50) VeloxAppendBatches +(50) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (51) ColumnarExchange Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/16.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/16.txt index 741b837976a4..19e334e888ec 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/16.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/16.txt @@ -6,19 +6,19 @@ AdaptiveSparkPlan (59) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33), Statistics(X) +- ColumnarExchange (32) - +- VeloxAppendBatches (31) + +- VeloxResizeBatches (31) +- ^ RegularHashAggregateExecTransformer (29) +- ^ InputIteratorTransformer (28) +- ShuffleQueryStage (26), Statistics(X) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ ProjectExecTransformer (22) +- ^ FlushableHashAggregateExecTransformer (21) +- ^ RegularHashAggregateExecTransformer (20) +- ^ InputIteratorTransformer (19) +- ShuffleQueryStage (17), Statistics(X) +- ColumnarExchange (16) - +- VeloxAppendBatches (15) + +- VeloxResizeBatches (15) +- ^ ProjectExecTransformer (13) +- ^ FlushableHashAggregateExecTransformer (12) +- ^ ProjectExecTransformer (11) @@ -117,9 +117,9 @@ Input [4]: [p_brand#X, p_type#X, p_size#X, ps_suppkey#X] Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] Arguments: false -(15) VeloxAppendBatches +(15) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] -Arguments: X +Arguments: X, X (16) ColumnarExchange Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] @@ -157,9 +157,9 @@ Input [4]: [p_brand#X, p_type#X, p_size#X, count#X] Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] @@ -186,9 +186,9 @@ Results [4]: [p_brand#X, p_type#X, p_size#X, count(ps_suppkey#X)#X AS supplier_c Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/17.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/17.txt index 1e2ed970aef4..38801e9f690a 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/17.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/17.txt @@ -7,7 +7,7 @@ AdaptiveSparkPlan (36) +- ^ InputIteratorTransformer (11) +- ShuffleQueryStage (9), Statistics(X) +- ColumnarExchange (8) - +- VeloxAppendBatches (7) + +- VeloxResizeBatches (7) +- ^ FlushableHashAggregateExecTransformer (5) +- ^ InputIteratorTransformer (4) +- RowToVeloxColumnar (2) @@ -59,9 +59,9 @@ Results [2]: [sum#X, isEmpty#X] Input [2]: [sum#X, isEmpty#X] Arguments: false -(7) VeloxAppendBatches +(7) VeloxResizeBatches Input [2]: [sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (8) ColumnarExchange Input [2]: [sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/18.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/18.txt index 05884fea7885..26995352c1cf 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/18.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/18.txt @@ -7,7 +7,7 @@ AdaptiveSparkPlan (88) +- ^ InputIteratorTransformer (51) +- ShuffleQueryStage (49), Statistics(X) +- ColumnarExchange (48) - +- VeloxAppendBatches (47) + +- VeloxResizeBatches (47) +- ^ ProjectExecTransformer (45) +- ^ FlushableHashAggregateExecTransformer (44) +- ^ ProjectExecTransformer (43) @@ -31,7 +31,7 @@ AdaptiveSparkPlan (88) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16), Statistics(X) : +- ColumnarExchange (15) - : +- VeloxAppendBatches (14) + : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ FlushableHashAggregateExecTransformer (11) : +- ^ Scan parquet (10) @@ -140,9 +140,9 @@ Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] @@ -276,9 +276,9 @@ Input [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, Input [8]: [hash_partition_key#X, c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] Arguments: false -(47) VeloxAppendBatches +(47) VeloxResizeBatches Input [8]: [hash_partition_key#X, c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (48) ColumnarExchange Input [8]: [hash_partition_key#X, c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/19.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/19.txt index f04466031352..ce6a3f10da03 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/19.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/19.txt @@ -6,7 +6,7 @@ AdaptiveSparkPlan (34) +- ^ InputIteratorTransformer (19) +- ShuffleQueryStage (17), Statistics(X) +- ColumnarExchange (16) - +- VeloxAppendBatches (15) + +- VeloxResizeBatches (15) +- ^ FlushableHashAggregateExecTransformer (13) +- ^ ProjectExecTransformer (12) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (11) @@ -96,9 +96,9 @@ Results [2]: [sum#X, isEmpty#X] Input [2]: [sum#X, isEmpty#X] Arguments: false -(15) VeloxAppendBatches +(15) VeloxResizeBatches Input [2]: [sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (16) ColumnarExchange Input [2]: [sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/20.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/20.txt index 5cd3c9d35c2c..125e495397e8 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/20.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/20.txt @@ -5,7 +5,7 @@ AdaptiveSparkPlan (109) +- AQEShuffleRead (69) +- ShuffleQueryStage (68), Statistics(X) +- ColumnarExchange (67) - +- VeloxAppendBatches (66) + +- VeloxResizeBatches (66) +- ^ ProjectExecTransformer (64) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (63) :- ^ ProjectExecTransformer (54) @@ -14,7 +14,7 @@ AdaptiveSparkPlan (109) : : +- AQEShuffleRead (8) : : +- ShuffleQueryStage (7), Statistics(X) : : +- ColumnarExchange (6) - : : +- VeloxAppendBatches (5) + : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) : : +- ^ Scan parquet (1) @@ -41,7 +41,7 @@ AdaptiveSparkPlan (109) : +- ^ InputIteratorTransformer (42) : +- ShuffleQueryStage (40), Statistics(X) : +- ColumnarExchange (39) - : +- VeloxAppendBatches (38) + : +- VeloxResizeBatches (38) : +- ^ ProjectExecTransformer (36) : +- ^ FlushableHashAggregateExecTransformer (35) : +- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (34) @@ -117,9 +117,9 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] @@ -254,9 +254,9 @@ Input [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] Input [5]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] Arguments: false -(38) VeloxAppendBatches +(38) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (39) ColumnarExchange Input [5]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] @@ -369,9 +369,9 @@ Input [4]: [s_name#X, s_address#X, s_nationkey#X, n_nationkey#X] Input [2]: [s_name#X, s_address#X] Arguments: false -(66) VeloxAppendBatches +(66) VeloxResizeBatches Input [2]: [s_name#X, s_address#X] -Arguments: X +Arguments: X, X (67) ColumnarExchange Input [2]: [s_name#X, s_address#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/21.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/21.txt index 3ffdec004af2..6e411317dd33 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/21.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/21.txt @@ -6,7 +6,7 @@ AdaptiveSparkPlan (92) +- ^ InputIteratorTransformer (56) +- ShuffleQueryStage (54), Statistics(X) +- ColumnarExchange (53) - +- VeloxAppendBatches (52) + +- VeloxResizeBatches (52) +- ^ ProjectExecTransformer (50) +- ^ FlushableHashAggregateExecTransformer (49) +- ^ ProjectExecTransformer (48) @@ -301,9 +301,9 @@ Input [2]: [s_name#X, count#X] Input [3]: [hash_partition_key#X, s_name#X, count#X] Arguments: false -(52) VeloxAppendBatches +(52) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_name#X, count#X] -Arguments: X +Arguments: X, X (53) ColumnarExchange Input [3]: [hash_partition_key#X, s_name#X, count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/22.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/22.txt index f30752effaa4..a4883c6228f4 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/22.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/22.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (40) +- ^ InputIteratorTransformer (25) +- ShuffleQueryStage (23), Statistics(X) +- ColumnarExchange (22) - +- VeloxAppendBatches (21) + +- VeloxResizeBatches (21) +- ^ RegularHashAggregateExecTransformer (19) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FlushableHashAggregateExecTransformer (11) +- ^ ProjectExecTransformer (10) @@ -95,9 +95,9 @@ Input [4]: [cntrycode#X, count#X, sum#X, isEmpty#X] Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] @@ -124,9 +124,9 @@ Results [3]: [cntrycode#X, count(1)#X AS numcust#X, sum(c_acctbal#X)#X AS totacc Input [3]: [cntrycode#X, numcust#X, totacctbal#X] Arguments: false -(21) VeloxAppendBatches +(21) VeloxResizeBatches Input [3]: [cntrycode#X, numcust#X, totacctbal#X] -Arguments: X +Arguments: X, X (22) ColumnarExchange Input [3]: [cntrycode#X, numcust#X, totacctbal#X] @@ -223,7 +223,7 @@ AdaptiveSparkPlan (60) +- ^ InputIteratorTransformer (50) +- ShuffleQueryStage (48), Statistics(X) +- ColumnarExchange (47) - +- VeloxAppendBatches (46) + +- VeloxResizeBatches (46) +- ^ FlushableHashAggregateExecTransformer (44) +- ^ ProjectExecTransformer (43) +- ^ FilterExecTransformer (42) @@ -263,9 +263,9 @@ Results [2]: [sum#X, count#X] Input [2]: [sum#X, count#X] Arguments: false -(46) VeloxAppendBatches +(46) VeloxResizeBatches Input [2]: [sum#X, count#X] -Arguments: X +Arguments: X, X (47) ColumnarExchange Input [2]: [sum#X, count#X] @@ -340,7 +340,7 @@ AdaptiveSparkPlan (60) +- ^ InputIteratorTransformer (50) +- ShuffleQueryStage (48), Statistics(X) +- ColumnarExchange (47) - +- VeloxAppendBatches (46) + +- VeloxResizeBatches (46) +- ^ FlushableHashAggregateExecTransformer (44) +- ^ ProjectExecTransformer (43) +- ^ FilterExecTransformer (42) diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/3.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/3.txt index d9f87ca9b538..41aa9b576a9d 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/3.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/3.txt @@ -8,7 +8,7 @@ AdaptiveSparkPlan (54) +- ^ InputIteratorTransformer (30) +- ShuffleQueryStage (28), Statistics(X) +- ColumnarExchange (27) - +- VeloxAppendBatches (26) + +- VeloxResizeBatches (26) +- ^ ProjectExecTransformer (24) +- ^ FlushableHashAggregateExecTransformer (23) +- ^ ProjectExecTransformer (22) @@ -160,9 +160,9 @@ Input [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] Input [6]: [hash_partition_key#X, l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] Arguments: false -(26) VeloxAppendBatches +(26) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (27) ColumnarExchange Input [6]: [hash_partition_key#X, l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/4.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/4.txt index db9f7716a365..7f87abe3e7a2 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/4.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/4.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (46) +- ^ InputIteratorTransformer (28) +- ShuffleQueryStage (26), Statistics(X) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ RegularHashAggregateExecTransformer (22) +- ^ InputIteratorTransformer (21) +- ShuffleQueryStage (19), Statistics(X) +- ColumnarExchange (18) - +- VeloxAppendBatches (17) + +- VeloxResizeBatches (17) +- ^ ProjectExecTransformer (15) +- ^ FlushableHashAggregateExecTransformer (14) +- ^ ProjectExecTransformer (13) @@ -114,9 +114,9 @@ Input [2]: [o_orderpriority#X, count#X] Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] Arguments: false -(17) VeloxAppendBatches +(17) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] -Arguments: X +Arguments: X, X (18) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] @@ -143,9 +143,9 @@ Results [2]: [o_orderpriority#X, count(1)#X AS order_count#X] Input [2]: [o_orderpriority#X, order_count#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [2]: [o_orderpriority#X, order_count#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [2]: [o_orderpriority#X, order_count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/5.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/5.txt index 67e9d847aefa..6780a88b37da 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/5.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/5.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (102) +- ^ InputIteratorTransformer (64) +- ShuffleQueryStage (62), Statistics(X) +- ColumnarExchange (61) - +- VeloxAppendBatches (60) + +- VeloxResizeBatches (60) +- ^ RegularHashAggregateExecTransformer (58) +- ^ InputIteratorTransformer (57) +- ShuffleQueryStage (55), Statistics(X) +- ColumnarExchange (54) - +- VeloxAppendBatches (53) + +- VeloxResizeBatches (53) +- ^ ProjectExecTransformer (51) +- ^ FlushableHashAggregateExecTransformer (50) +- ^ ProjectExecTransformer (49) @@ -314,9 +314,9 @@ Input [3]: [n_name#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] Arguments: false -(53) VeloxAppendBatches +(53) VeloxResizeBatches Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (54) ColumnarExchange Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] @@ -343,9 +343,9 @@ Results [2]: [n_name#X, sum(CheckOverflow((promote_precision(cast(l_extendedpric Input [2]: [n_name#X, revenue#X] Arguments: false -(60) VeloxAppendBatches +(60) VeloxResizeBatches Input [2]: [n_name#X, revenue#X] -Arguments: X +Arguments: X, X (61) ColumnarExchange Input [2]: [n_name#X, revenue#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/6.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/6.txt index 3432579a0de0..864cdf3832e1 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/6.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/6.txt @@ -6,7 +6,7 @@ AdaptiveSparkPlan (20) +- ^ InputIteratorTransformer (10) +- ShuffleQueryStage (8), Statistics(X) +- ColumnarExchange (7) - +- VeloxAppendBatches (6) + +- VeloxResizeBatches (6) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ FilterExecTransformer (2) @@ -46,9 +46,9 @@ Results [2]: [sum#X, isEmpty#X] Input [2]: [sum#X, isEmpty#X] Arguments: false -(6) VeloxAppendBatches +(6) VeloxResizeBatches Input [2]: [sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (7) ColumnarExchange Input [2]: [sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/7.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/7.txt index 3db2d7669f14..5f81d7e4298b 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/7.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/7.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (95) +- ^ InputIteratorTransformer (59) +- ShuffleQueryStage (57), Statistics(X) +- ColumnarExchange (56) - +- VeloxAppendBatches (55) + +- VeloxResizeBatches (55) +- ^ RegularHashAggregateExecTransformer (53) +- ^ InputIteratorTransformer (52) +- ShuffleQueryStage (50), Statistics(X) +- ColumnarExchange (49) - +- VeloxAppendBatches (48) + +- VeloxResizeBatches (48) +- ^ ProjectExecTransformer (46) +- ^ FlushableHashAggregateExecTransformer (45) +- ^ ProjectExecTransformer (44) @@ -284,9 +284,9 @@ Input [5]: [supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] @@ -313,9 +313,9 @@ Results [4]: [supp_nation#X, cust_nation#X, l_year#X, sum(volume#X)#X AS revenue Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] Arguments: false -(55) VeloxAppendBatches +(55) VeloxResizeBatches Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] -Arguments: X +Arguments: X, X (56) ColumnarExchange Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/8.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/8.txt index f77b5fca35c8..292fc5bfd4a8 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/8.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/8.txt @@ -6,13 +6,13 @@ AdaptiveSparkPlan (131) +- ^ InputIteratorTransformer (83) +- ShuffleQueryStage (81), Statistics(X) +- ColumnarExchange (80) - +- VeloxAppendBatches (79) + +- VeloxResizeBatches (79) +- ^ ProjectExecTransformer (77) +- ^ RegularHashAggregateExecTransformer (76) +- ^ InputIteratorTransformer (75) +- ShuffleQueryStage (73), Statistics(X) +- ColumnarExchange (72) - +- VeloxAppendBatches (71) + +- VeloxResizeBatches (71) +- ^ ProjectExecTransformer (69) +- ^ FlushableHashAggregateExecTransformer (68) +- ^ ProjectExecTransformer (67) @@ -415,9 +415,9 @@ Input [5]: [o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] Arguments: false -(71) VeloxAppendBatches +(71) VeloxResizeBatches Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (72) ColumnarExchange Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] @@ -448,9 +448,9 @@ Input [3]: [o_year#X, sum(CASE WHEN (nation#X = BRAZIL) THEN volume#X ELSE 0.000 Input [2]: [o_year#X, mkt_share#X] Arguments: false -(79) VeloxAppendBatches +(79) VeloxResizeBatches Input [2]: [o_year#X, mkt_share#X] -Arguments: X +Arguments: X, X (80) ColumnarExchange Input [2]: [o_year#X, mkt_share#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/9.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/9.txt index 1be487ed123a..b5ed852a37e4 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/9.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/9.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (100) +- ^ InputIteratorTransformer (63) +- ShuffleQueryStage (61), Statistics(X) +- ColumnarExchange (60) - +- VeloxAppendBatches (59) + +- VeloxResizeBatches (59) +- ^ RegularHashAggregateExecTransformer (57) +- ^ InputIteratorTransformer (56) +- ShuffleQueryStage (54), Statistics(X) +- ColumnarExchange (53) - +- VeloxAppendBatches (52) + +- VeloxResizeBatches (52) +- ^ ProjectExecTransformer (50) +- ^ FlushableHashAggregateExecTransformer (49) +- ^ ProjectExecTransformer (48) @@ -308,9 +308,9 @@ Input [4]: [nation#X, o_year#X, sum#X, isEmpty#X] Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] Arguments: false -(52) VeloxAppendBatches +(52) VeloxResizeBatches Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (53) ColumnarExchange Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] @@ -337,9 +337,9 @@ Results [3]: [nation#X, o_year#X, sum(amount#X)#X AS sum_profit#X] Input [3]: [nation#X, o_year#X, sum_profit#X] Arguments: false -(59) VeloxAppendBatches +(59) VeloxResizeBatches Input [3]: [nation#X, o_year#X, sum_profit#X] -Arguments: X +Arguments: X, X (60) ColumnarExchange Input [3]: [nation#X, o_year#X, sum_profit#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/1.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/1.txt index 1e53cd90e1b3..0c773785eafc 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/1.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/1.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (30) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ RegularHashAggregateExecTransformer (12) +- ^ InputIteratorTransformer (11) +- ShuffleQueryStage (9), Statistics(X) +- ColumnarExchange (8) - +- VeloxAppendBatches (7) + +- VeloxResizeBatches (7) +- ^ ProjectExecTransformer (5) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) @@ -58,9 +58,9 @@ Input [17]: [l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] Arguments: false -(7) VeloxAppendBatches +(7) VeloxResizeBatches Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] -Arguments: X +Arguments: X, X (8) ColumnarExchange Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] @@ -87,9 +87,9 @@ Results [10]: [l_returnflag#X, l_linestatus#X, sum(l_quantity#X)#X AS sum_qty#X, Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/10.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/10.txt index 098b6610a2e1..c4f94275a440 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/10.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/10.txt @@ -8,7 +8,7 @@ AdaptiveSparkPlan (68) +- ^ InputIteratorTransformer (39) +- ShuffleQueryStage (37), Statistics(X) +- ColumnarExchange (36) - +- VeloxAppendBatches (35) + +- VeloxResizeBatches (35) +- ^ ProjectExecTransformer (33) +- ^ FlushableHashAggregateExecTransformer (32) +- ^ ProjectExecTransformer (31) @@ -213,9 +213,9 @@ Input [9]: [c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] Arguments: false -(35) VeloxAppendBatches +(35) VeloxResizeBatches Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (36) ColumnarExchange Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/11.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/11.txt index 725debe00d5e..cb4bff2a0ea3 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/11.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/11.txt @@ -6,13 +6,13 @@ AdaptiveSparkPlan (60) +- ^ InputIteratorTransformer (37) +- ShuffleQueryStage (35), Statistics(X) +- ColumnarExchange (34) - +- VeloxAppendBatches (33) + +- VeloxResizeBatches (33) +- ^ FilterExecTransformer (31) +- ^ RegularHashAggregateExecTransformer (30) +- ^ InputIteratorTransformer (29) +- ShuffleQueryStage (27), Statistics(X) +- ColumnarExchange (26) - +- VeloxAppendBatches (25) + +- VeloxResizeBatches (25) +- ^ ProjectExecTransformer (23) +- ^ FlushableHashAggregateExecTransformer (22) +- ^ ProjectExecTransformer (21) @@ -162,9 +162,9 @@ Input [3]: [ps_partkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] Arguments: false -(25) VeloxAppendBatches +(25) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (26) ColumnarExchange Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] @@ -195,9 +195,9 @@ Arguments: (isnotnull(value#X) AND (cast(value#X as decimal(38,6)) > Subquery su Input [2]: [ps_partkey#X, value#X] Arguments: false -(33) VeloxAppendBatches +(33) VeloxResizeBatches Input [2]: [ps_partkey#X, value#X] -Arguments: X +Arguments: X, X (34) ColumnarExchange Input [2]: [ps_partkey#X, value#X] @@ -334,7 +334,7 @@ AdaptiveSparkPlan (102) +- ^ InputIteratorTransformer (81) +- ShuffleQueryStage (79), Statistics(X) +- ColumnarExchange (78) - +- VeloxAppendBatches (77) + +- VeloxResizeBatches (77) +- ^ FlushableHashAggregateExecTransformer (75) +- ^ ProjectExecTransformer (74) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (73) @@ -435,9 +435,9 @@ Results [2]: [sum#X, isEmpty#X] Input [2]: [sum#X, isEmpty#X] Arguments: false -(77) VeloxAppendBatches +(77) VeloxResizeBatches Input [2]: [sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (78) ColumnarExchange Input [2]: [sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/12.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/12.txt index e2b9e5153009..099fdbce656a 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/12.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/12.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (44) +- ^ InputIteratorTransformer (27) +- ShuffleQueryStage (25), Statistics(X) +- ColumnarExchange (24) - +- VeloxAppendBatches (23) + +- VeloxResizeBatches (23) +- ^ RegularHashAggregateExecTransformer (21) +- ^ InputIteratorTransformer (20) +- ShuffleQueryStage (18), Statistics(X) +- ColumnarExchange (17) - +- VeloxAppendBatches (16) + +- VeloxResizeBatches (16) +- ^ ProjectExecTransformer (14) +- ^ FlushableHashAggregateExecTransformer (13) +- ^ ProjectExecTransformer (12) @@ -109,9 +109,9 @@ Input [3]: [l_shipmode#X, sum#X, sum#X] Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] Arguments: false -(16) VeloxAppendBatches +(16) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] -Arguments: X +Arguments: X, X (17) ColumnarExchange Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] @@ -138,9 +138,9 @@ Results [3]: [l_shipmode#X, sum(CASE WHEN ((o_orderpriority#X = 1-URGENT) OR (o_ Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] Arguments: false -(23) VeloxAppendBatches +(23) VeloxResizeBatches Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] -Arguments: X +Arguments: X, X (24) ColumnarExchange Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/13.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/13.txt index f64de4dee4b2..3a2fb9b78760 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/13.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/13.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (55) +- ^ InputIteratorTransformer (36) +- ShuffleQueryStage (34), Statistics(X) +- ColumnarExchange (33) - +- VeloxAppendBatches (32) + +- VeloxResizeBatches (32) +- ^ RegularHashAggregateExecTransformer (30) +- ^ InputIteratorTransformer (29) +- ShuffleQueryStage (27), Statistics(X) +- ColumnarExchange (26) - +- VeloxAppendBatches (25) + +- VeloxResizeBatches (25) +- ^ ProjectExecTransformer (23) +- ^ FlushableHashAggregateExecTransformer (22) +- ^ ProjectExecTransformer (21) @@ -19,7 +19,7 @@ AdaptiveSparkPlan (55) +- ^ InputIteratorTransformer (19) +- ShuffleQueryStage (17), Statistics(X) +- ColumnarExchange (16) - +- VeloxAppendBatches (15) + +- VeloxResizeBatches (15) +- ^ ProjectExecTransformer (13) +- ^ FlushableHashAggregateExecTransformer (12) +- ^ ProjectExecTransformer (11) @@ -113,9 +113,9 @@ Input [2]: [c_custkey#X, count#X] Input [3]: [hash_partition_key#X, c_custkey#X, count#X] Arguments: false -(15) VeloxAppendBatches +(15) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_custkey#X, count#X] -Arguments: X +Arguments: X, X (16) ColumnarExchange Input [3]: [hash_partition_key#X, c_custkey#X, count#X] @@ -157,9 +157,9 @@ Input [2]: [c_count#X, count#X] Input [3]: [hash_partition_key#X, c_count#X, count#X] Arguments: false -(25) VeloxAppendBatches +(25) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_count#X, count#X] -Arguments: X +Arguments: X, X (26) ColumnarExchange Input [3]: [hash_partition_key#X, c_count#X, count#X] @@ -186,9 +186,9 @@ Results [2]: [c_count#X, count(1)#X AS custdist#X] Input [2]: [c_count#X, custdist#X] Arguments: false -(32) VeloxAppendBatches +(32) VeloxResizeBatches Input [2]: [c_count#X, custdist#X] -Arguments: X +Arguments: X, X (33) ColumnarExchange Input [2]: [c_count#X, custdist#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/14.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/14.txt index 901c481fac69..ed116936e3e2 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/14.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/14.txt @@ -7,7 +7,7 @@ AdaptiveSparkPlan (35) +- ^ InputIteratorTransformer (19) +- ShuffleQueryStage (17), Statistics(X) +- ColumnarExchange (16) - +- VeloxAppendBatches (15) + +- VeloxResizeBatches (15) +- ^ FlushableHashAggregateExecTransformer (13) +- ^ ProjectExecTransformer (12) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (11) @@ -98,9 +98,9 @@ Results [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] Arguments: false -(15) VeloxAppendBatches +(15) VeloxResizeBatches Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (16) ColumnarExchange Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/15.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/15.txt index a87fdff4537d..e7ac9b2efce1 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/15.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/15.txt @@ -5,7 +5,7 @@ AdaptiveSparkPlan (43) +- AQEShuffleRead (27) +- ShuffleQueryStage (26), Statistics(X) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ ProjectExecTransformer (22) +- ^ BroadcastHashJoinExecTransformer Inner BuildLeft (21) :- ^ InputIteratorTransformer (7) @@ -18,7 +18,7 @@ AdaptiveSparkPlan (43) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FlushableHashAggregateExecTransformer (11) +- ^ ProjectExecTransformer (10) @@ -100,9 +100,9 @@ Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] @@ -143,9 +143,9 @@ Input [6]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, supplier_no#X, total_ Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] @@ -249,7 +249,7 @@ AdaptiveSparkPlan (69) +- ^ InputIteratorTransformer (54) +- ShuffleQueryStage (52), Statistics(X) +- ColumnarExchange (51) - +- VeloxAppendBatches (50) + +- VeloxResizeBatches (50) +- ^ ProjectExecTransformer (48) +- ^ FlushableHashAggregateExecTransformer (47) +- ^ ProjectExecTransformer (46) @@ -296,9 +296,9 @@ Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] Arguments: false -(50) VeloxAppendBatches +(50) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (51) ColumnarExchange Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/16.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/16.txt index e2f073b8908f..2cc727f9782b 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/16.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/16.txt @@ -6,19 +6,19 @@ AdaptiveSparkPlan (59) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33), Statistics(X) +- ColumnarExchange (32) - +- VeloxAppendBatches (31) + +- VeloxResizeBatches (31) +- ^ RegularHashAggregateExecTransformer (29) +- ^ InputIteratorTransformer (28) +- ShuffleQueryStage (26), Statistics(X) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ ProjectExecTransformer (22) +- ^ FlushableHashAggregateExecTransformer (21) +- ^ RegularHashAggregateExecTransformer (20) +- ^ InputIteratorTransformer (19) +- ShuffleQueryStage (17), Statistics(X) +- ColumnarExchange (16) - +- VeloxAppendBatches (15) + +- VeloxResizeBatches (15) +- ^ ProjectExecTransformer (13) +- ^ FlushableHashAggregateExecTransformer (12) +- ^ ProjectExecTransformer (11) @@ -118,9 +118,9 @@ Input [4]: [p_brand#X, p_type#X, p_size#X, ps_suppkey#X] Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] Arguments: false -(15) VeloxAppendBatches +(15) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] -Arguments: X +Arguments: X, X (16) ColumnarExchange Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] @@ -158,9 +158,9 @@ Input [4]: [p_brand#X, p_type#X, p_size#X, count#X] Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] @@ -187,9 +187,9 @@ Results [4]: [p_brand#X, p_type#X, p_size#X, count(ps_suppkey#X)#X AS supplier_c Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/17.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/17.txt index 77733ad6f8a5..77c18798faa2 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/17.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/17.txt @@ -7,7 +7,7 @@ AdaptiveSparkPlan (36) +- ^ InputIteratorTransformer (11) +- ShuffleQueryStage (9), Statistics(X) +- ColumnarExchange (8) - +- VeloxAppendBatches (7) + +- VeloxResizeBatches (7) +- ^ FlushableHashAggregateExecTransformer (5) +- ^ InputIteratorTransformer (4) +- RowToVeloxColumnar (2) @@ -59,9 +59,9 @@ Results [2]: [sum#X, isEmpty#X] Input [2]: [sum#X, isEmpty#X] Arguments: false -(7) VeloxAppendBatches +(7) VeloxResizeBatches Input [2]: [sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (8) ColumnarExchange Input [2]: [sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/18.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/18.txt index 4f36a185da72..88e9352841ac 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/18.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/18.txt @@ -7,7 +7,7 @@ AdaptiveSparkPlan (88) +- ^ InputIteratorTransformer (51) +- ShuffleQueryStage (49), Statistics(X) +- ColumnarExchange (48) - +- VeloxAppendBatches (47) + +- VeloxResizeBatches (47) +- ^ ProjectExecTransformer (45) +- ^ FlushableHashAggregateExecTransformer (44) +- ^ ProjectExecTransformer (43) @@ -31,7 +31,7 @@ AdaptiveSparkPlan (88) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16), Statistics(X) : +- ColumnarExchange (15) - : +- VeloxAppendBatches (14) + : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ FlushableHashAggregateExecTransformer (11) : +- ^ Scan parquet (10) @@ -140,9 +140,9 @@ Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] @@ -280,9 +280,9 @@ Input [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, Input [8]: [hash_partition_key#X, c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] Arguments: false -(47) VeloxAppendBatches +(47) VeloxResizeBatches Input [8]: [hash_partition_key#X, c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (48) ColumnarExchange Input [8]: [hash_partition_key#X, c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/19.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/19.txt index 7854a767b26b..01b07807e557 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/19.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/19.txt @@ -6,7 +6,7 @@ AdaptiveSparkPlan (34) +- ^ InputIteratorTransformer (19) +- ShuffleQueryStage (17), Statistics(X) +- ColumnarExchange (16) - +- VeloxAppendBatches (15) + +- VeloxResizeBatches (15) +- ^ FlushableHashAggregateExecTransformer (13) +- ^ ProjectExecTransformer (12) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (11) @@ -97,9 +97,9 @@ Results [2]: [sum#X, isEmpty#X] Input [2]: [sum#X, isEmpty#X] Arguments: false -(15) VeloxAppendBatches +(15) VeloxResizeBatches Input [2]: [sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (16) ColumnarExchange Input [2]: [sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/20.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/20.txt index 981017da501e..f4eea85b7b42 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/20.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/20.txt @@ -5,7 +5,7 @@ AdaptiveSparkPlan (98) +- AQEShuffleRead (61) +- ShuffleQueryStage (60), Statistics(X) +- ColumnarExchange (59) - +- VeloxAppendBatches (58) + +- VeloxResizeBatches (58) +- ^ ProjectExecTransformer (56) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (55) :- ^ ProjectExecTransformer (46) @@ -35,7 +35,7 @@ AdaptiveSparkPlan (98) : +- ^ InputIteratorTransformer (34) : +- ShuffleQueryStage (32), Statistics(X) : +- ColumnarExchange (31) - : +- VeloxAppendBatches (30) + : +- VeloxResizeBatches (30) : +- ^ ProjectExecTransformer (28) : +- ^ FlushableHashAggregateExecTransformer (27) : +- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (26) @@ -217,9 +217,9 @@ Input [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] Input [5]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] Arguments: false -(30) VeloxAppendBatches +(30) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (31) ColumnarExchange Input [5]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] @@ -335,9 +335,9 @@ Input [4]: [s_name#X, s_address#X, s_nationkey#X, n_nationkey#X] Input [2]: [s_name#X, s_address#X] Arguments: false -(58) VeloxAppendBatches +(58) VeloxResizeBatches Input [2]: [s_name#X, s_address#X] -Arguments: X +Arguments: X, X (59) ColumnarExchange Input [2]: [s_name#X, s_address#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/21.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/21.txt index 5aef62e33765..ae73b6fd0423 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/21.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/21.txt @@ -6,7 +6,7 @@ AdaptiveSparkPlan (92) +- ^ InputIteratorTransformer (56) +- ShuffleQueryStage (54), Statistics(X) +- ColumnarExchange (53) - +- VeloxAppendBatches (52) + +- VeloxResizeBatches (52) +- ^ ProjectExecTransformer (50) +- ^ FlushableHashAggregateExecTransformer (49) +- ^ ProjectExecTransformer (48) @@ -306,9 +306,9 @@ Input [2]: [s_name#X, count#X] Input [3]: [hash_partition_key#X, s_name#X, count#X] Arguments: false -(52) VeloxAppendBatches +(52) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_name#X, count#X] -Arguments: X +Arguments: X, X (53) ColumnarExchange Input [3]: [hash_partition_key#X, s_name#X, count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/22.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/22.txt index fbda7224d642..5526710f89e1 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/22.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/22.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (40) +- ^ InputIteratorTransformer (25) +- ShuffleQueryStage (23), Statistics(X) +- ColumnarExchange (22) - +- VeloxAppendBatches (21) + +- VeloxResizeBatches (21) +- ^ RegularHashAggregateExecTransformer (19) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FlushableHashAggregateExecTransformer (11) +- ^ ProjectExecTransformer (10) @@ -96,9 +96,9 @@ Input [4]: [cntrycode#X, count#X, sum#X, isEmpty#X] Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] @@ -125,9 +125,9 @@ Results [3]: [cntrycode#X, count(1)#X AS numcust#X, sum(c_acctbal#X)#X AS totacc Input [3]: [cntrycode#X, numcust#X, totacctbal#X] Arguments: false -(21) VeloxAppendBatches +(21) VeloxResizeBatches Input [3]: [cntrycode#X, numcust#X, totacctbal#X] -Arguments: X +Arguments: X, X (22) ColumnarExchange Input [3]: [cntrycode#X, numcust#X, totacctbal#X] @@ -225,7 +225,7 @@ AdaptiveSparkPlan (60) +- ^ InputIteratorTransformer (50) +- ShuffleQueryStage (48), Statistics(X) +- ColumnarExchange (47) - +- VeloxAppendBatches (46) + +- VeloxResizeBatches (46) +- ^ FlushableHashAggregateExecTransformer (44) +- ^ ProjectExecTransformer (43) +- ^ FilterExecTransformer (42) @@ -265,9 +265,9 @@ Results [2]: [sum#X, count#X] Input [2]: [sum#X, count#X] Arguments: false -(46) VeloxAppendBatches +(46) VeloxResizeBatches Input [2]: [sum#X, count#X] -Arguments: X +Arguments: X, X (47) ColumnarExchange Input [2]: [sum#X, count#X] @@ -342,7 +342,7 @@ AdaptiveSparkPlan (60) +- ^ InputIteratorTransformer (50) +- ShuffleQueryStage (48), Statistics(X) +- ColumnarExchange (47) - +- VeloxAppendBatches (46) + +- VeloxResizeBatches (46) +- ^ FlushableHashAggregateExecTransformer (44) +- ^ ProjectExecTransformer (43) +- ^ FilterExecTransformer (42) diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/3.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/3.txt index 6d518ac27214..4a37e3343dfa 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/3.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/3.txt @@ -8,7 +8,7 @@ AdaptiveSparkPlan (54) +- ^ InputIteratorTransformer (30) +- ShuffleQueryStage (28), Statistics(X) +- ColumnarExchange (27) - +- VeloxAppendBatches (26) + +- VeloxResizeBatches (26) +- ^ ProjectExecTransformer (24) +- ^ FlushableHashAggregateExecTransformer (23) +- ^ ProjectExecTransformer (22) @@ -162,9 +162,9 @@ Input [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] Input [6]: [hash_partition_key#X, l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] Arguments: false -(26) VeloxAppendBatches +(26) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (27) ColumnarExchange Input [6]: [hash_partition_key#X, l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/4.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/4.txt index bb6c149c39e1..0bff0b16383f 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/4.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/4.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (46) +- ^ InputIteratorTransformer (28) +- ShuffleQueryStage (26), Statistics(X) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ RegularHashAggregateExecTransformer (22) +- ^ InputIteratorTransformer (21) +- ShuffleQueryStage (19), Statistics(X) +- ColumnarExchange (18) - +- VeloxAppendBatches (17) + +- VeloxResizeBatches (17) +- ^ ProjectExecTransformer (15) +- ^ FlushableHashAggregateExecTransformer (14) +- ^ ProjectExecTransformer (13) @@ -115,9 +115,9 @@ Input [2]: [o_orderpriority#X, count#X] Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] Arguments: false -(17) VeloxAppendBatches +(17) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] -Arguments: X +Arguments: X, X (18) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] @@ -144,9 +144,9 @@ Results [2]: [o_orderpriority#X, count(1)#X AS order_count#X] Input [2]: [o_orderpriority#X, order_count#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [2]: [o_orderpriority#X, order_count#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [2]: [o_orderpriority#X, order_count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/5.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/5.txt index afb9aa369966..ff4a7828c4b4 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/5.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/5.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (102) +- ^ InputIteratorTransformer (64) +- ShuffleQueryStage (62), Statistics(X) +- ColumnarExchange (61) - +- VeloxAppendBatches (60) + +- VeloxResizeBatches (60) +- ^ RegularHashAggregateExecTransformer (58) +- ^ InputIteratorTransformer (57) +- ShuffleQueryStage (55), Statistics(X) +- ColumnarExchange (54) - +- VeloxAppendBatches (53) + +- VeloxResizeBatches (53) +- ^ ProjectExecTransformer (51) +- ^ FlushableHashAggregateExecTransformer (50) +- ^ ProjectExecTransformer (49) @@ -319,9 +319,9 @@ Input [3]: [n_name#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] Arguments: false -(53) VeloxAppendBatches +(53) VeloxResizeBatches Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (54) ColumnarExchange Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] @@ -348,9 +348,9 @@ Results [2]: [n_name#X, sum((l_extendedprice#X * (1 - l_discount#X)))#X AS reven Input [2]: [n_name#X, revenue#X] Arguments: false -(60) VeloxAppendBatches +(60) VeloxResizeBatches Input [2]: [n_name#X, revenue#X] -Arguments: X +Arguments: X, X (61) ColumnarExchange Input [2]: [n_name#X, revenue#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/6.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/6.txt index ddc921e22d0f..9463fa1da9d9 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/6.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/6.txt @@ -6,7 +6,7 @@ AdaptiveSparkPlan (20) +- ^ InputIteratorTransformer (10) +- ShuffleQueryStage (8), Statistics(X) +- ColumnarExchange (7) - +- VeloxAppendBatches (6) + +- VeloxResizeBatches (6) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ FilterExecTransformer (2) @@ -46,9 +46,9 @@ Results [2]: [sum#X, isEmpty#X] Input [2]: [sum#X, isEmpty#X] Arguments: false -(6) VeloxAppendBatches +(6) VeloxResizeBatches Input [2]: [sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (7) ColumnarExchange Input [2]: [sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/7.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/7.txt index 5ca9bbe39ef2..0925850d9237 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/7.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/7.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (95) +- ^ InputIteratorTransformer (59) +- ShuffleQueryStage (57), Statistics(X) +- ColumnarExchange (56) - +- VeloxAppendBatches (55) + +- VeloxResizeBatches (55) +- ^ RegularHashAggregateExecTransformer (53) +- ^ InputIteratorTransformer (52) +- ShuffleQueryStage (50), Statistics(X) +- ColumnarExchange (49) - +- VeloxAppendBatches (48) + +- VeloxResizeBatches (48) +- ^ ProjectExecTransformer (46) +- ^ FlushableHashAggregateExecTransformer (45) +- ^ ProjectExecTransformer (44) @@ -289,9 +289,9 @@ Input [5]: [supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] @@ -318,9 +318,9 @@ Results [4]: [supp_nation#X, cust_nation#X, l_year#X, sum(volume#X)#X AS revenue Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] Arguments: false -(55) VeloxAppendBatches +(55) VeloxResizeBatches Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] -Arguments: X +Arguments: X, X (56) ColumnarExchange Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/8.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/8.txt index 6c2c8eb46bc9..56fd86d255b4 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/8.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/8.txt @@ -6,13 +6,13 @@ AdaptiveSparkPlan (131) +- ^ InputIteratorTransformer (83) +- ShuffleQueryStage (81), Statistics(X) +- ColumnarExchange (80) - +- VeloxAppendBatches (79) + +- VeloxResizeBatches (79) +- ^ ProjectExecTransformer (77) +- ^ RegularHashAggregateExecTransformer (76) +- ^ InputIteratorTransformer (75) +- ShuffleQueryStage (73), Statistics(X) +- ColumnarExchange (72) - +- VeloxAppendBatches (71) + +- VeloxResizeBatches (71) +- ^ ProjectExecTransformer (69) +- ^ FlushableHashAggregateExecTransformer (68) +- ^ ProjectExecTransformer (67) @@ -422,9 +422,9 @@ Input [5]: [o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] Arguments: false -(71) VeloxAppendBatches +(71) VeloxResizeBatches Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (72) ColumnarExchange Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] @@ -455,9 +455,9 @@ Input [3]: [o_year#X, sum(CASE WHEN (nation#X = BRAZIL) THEN volume#X ELSE 0.000 Input [2]: [o_year#X, mkt_share#X] Arguments: false -(79) VeloxAppendBatches +(79) VeloxResizeBatches Input [2]: [o_year#X, mkt_share#X] -Arguments: X +Arguments: X, X (80) ColumnarExchange Input [2]: [o_year#X, mkt_share#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/9.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/9.txt index 6a4faab249c8..1f25540dd136 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/9.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/9.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (100) +- ^ InputIteratorTransformer (63) +- ShuffleQueryStage (61), Statistics(X) +- ColumnarExchange (60) - +- VeloxAppendBatches (59) + +- VeloxResizeBatches (59) +- ^ RegularHashAggregateExecTransformer (57) +- ^ InputIteratorTransformer (56) +- ShuffleQueryStage (54), Statistics(X) +- ColumnarExchange (53) - +- VeloxAppendBatches (52) + +- VeloxResizeBatches (52) +- ^ ProjectExecTransformer (50) +- ^ FlushableHashAggregateExecTransformer (49) +- ^ ProjectExecTransformer (48) @@ -313,9 +313,9 @@ Input [4]: [nation#X, o_year#X, sum#X, isEmpty#X] Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] Arguments: false -(52) VeloxAppendBatches +(52) VeloxResizeBatches Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (53) ColumnarExchange Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] @@ -342,9 +342,9 @@ Results [3]: [nation#X, o_year#X, sum(amount#X)#X AS sum_profit#X] Input [3]: [nation#X, o_year#X, sum_profit#X] Arguments: false -(59) VeloxAppendBatches +(59) VeloxResizeBatches Input [3]: [nation#X, o_year#X, sum_profit#X] -Arguments: X +Arguments: X, X (60) ColumnarExchange Input [3]: [nation#X, o_year#X, sum_profit#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/1.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/1.txt index 53edb933c1fb..deb09c6c1c0f 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/1.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/1.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (30) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ RegularHashAggregateExecTransformer (12) +- ^ InputIteratorTransformer (11) +- ShuffleQueryStage (9) +- ColumnarExchange (8) - +- VeloxAppendBatches (7) + +- VeloxResizeBatches (7) +- ^ ProjectExecTransformer (5) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) @@ -58,9 +58,9 @@ Input [17]: [l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] Arguments: false -(7) VeloxAppendBatches +(7) VeloxResizeBatches Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] -Arguments: X +Arguments: X, X (8) ColumnarExchange Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] @@ -87,9 +87,9 @@ Results [10]: [l_returnflag#X, l_linestatus#X, sum(l_quantity#X)#X AS sum_qty#X, Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/10.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/10.txt index ec46bfd07b91..49e18f05aa63 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/10.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/10.txt @@ -8,7 +8,7 @@ AdaptiveSparkPlan (100) +- ^ InputIteratorTransformer (62) +- ShuffleQueryStage (60) +- ColumnarExchange (59) - +- VeloxAppendBatches (58) + +- VeloxResizeBatches (58) +- ^ ProjectExecTransformer (56) +- ^ FlushableHashAggregateExecTransformer (55) +- ^ ProjectExecTransformer (54) @@ -16,40 +16,40 @@ AdaptiveSparkPlan (100) :- ^ InputIteratorTransformer (43) : +- ShuffleQueryStage (41) : +- ColumnarExchange (40) - : +- VeloxAppendBatches (39) + : +- VeloxResizeBatches (39) : +- ^ ProjectExecTransformer (37) : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (36) : :- ^ InputIteratorTransformer (26) : : +- ShuffleQueryStage (24) : : +- ColumnarExchange (23) - : : +- VeloxAppendBatches (22) + : : +- VeloxResizeBatches (22) : : +- ^ ProjectExecTransformer (20) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) : : :- ^ InputIteratorTransformer (9) : : : +- ShuffleQueryStage (7) : : : +- ColumnarExchange (6) - : : : +- VeloxAppendBatches (5) + : : : +- VeloxResizeBatches (5) : : : +- ^ ProjectExecTransformer (3) : : : +- ^ NoopFilter (2) : : : +- ^ Scan parquet (1) : : +- ^ InputIteratorTransformer (18) : : +- ShuffleQueryStage (16) : : +- ColumnarExchange (15) - : : +- VeloxAppendBatches (14) + : : +- VeloxResizeBatches (14) : : +- ^ ProjectExecTransformer (12) : : +- ^ NoopFilter (11) : : +- ^ Scan parquet (10) : +- ^ InputIteratorTransformer (35) : +- ShuffleQueryStage (33) : +- ColumnarExchange (32) - : +- VeloxAppendBatches (31) + : +- VeloxResizeBatches (31) : +- ^ ProjectExecTransformer (29) : +- ^ NoopFilter (28) : +- ^ Scan parquet (27) +- ^ InputIteratorTransformer (52) +- ShuffleQueryStage (50) +- ColumnarExchange (49) - +- VeloxAppendBatches (48) + +- VeloxResizeBatches (48) +- ^ ProjectExecTransformer (46) +- ^ NoopFilter (45) +- ^ Scan parquet (44) @@ -107,9 +107,9 @@ Input [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct Input [8]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [8]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [8]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] @@ -144,9 +144,9 @@ Input [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] @@ -175,9 +175,9 @@ Input [9]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct Input [9]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, o_orderkey#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [9]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, o_orderkey#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [9]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, o_orderkey#X] @@ -212,9 +212,9 @@ Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_returnflag#X] Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] @@ -243,9 +243,9 @@ Input [11]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acc Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, l_extendedprice#X, l_discount#X] Arguments: false -(39) VeloxAppendBatches +(39) VeloxResizeBatches Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (40) ColumnarExchange Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, l_extendedprice#X, l_discount#X] @@ -280,9 +280,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] @@ -322,9 +322,9 @@ Input [9]: [c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] Arguments: false -(58) VeloxAppendBatches +(58) VeloxResizeBatches Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (59) ColumnarExchange Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/11.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/11.txt index cccf1408bea9..7aff321e6ae6 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/11.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/11.txt @@ -6,13 +6,13 @@ AdaptiveSparkPlan (82) +- ^ InputIteratorTransformer (53) +- ShuffleQueryStage (51) +- ColumnarExchange (50) - +- VeloxAppendBatches (49) + +- VeloxResizeBatches (49) +- ^ FilterExecTransformer (47) +- ^ RegularHashAggregateExecTransformer (46) +- ^ InputIteratorTransformer (45) +- ShuffleQueryStage (43) +- ColumnarExchange (42) - +- VeloxAppendBatches (41) + +- VeloxResizeBatches (41) +- ^ ProjectExecTransformer (39) +- ^ FlushableHashAggregateExecTransformer (38) +- ^ ProjectExecTransformer (37) @@ -20,27 +20,27 @@ AdaptiveSparkPlan (82) :- ^ InputIteratorTransformer (26) : +- ShuffleQueryStage (24) : +- ColumnarExchange (23) - : +- VeloxAppendBatches (22) + : +- VeloxResizeBatches (22) : +- ^ ProjectExecTransformer (20) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7) : : +- ColumnarExchange (6) - : : +- VeloxAppendBatches (5) + : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ NoopFilter (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16) : +- ColumnarExchange (15) - : +- VeloxAppendBatches (14) + : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ NoopFilter (11) : +- ^ Scan parquet (10) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33) +- ColumnarExchange (32) - +- VeloxAppendBatches (31) + +- VeloxResizeBatches (31) +- ^ ProjectExecTransformer (29) +- ^ NoopFilter (28) +- ^ Scan parquet (27) @@ -91,9 +91,9 @@ Input [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Input [5]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [5]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [5]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] @@ -128,9 +128,9 @@ Input [2]: [s_suppkey#X, s_nationkey#X] Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -159,9 +159,9 @@ Input [6]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X, s_suppke Input [5]: [hash_partition_key#X, ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [5]: [hash_partition_key#X, ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [5]: [hash_partition_key#X, ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] @@ -196,9 +196,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [2]: [hash_partition_key#X, n_nationkey#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [2]: [hash_partition_key#X, n_nationkey#X] @@ -238,9 +238,9 @@ Input [3]: [ps_partkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] Arguments: false -(41) VeloxAppendBatches +(41) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (42) ColumnarExchange Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] @@ -271,9 +271,9 @@ Arguments: (isnotnull(value#X) AND (cast(value#X as decimal(38,6)) > Subquery su Input [2]: [ps_partkey#X, value#X] Arguments: false -(49) VeloxAppendBatches +(49) VeloxResizeBatches Input [2]: [ps_partkey#X, value#X] -Arguments: X +Arguments: X, X (50) ColumnarExchange Input [2]: [ps_partkey#X, value#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/12.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/12.txt index 17cdf62608cc..0ac613d3792b 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/12.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/12.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (55) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33) +- ColumnarExchange (32) - +- VeloxAppendBatches (31) + +- VeloxResizeBatches (31) +- ^ RegularHashAggregateExecTransformer (29) +- ^ InputIteratorTransformer (28) +- ShuffleQueryStage (26) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ ProjectExecTransformer (22) +- ^ FlushableHashAggregateExecTransformer (21) +- ^ ProjectExecTransformer (20) @@ -19,14 +19,14 @@ AdaptiveSparkPlan (55) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ NoopFilter (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ NoopFilter (11) +- ^ Scan parquet (10) @@ -68,9 +68,9 @@ Input [2]: [o_orderkey#X, o_orderpriority#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] @@ -105,9 +105,9 @@ Input [5]: [l_orderkey#X, l_shipdate#X, l_commitdate#X, l_receiptdate#X, l_shipm Input [3]: [hash_partition_key#X, l_orderkey#X, l_shipmode#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_shipmode#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_shipmode#X] @@ -147,9 +147,9 @@ Input [3]: [l_shipmode#X, sum#X, sum#X] Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] @@ -176,9 +176,9 @@ Results [3]: [l_shipmode#X, sum(CASE WHEN ((o_orderpriority#X = 1-URGENT) OR (o_ Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/13.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/13.txt index 730f0e0a438a..de18e59ae904 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/13.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/13.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (58) +- ^ InputIteratorTransformer (37) +- ShuffleQueryStage (35) +- ColumnarExchange (34) - +- VeloxAppendBatches (33) + +- VeloxResizeBatches (33) +- ^ RegularHashAggregateExecTransformer (31) +- ^ InputIteratorTransformer (30) +- ShuffleQueryStage (28) +- ColumnarExchange (27) - +- VeloxAppendBatches (26) + +- VeloxResizeBatches (26) +- ^ ProjectExecTransformer (24) +- ^ FlushableHashAggregateExecTransformer (23) +- ^ ProjectExecTransformer (22) @@ -22,13 +22,13 @@ AdaptiveSparkPlan (58) :- ^ InputIteratorTransformer (8) : +- ShuffleQueryStage (6) : +- ColumnarExchange (5) - : +- VeloxAppendBatches (4) + : +- VeloxResizeBatches (4) : +- ^ ProjectExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (17) +- ShuffleQueryStage (15) +- ColumnarExchange (14) - +- VeloxAppendBatches (13) + +- VeloxResizeBatches (13) +- ^ ProjectExecTransformer (11) +- ^ NoopFilter (10) +- ^ Scan parquet (9) @@ -66,9 +66,9 @@ Input [1]: [c_custkey#X] Input [2]: [hash_partition_key#X, c_custkey#X] Arguments: false -(4) VeloxAppendBatches +(4) VeloxResizeBatches Input [2]: [hash_partition_key#X, c_custkey#X] -Arguments: X +Arguments: X, X (5) ColumnarExchange Input [2]: [hash_partition_key#X, c_custkey#X] @@ -103,9 +103,9 @@ Input [3]: [o_orderkey#X, o_custkey#X, o_comment#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] Arguments: false -(13) VeloxAppendBatches +(13) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] -Arguments: X +Arguments: X, X (14) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] @@ -163,9 +163,9 @@ Input [2]: [c_count#X, count#X] Input [3]: [hash_partition_key#X, c_count#X, count#X] Arguments: false -(26) VeloxAppendBatches +(26) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_count#X, count#X] -Arguments: X +Arguments: X, X (27) ColumnarExchange Input [3]: [hash_partition_key#X, c_count#X, count#X] @@ -192,9 +192,9 @@ Results [2]: [c_count#X, count(1)#X AS custdist#X] Input [2]: [c_count#X, custdist#X] Arguments: false -(33) VeloxAppendBatches +(33) VeloxResizeBatches Input [2]: [c_count#X, custdist#X] -Arguments: X +Arguments: X, X (34) ColumnarExchange Input [2]: [c_count#X, custdist#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/14.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/14.txt index 55111a31f874..66c324638566 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/14.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/14.txt @@ -10,14 +10,14 @@ AdaptiveSparkPlan (39) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ NoopFilter (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ NoopFilter (11) +- ^ Scan parquet (10) @@ -56,9 +56,9 @@ Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Input [4]: [hash_partition_key#X, l_partkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_partkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [4]: [hash_partition_key#X, l_partkey#X, l_extendedprice#X, l_discount#X] @@ -93,9 +93,9 @@ Input [2]: [p_partkey#X, p_type#X] Input [3]: [hash_partition_key#X, p_partkey#X, p_type#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, p_partkey#X, p_type#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, p_partkey#X, p_type#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/15.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/15.txt index db2df6c87544..3aac8d913ac7 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/15.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/15.txt @@ -6,13 +6,13 @@ AdaptiveSparkPlan (50) +- ^ InputIteratorTransformer (30) +- ShuffleQueryStage (28) +- ColumnarExchange (27) - +- VeloxAppendBatches (26) + +- VeloxResizeBatches (26) +- ^ ProjectExecTransformer (24) +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (23) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ NoopFilter (2) : +- ^ Scan parquet (1) @@ -21,7 +21,7 @@ AdaptiveSparkPlan (50) +- ^ InputIteratorTransformer (20) +- ShuffleQueryStage (18) +- ColumnarExchange (17) - +- VeloxAppendBatches (16) + +- VeloxResizeBatches (16) +- ^ ProjectExecTransformer (14) +- ^ FlushableHashAggregateExecTransformer (13) +- ^ ProjectExecTransformer (12) @@ -65,9 +65,9 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_phone#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_phone#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_phone#X] @@ -113,9 +113,9 @@ Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] Arguments: false -(16) VeloxAppendBatches +(16) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (17) ColumnarExchange Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] @@ -155,9 +155,9 @@ Input [6]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, supplier_no#X, total_ Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] Arguments: false -(26) VeloxAppendBatches +(26) VeloxResizeBatches Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] -Arguments: X +Arguments: X, X (27) ColumnarExchange Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/16.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/16.txt index 2eb5668906ba..abf71575d854 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/16.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/16.txt @@ -6,19 +6,19 @@ AdaptiveSparkPlan (71) +- ^ InputIteratorTransformer (44) +- ShuffleQueryStage (42) +- ColumnarExchange (41) - +- VeloxAppendBatches (40) + +- VeloxResizeBatches (40) +- ^ RegularHashAggregateExecTransformer (38) +- ^ InputIteratorTransformer (37) +- ShuffleQueryStage (35) +- ColumnarExchange (34) - +- VeloxAppendBatches (33) + +- VeloxResizeBatches (33) +- ^ ProjectExecTransformer (31) +- ^ FlushableHashAggregateExecTransformer (30) +- ^ RegularHashAggregateExecTransformer (29) +- ^ InputIteratorTransformer (28) +- ShuffleQueryStage (26) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ ProjectExecTransformer (22) +- ^ FlushableHashAggregateExecTransformer (21) +- ^ ProjectExecTransformer (20) @@ -26,14 +26,14 @@ AdaptiveSparkPlan (71) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ NoopFilter (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ NoopFilter (11) +- ^ Scan parquet (10) @@ -82,9 +82,9 @@ Input [2]: [ps_partkey#X, ps_suppkey#X] Input [3]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X] @@ -119,9 +119,9 @@ Input [4]: [p_partkey#X, p_brand#X, p_type#X, p_size#X] Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_type#X, p_size#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_type#X, p_size#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_type#X, p_size#X] @@ -161,9 +161,9 @@ Input [4]: [p_brand#X, p_type#X, p_size#X, ps_suppkey#X] Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] @@ -201,9 +201,9 @@ Input [4]: [p_brand#X, p_type#X, p_size#X, count#X] Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] Arguments: false -(33) VeloxAppendBatches +(33) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] -Arguments: X +Arguments: X, X (34) ColumnarExchange Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] @@ -230,9 +230,9 @@ Results [4]: [p_brand#X, p_type#X, p_size#X, count(ps_suppkey#X)#X AS supplier_c Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] Arguments: false -(40) VeloxAppendBatches +(40) VeloxResizeBatches Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] -Arguments: X +Arguments: X, X (41) ColumnarExchange Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/17.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/17.txt index 5226aacff753..b3b530a92eb6 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/17.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/17.txt @@ -12,14 +12,14 @@ AdaptiveSparkPlan (63) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7) : : +- ColumnarExchange (6) - : : +- VeloxAppendBatches (5) + : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ NoopFilter (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16) : +- ColumnarExchange (15) - : +- VeloxAppendBatches (14) + : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ NoopFilter (11) : +- ^ Scan parquet (10) @@ -29,7 +29,7 @@ AdaptiveSparkPlan (63) +- ^ InputIteratorTransformer (30) +- ShuffleQueryStage (28) +- ColumnarExchange (27) - +- VeloxAppendBatches (26) + +- VeloxResizeBatches (26) +- ^ ProjectExecTransformer (24) +- ^ FlushableHashAggregateExecTransformer (23) +- ^ NoopFilter (22) @@ -78,9 +78,9 @@ Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Input [4]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [4]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X] @@ -115,9 +115,9 @@ Input [3]: [p_partkey#X, p_brand#X, p_container#X] Input [2]: [hash_partition_key#X, p_partkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [2]: [hash_partition_key#X, p_partkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [2]: [hash_partition_key#X, p_partkey#X] @@ -168,9 +168,9 @@ Input [3]: [l_partkey#X, sum#X, count#X] Input [4]: [hash_partition_key#X, l_partkey#X, sum#X, count#X] Arguments: false -(26) VeloxAppendBatches +(26) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_partkey#X, sum#X, count#X] -Arguments: X +Arguments: X, X (27) ColumnarExchange Input [4]: [hash_partition_key#X, l_partkey#X, sum#X, count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/18.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/18.txt index c1287b2d685a..7845c0868dc3 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/18.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/18.txt @@ -10,26 +10,26 @@ AdaptiveSparkPlan (110) :- ^ InputIteratorTransformer (46) : +- ShuffleQueryStage (44) : +- ColumnarExchange (43) - : +- VeloxAppendBatches (42) + : +- VeloxResizeBatches (42) : +- ^ ProjectExecTransformer (40) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (39) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7) : : +- ColumnarExchange (6) - : : +- VeloxAppendBatches (5) + : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ NoopFilter (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (38) : +- ShuffleQueryStage (36) : +- ColumnarExchange (35) - : +- VeloxAppendBatches (34) + : +- VeloxResizeBatches (34) : +- ^ ProjectExecTransformer (32) : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (31) : :- ^ InputIteratorTransformer (18) : : +- ShuffleQueryStage (16) : : +- ColumnarExchange (15) - : : +- VeloxAppendBatches (14) + : : +- VeloxResizeBatches (14) : : +- ^ ProjectExecTransformer (12) : : +- ^ NoopFilter (11) : : +- ^ Scan parquet (10) @@ -39,7 +39,7 @@ AdaptiveSparkPlan (110) : +- ^ InputIteratorTransformer (27) : +- ShuffleQueryStage (25) : +- ColumnarExchange (24) - : +- VeloxAppendBatches (23) + : +- VeloxResizeBatches (23) : +- ^ ProjectExecTransformer (21) : +- ^ FlushableHashAggregateExecTransformer (20) : +- ^ Scan parquet (19) @@ -47,7 +47,7 @@ AdaptiveSparkPlan (110) :- ^ InputIteratorTransformer (55) : +- ShuffleQueryStage (53) : +- ColumnarExchange (52) - : +- VeloxAppendBatches (51) + : +- VeloxResizeBatches (51) : +- ^ ProjectExecTransformer (49) : +- ^ NoopFilter (48) : +- ^ Scan parquet (47) @@ -118,9 +118,9 @@ Input [2]: [c_custkey#X, c_name#X] Input [3]: [hash_partition_key#X, c_custkey#X, c_name#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_custkey#X, c_name#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, c_custkey#X, c_name#X] @@ -155,9 +155,9 @@ Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] @@ -194,9 +194,9 @@ Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] Arguments: false -(23) VeloxAppendBatches +(23) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (24) ColumnarExchange Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] @@ -240,9 +240,9 @@ Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: false -(34) VeloxAppendBatches +(34) VeloxResizeBatches Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] -Arguments: X +Arguments: X, X (35) ColumnarExchange Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] @@ -271,9 +271,9 @@ Input [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_ Input [6]: [hash_partition_key#X, c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] Arguments: false -(42) VeloxAppendBatches +(42) VeloxResizeBatches Input [6]: [hash_partition_key#X, c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] -Arguments: X +Arguments: X, X (43) ColumnarExchange Input [6]: [hash_partition_key#X, c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] @@ -308,9 +308,9 @@ Input [2]: [l_orderkey#X, l_quantity#X] Input [3]: [hash_partition_key#X, l_orderkey#X, l_quantity#X] Arguments: false -(51) VeloxAppendBatches +(51) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_quantity#X] -Arguments: X +Arguments: X, X (52) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_quantity#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/19.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/19.txt index 21e4f472f3b3..a6fc173e233b 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/19.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/19.txt @@ -9,14 +9,14 @@ AdaptiveSparkPlan (38) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ NoopFilter (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ NoopFilter (11) +- ^ Scan parquet (10) @@ -55,9 +55,9 @@ Input [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipin Input [5]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [5]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] @@ -92,9 +92,9 @@ Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_size#X, p_container#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_size#X, p_container#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_size#X, p_container#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/20.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/20.txt index 1ac0992834eb..46cf1c79418f 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/20.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/20.txt @@ -6,52 +6,52 @@ AdaptiveSparkPlan (146) +- ^ InputIteratorTransformer (93) +- ShuffleQueryStage (91) +- ColumnarExchange (90) - +- VeloxAppendBatches (89) + +- VeloxResizeBatches (89) +- ^ ProjectExecTransformer (87) +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (86) :- ^ InputIteratorTransformer (76) : +- ShuffleQueryStage (74) : +- ColumnarExchange (73) - : +- VeloxAppendBatches (72) + : +- VeloxResizeBatches (72) : +- ^ ProjectExecTransformer (70) : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (69) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7) : : +- ColumnarExchange (6) - : : +- VeloxAppendBatches (5) + : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ NoopFilter (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (68) : +- ShuffleQueryStage (66) : +- ColumnarExchange (65) - : +- VeloxAppendBatches (64) + : +- VeloxResizeBatches (64) : +- ^ ProjectExecTransformer (62) : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (61) : :- ^ InputIteratorTransformer (35) : : +- ShuffleQueryStage (33) : : +- ColumnarExchange (32) - : : +- VeloxAppendBatches (31) + : : +- VeloxResizeBatches (31) : : +- ^ ProjectExecTransformer (29) : : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (28) : : :- ^ InputIteratorTransformer (18) : : : +- ShuffleQueryStage (16) : : : +- ColumnarExchange (15) - : : : +- VeloxAppendBatches (14) + : : : +- VeloxResizeBatches (14) : : : +- ^ ProjectExecTransformer (12) : : : +- ^ NoopFilter (11) : : : +- ^ Scan parquet (10) : : +- ^ InputIteratorTransformer (27) : : +- ShuffleQueryStage (25) : : +- ColumnarExchange (24) - : : +- VeloxAppendBatches (23) + : : +- VeloxResizeBatches (23) : : +- ^ ProjectExecTransformer (21) : : +- ^ NoopFilter (20) : : +- ^ Scan parquet (19) : +- ^ InputIteratorTransformer (60) : +- ShuffleQueryStage (58) : +- ColumnarExchange (57) - : +- VeloxAppendBatches (56) + : +- VeloxResizeBatches (56) : +- ^ ProjectExecTransformer (54) : +- ^ FilterExecTransformer (53) : +- ^ ProjectExecTransformer (52) @@ -61,7 +61,7 @@ AdaptiveSparkPlan (146) : :- ^ InputIteratorTransformer (44) : : +- ShuffleQueryStage (42) : : +- ColumnarExchange (41) - : : +- VeloxAppendBatches (40) + : : +- VeloxResizeBatches (40) : : +- ^ ProjectExecTransformer (38) : : +- ^ NoopFilter (37) : : +- ^ Scan parquet (36) @@ -71,7 +71,7 @@ AdaptiveSparkPlan (146) +- ^ InputIteratorTransformer (85) +- ShuffleQueryStage (83) +- ColumnarExchange (82) - +- VeloxAppendBatches (81) + +- VeloxResizeBatches (81) +- ^ ProjectExecTransformer (79) +- ^ NoopFilter (78) +- ^ Scan parquet (77) @@ -146,9 +146,9 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] @@ -183,9 +183,9 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] @@ -220,9 +220,9 @@ Input [2]: [p_partkey#X, p_name#X] Input [2]: [hash_partition_key#X, p_partkey#X] Arguments: false -(23) VeloxAppendBatches +(23) VeloxResizeBatches Input [2]: [hash_partition_key#X, p_partkey#X] -Arguments: X +Arguments: X, X (24) ColumnarExchange Input [2]: [hash_partition_key#X, p_partkey#X] @@ -251,9 +251,9 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] @@ -288,9 +288,9 @@ Input [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] Input [4]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, l_quantity#X] Arguments: false -(40) VeloxAppendBatches +(40) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, l_quantity#X] -Arguments: X +Arguments: X, X (41) ColumnarExchange Input [4]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, l_quantity#X] @@ -354,9 +354,9 @@ Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: false -(56) VeloxAppendBatches +(56) VeloxResizeBatches Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -Arguments: X +Arguments: X, X (57) ColumnarExchange Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] @@ -385,9 +385,9 @@ Input [6]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, (0.5 * sum(l_quantity))#X Input [2]: [hash_partition_key#X, ps_suppkey#X] Arguments: false -(64) VeloxAppendBatches +(64) VeloxResizeBatches Input [2]: [hash_partition_key#X, ps_suppkey#X] -Arguments: X +Arguments: X, X (65) ColumnarExchange Input [2]: [hash_partition_key#X, ps_suppkey#X] @@ -416,9 +416,9 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] Arguments: false -(72) VeloxAppendBatches +(72) VeloxResizeBatches Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] -Arguments: X +Arguments: X, X (73) ColumnarExchange Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] @@ -453,9 +453,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: false -(81) VeloxAppendBatches +(81) VeloxResizeBatches Input [2]: [hash_partition_key#X, n_nationkey#X] -Arguments: X +Arguments: X, X (82) ColumnarExchange Input [2]: [hash_partition_key#X, n_nationkey#X] @@ -484,9 +484,9 @@ Input [4]: [s_name#X, s_address#X, s_nationkey#X, n_nationkey#X] Input [2]: [s_name#X, s_address#X] Arguments: false -(89) VeloxAppendBatches +(89) VeloxResizeBatches Input [2]: [s_name#X, s_address#X] -Arguments: X +Arguments: X, X (90) ColumnarExchange Input [2]: [s_name#X, s_address#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/21.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/21.txt index e2a72528c4ed..e1e46cf28ffd 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/21.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/21.txt @@ -7,7 +7,7 @@ AdaptiveSparkPlan (138) +- ^ InputIteratorTransformer (88) +- ShuffleQueryStage (86) +- ColumnarExchange (85) - +- VeloxAppendBatches (84) + +- VeloxResizeBatches (84) +- ^ ProjectExecTransformer (82) +- ^ FlushableHashAggregateExecTransformer (81) +- ^ ProjectExecTransformer (80) @@ -15,60 +15,60 @@ AdaptiveSparkPlan (138) :- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67) : +- ColumnarExchange (66) - : +- VeloxAppendBatches (65) + : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (62) : :- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50) : : +- ColumnarExchange (49) - : : +- VeloxAppendBatches (48) + : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (45) : : :- ^ InputIteratorTransformer (9) : : : +- ShuffleQueryStage (7) : : : +- ColumnarExchange (6) - : : : +- VeloxAppendBatches (5) + : : : +- VeloxResizeBatches (5) : : : +- ^ ProjectExecTransformer (3) : : : +- ^ NoopFilter (2) : : : +- ^ Scan parquet (1) : : +- ^ InputIteratorTransformer (44) : : +- ShuffleQueryStage (42) : : +- ColumnarExchange (41) - : : +- VeloxAppendBatches (40) + : : +- VeloxResizeBatches (40) : : +- ^ ProjectExecTransformer (38) : : +- ^ ShuffledHashJoinExecTransformer LeftAnti BuildRight (37) : : :- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (27) : : : :- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16) : : : : +- ColumnarExchange (15) - : : : : +- VeloxAppendBatches (14) + : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ NoopFilter (11) : : : : +- ^ Scan parquet (10) : : : +- ^ InputIteratorTransformer (26) : : : +- ShuffleQueryStage (24) : : : +- ColumnarExchange (23) - : : : +- VeloxAppendBatches (22) + : : : +- VeloxResizeBatches (22) : : : +- ^ ProjectExecTransformer (20) : : : +- ^ Scan parquet (19) : : +- ^ InputIteratorTransformer (36) : : +- ShuffleQueryStage (34) : : +- ColumnarExchange (33) - : : +- VeloxAppendBatches (32) + : : +- VeloxResizeBatches (32) : : +- ^ ProjectExecTransformer (30) : : +- ^ NoopFilter (29) : : +- ^ Scan parquet (28) : +- ^ InputIteratorTransformer (61) : +- ShuffleQueryStage (59) : +- ColumnarExchange (58) - : +- VeloxAppendBatches (57) + : +- VeloxResizeBatches (57) : +- ^ ProjectExecTransformer (55) : +- ^ NoopFilter (54) : +- ^ Scan parquet (53) +- ^ InputIteratorTransformer (78) +- ShuffleQueryStage (76) +- ColumnarExchange (75) - +- VeloxAppendBatches (74) + +- VeloxResizeBatches (74) +- ^ ProjectExecTransformer (72) +- ^ NoopFilter (71) +- ^ Scan parquet (70) @@ -139,9 +139,9 @@ Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X] Input [4]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_nationkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [4]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_nationkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [4]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_nationkey#X] @@ -176,9 +176,9 @@ Input [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] @@ -208,9 +208,9 @@ Input [2]: [l_orderkey#X, l_suppkey#X] Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] @@ -250,9 +250,9 @@ Input [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] Arguments: false -(32) VeloxAppendBatches +(32) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] -Arguments: X +Arguments: X, X (33) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] @@ -281,9 +281,9 @@ Input [2]: [l_orderkey#X, l_suppkey#X] Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] Arguments: false -(40) VeloxAppendBatches +(40) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] -Arguments: X +Arguments: X, X (41) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] @@ -312,9 +312,9 @@ Input [5]: [s_suppkey#X, s_name#X, s_nationkey#X, l_orderkey#X, l_suppkey#X] Input [4]: [hash_partition_key#X, s_name#X, s_nationkey#X, l_orderkey#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [4]: [hash_partition_key#X, s_name#X, s_nationkey#X, l_orderkey#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [4]: [hash_partition_key#X, s_name#X, s_nationkey#X, l_orderkey#X] @@ -349,9 +349,9 @@ Input [2]: [o_orderkey#X, o_orderstatus#X] Input [2]: [hash_partition_key#X, o_orderkey#X] Arguments: false -(57) VeloxAppendBatches +(57) VeloxResizeBatches Input [2]: [hash_partition_key#X, o_orderkey#X] -Arguments: X +Arguments: X, X (58) ColumnarExchange Input [2]: [hash_partition_key#X, o_orderkey#X] @@ -380,9 +380,9 @@ Input [4]: [s_name#X, s_nationkey#X, l_orderkey#X, o_orderkey#X] Input [3]: [hash_partition_key#X, s_name#X, s_nationkey#X] Arguments: false -(65) VeloxAppendBatches +(65) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_name#X, s_nationkey#X] -Arguments: X +Arguments: X, X (66) ColumnarExchange Input [3]: [hash_partition_key#X, s_name#X, s_nationkey#X] @@ -417,9 +417,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: false -(74) VeloxAppendBatches +(74) VeloxResizeBatches Input [2]: [hash_partition_key#X, n_nationkey#X] -Arguments: X +Arguments: X, X (75) ColumnarExchange Input [2]: [hash_partition_key#X, n_nationkey#X] @@ -459,9 +459,9 @@ Input [2]: [s_name#X, count#X] Input [3]: [hash_partition_key#X, s_name#X, count#X] Arguments: false -(84) VeloxAppendBatches +(84) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_name#X, count#X] -Arguments: X +Arguments: X, X (85) ColumnarExchange Input [3]: [hash_partition_key#X, s_name#X, count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/22.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/22.txt index 984abd470378..8582ecdb64d4 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/22.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/22.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (52) +- ^ InputIteratorTransformer (34) +- ShuffleQueryStage (32) +- ColumnarExchange (31) - +- VeloxAppendBatches (30) + +- VeloxResizeBatches (30) +- ^ RegularHashAggregateExecTransformer (28) +- ^ InputIteratorTransformer (27) +- ShuffleQueryStage (25) +- ColumnarExchange (24) - +- VeloxAppendBatches (23) + +- VeloxResizeBatches (23) +- ^ ProjectExecTransformer (21) +- ^ FlushableHashAggregateExecTransformer (20) +- ^ ProjectExecTransformer (19) @@ -19,14 +19,14 @@ AdaptiveSparkPlan (52) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ NoopFilter (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (17) +- ShuffleQueryStage (15) +- ColumnarExchange (14) - +- VeloxAppendBatches (13) + +- VeloxResizeBatches (13) +- ^ ProjectExecTransformer (11) +- ^ Scan parquet (10) +- == Initial Plan == @@ -65,9 +65,9 @@ Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X] Input [4]: [hash_partition_key#X, c_custkey#X, c_phone#X, c_acctbal#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [4]: [hash_partition_key#X, c_custkey#X, c_phone#X, c_acctbal#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [4]: [hash_partition_key#X, c_custkey#X, c_phone#X, c_acctbal#X] @@ -97,9 +97,9 @@ Input [1]: [o_custkey#X] Input [2]: [hash_partition_key#X, o_custkey#X] Arguments: false -(13) VeloxAppendBatches +(13) VeloxResizeBatches Input [2]: [hash_partition_key#X, o_custkey#X] -Arguments: X +Arguments: X, X (14) ColumnarExchange Input [2]: [hash_partition_key#X, o_custkey#X] @@ -139,9 +139,9 @@ Input [4]: [cntrycode#X, count#X, sum#X, isEmpty#X] Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] Arguments: false -(23) VeloxAppendBatches +(23) VeloxResizeBatches Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (24) ColumnarExchange Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] @@ -168,9 +168,9 @@ Results [3]: [cntrycode#X, count(1)#X AS numcust#X, sum(c_acctbal#X)#X AS totacc Input [3]: [cntrycode#X, numcust#X, totacctbal#X] Arguments: false -(30) VeloxAppendBatches +(30) VeloxResizeBatches Input [3]: [cntrycode#X, numcust#X, totacctbal#X] -Arguments: X +Arguments: X, X (31) ColumnarExchange Input [3]: [cntrycode#X, numcust#X, totacctbal#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/3.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/3.txt index 58484edaa685..f1f128ce00fb 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/3.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/3.txt @@ -11,27 +11,27 @@ AdaptiveSparkPlan (67) :- ^ InputIteratorTransformer (26) : +- ShuffleQueryStage (24) : +- ColumnarExchange (23) - : +- VeloxAppendBatches (22) + : +- VeloxResizeBatches (22) : +- ^ ProjectExecTransformer (20) : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7) : : +- ColumnarExchange (6) - : : +- VeloxAppendBatches (5) + : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ NoopFilter (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16) : +- ColumnarExchange (15) - : +- VeloxAppendBatches (14) + : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ NoopFilter (11) : +- ^ Scan parquet (10) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33) +- ColumnarExchange (32) - +- VeloxAppendBatches (31) + +- VeloxResizeBatches (31) +- ^ ProjectExecTransformer (29) +- ^ NoopFilter (28) +- ^ Scan parquet (27) @@ -80,9 +80,9 @@ Input [2]: [c_custkey#X, c_mktsegment#X] Input [2]: [hash_partition_key#X, c_custkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [2]: [hash_partition_key#X, c_custkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [2]: [hash_partition_key#X, c_custkey#X] @@ -117,9 +117,9 @@ Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] @@ -148,9 +148,9 @@ Input [5]: [c_custkey#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriorit Input [4]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X, o_shippriority#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [4]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X, o_shippriority#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [4]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X, o_shippriority#X] @@ -185,9 +185,9 @@ Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/4.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/4.txt index cb7a3c3a0955..6eb069e562de 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/4.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/4.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (56) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33) +- ColumnarExchange (32) - +- VeloxAppendBatches (31) + +- VeloxResizeBatches (31) +- ^ RegularHashAggregateExecTransformer (29) +- ^ InputIteratorTransformer (28) +- ShuffleQueryStage (26) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ ProjectExecTransformer (22) +- ^ FlushableHashAggregateExecTransformer (21) +- ^ ProjectExecTransformer (20) @@ -19,14 +19,14 @@ AdaptiveSparkPlan (56) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ NoopFilter (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ NoopFilter (11) +- ^ Scan parquet (10) @@ -69,9 +69,9 @@ Input [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] @@ -106,9 +106,9 @@ Input [3]: [l_orderkey#X, l_commitdate#X, l_receiptdate#X] Input [2]: [hash_partition_key#X, l_orderkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [2]: [hash_partition_key#X, l_orderkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [2]: [hash_partition_key#X, l_orderkey#X] @@ -148,9 +148,9 @@ Input [2]: [o_orderpriority#X, count#X] Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] @@ -177,9 +177,9 @@ Results [2]: [o_orderpriority#X, count(1)#X AS order_count#X] Input [2]: [o_orderpriority#X, order_count#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [2]: [o_orderpriority#X, order_count#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [2]: [o_orderpriority#X, order_count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/5.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/5.txt index 930a5a0bf488..1c71444a83d4 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/5.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/5.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (156) +- ^ InputIteratorTransformer (103) +- ShuffleQueryStage (101) +- ColumnarExchange (100) - +- VeloxAppendBatches (99) + +- VeloxResizeBatches (99) +- ^ RegularHashAggregateExecTransformer (97) +- ^ InputIteratorTransformer (96) +- ShuffleQueryStage (94) +- ColumnarExchange (93) - +- VeloxAppendBatches (92) + +- VeloxResizeBatches (92) +- ^ ProjectExecTransformer (90) +- ^ FlushableHashAggregateExecTransformer (89) +- ^ ProjectExecTransformer (88) @@ -19,66 +19,66 @@ AdaptiveSparkPlan (156) :- ^ InputIteratorTransformer (77) : +- ShuffleQueryStage (75) : +- ColumnarExchange (74) - : +- VeloxAppendBatches (73) + : +- VeloxResizeBatches (73) : +- ^ ProjectExecTransformer (71) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (70) : :- ^ InputIteratorTransformer (60) : : +- ShuffleQueryStage (58) : : +- ColumnarExchange (57) - : : +- VeloxAppendBatches (56) + : : +- VeloxResizeBatches (56) : : +- ^ ProjectExecTransformer (54) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (53) : : :- ^ InputIteratorTransformer (43) : : : +- ShuffleQueryStage (41) : : : +- ColumnarExchange (40) - : : : +- VeloxAppendBatches (39) + : : : +- VeloxResizeBatches (39) : : : +- ^ ProjectExecTransformer (37) : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (36) : : : :- ^ InputIteratorTransformer (26) : : : : +- ShuffleQueryStage (24) : : : : +- ColumnarExchange (23) - : : : : +- VeloxAppendBatches (22) + : : : : +- VeloxResizeBatches (22) : : : : +- ^ ProjectExecTransformer (20) : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) : : : : :- ^ InputIteratorTransformer (9) : : : : : +- ShuffleQueryStage (7) : : : : : +- ColumnarExchange (6) - : : : : : +- VeloxAppendBatches (5) + : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ NoopFilter (2) : : : : : +- ^ Scan parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16) : : : : +- ColumnarExchange (15) - : : : : +- VeloxAppendBatches (14) + : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ NoopFilter (11) : : : : +- ^ Scan parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33) : : : +- ColumnarExchange (32) - : : : +- VeloxAppendBatches (31) + : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ NoopFilter (28) : : : +- ^ Scan parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50) : : +- ColumnarExchange (49) - : : +- VeloxAppendBatches (48) + : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ NoopFilter (45) : : +- ^ Scan parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67) : +- ColumnarExchange (66) - : +- VeloxAppendBatches (65) + : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ NoopFilter (62) : +- ^ Scan parquet (61) +- ^ InputIteratorTransformer (86) +- ShuffleQueryStage (84) +- ColumnarExchange (83) - +- VeloxAppendBatches (82) + +- VeloxResizeBatches (82) +- ^ ProjectExecTransformer (80) +- ^ NoopFilter (79) +- ^ Scan parquet (78) @@ -153,9 +153,9 @@ Input [2]: [c_custkey#X, c_nationkey#X] Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] @@ -190,9 +190,9 @@ Input [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] @@ -221,9 +221,9 @@ Input [4]: [c_custkey#X, c_nationkey#X, o_orderkey#X, o_custkey#X] Input [3]: [hash_partition_key#X, c_nationkey#X, o_orderkey#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_nationkey#X, o_orderkey#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [3]: [hash_partition_key#X, c_nationkey#X, o_orderkey#X] @@ -258,9 +258,9 @@ Input [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] @@ -289,9 +289,9 @@ Input [6]: [c_nationkey#X, o_orderkey#X, l_orderkey#X, l_suppkey#X, l_extendedpr Input [5]: [hash_partition_key#X, c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(39) VeloxAppendBatches +(39) VeloxResizeBatches Input [5]: [hash_partition_key#X, c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (40) ColumnarExchange Input [5]: [hash_partition_key#X, c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] @@ -326,9 +326,9 @@ Input [2]: [s_suppkey#X, s_nationkey#X] Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -357,9 +357,9 @@ Input [6]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, s_suppk Input [4]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X] Arguments: false -(56) VeloxAppendBatches +(56) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X] -Arguments: X +Arguments: X, X (57) ColumnarExchange Input [4]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X] @@ -394,9 +394,9 @@ Input [3]: [n_nationkey#X, n_name#X, n_regionkey#X] Input [4]: [hash_partition_key#X, n_nationkey#X, n_name#X, n_regionkey#X] Arguments: false -(65) VeloxAppendBatches +(65) VeloxResizeBatches Input [4]: [hash_partition_key#X, n_nationkey#X, n_name#X, n_regionkey#X] -Arguments: X +Arguments: X, X (66) ColumnarExchange Input [4]: [hash_partition_key#X, n_nationkey#X, n_name#X, n_regionkey#X] @@ -425,9 +425,9 @@ Input [6]: [l_extendedprice#X, l_discount#X, s_nationkey#X, n_nationkey#X, n_nam Input [5]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] Arguments: false -(73) VeloxAppendBatches +(73) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] -Arguments: X +Arguments: X, X (74) ColumnarExchange Input [5]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] @@ -462,9 +462,9 @@ Input [2]: [r_regionkey#X, r_name#X] Input [2]: [hash_partition_key#X, r_regionkey#X] Arguments: false -(82) VeloxAppendBatches +(82) VeloxResizeBatches Input [2]: [hash_partition_key#X, r_regionkey#X] -Arguments: X +Arguments: X, X (83) ColumnarExchange Input [2]: [hash_partition_key#X, r_regionkey#X] @@ -504,9 +504,9 @@ Input [3]: [n_name#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] Arguments: false -(92) VeloxAppendBatches +(92) VeloxResizeBatches Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (93) ColumnarExchange Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] @@ -533,9 +533,9 @@ Results [2]: [n_name#X, sum(CheckOverflow((promote_precision(cast(l_extendedpric Input [2]: [n_name#X, revenue#X] Arguments: false -(99) VeloxAppendBatches +(99) VeloxResizeBatches Input [2]: [n_name#X, revenue#X] -Arguments: X +Arguments: X, X (100) ColumnarExchange Input [2]: [n_name#X, revenue#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/6.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/6.txt index b6f876d48e5a..f30bcb0da801 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/6.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/6.txt @@ -6,7 +6,7 @@ AdaptiveSparkPlan (20) +- ^ InputIteratorTransformer (10) +- ShuffleQueryStage (8) +- ColumnarExchange (7) - +- VeloxAppendBatches (6) + +- VeloxResizeBatches (6) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ NoopFilter (2) @@ -46,9 +46,9 @@ Results [2]: [sum#X, isEmpty#X] Input [2]: [sum#X, isEmpty#X] Arguments: false -(6) VeloxAppendBatches +(6) VeloxResizeBatches Input [2]: [sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (7) ColumnarExchange Input [2]: [sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/7.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/7.txt index d9eb23cb737e..0179c5ce37f1 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/7.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/7.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (149) +- ^ InputIteratorTransformer (98) +- ShuffleQueryStage (96) +- ColumnarExchange (95) - +- VeloxAppendBatches (94) + +- VeloxResizeBatches (94) +- ^ RegularHashAggregateExecTransformer (92) +- ^ InputIteratorTransformer (91) +- ShuffleQueryStage (89) +- ColumnarExchange (88) - +- VeloxAppendBatches (87) + +- VeloxResizeBatches (87) +- ^ ProjectExecTransformer (85) +- ^ FlushableHashAggregateExecTransformer (84) +- ^ ProjectExecTransformer (83) @@ -19,59 +19,59 @@ AdaptiveSparkPlan (149) :- ^ InputIteratorTransformer (77) : +- ShuffleQueryStage (75) : +- ColumnarExchange (74) - : +- VeloxAppendBatches (73) + : +- VeloxResizeBatches (73) : +- ^ ProjectExecTransformer (71) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (70) : :- ^ InputIteratorTransformer (60) : : +- ShuffleQueryStage (58) : : +- ColumnarExchange (57) - : : +- VeloxAppendBatches (56) + : : +- VeloxResizeBatches (56) : : +- ^ ProjectExecTransformer (54) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (53) : : :- ^ InputIteratorTransformer (43) : : : +- ShuffleQueryStage (41) : : : +- ColumnarExchange (40) - : : : +- VeloxAppendBatches (39) + : : : +- VeloxResizeBatches (39) : : : +- ^ ProjectExecTransformer (37) : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (36) : : : :- ^ InputIteratorTransformer (26) : : : : +- ShuffleQueryStage (24) : : : : +- ColumnarExchange (23) - : : : : +- VeloxAppendBatches (22) + : : : : +- VeloxResizeBatches (22) : : : : +- ^ ProjectExecTransformer (20) : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) : : : : :- ^ InputIteratorTransformer (9) : : : : : +- ShuffleQueryStage (7) : : : : : +- ColumnarExchange (6) - : : : : : +- VeloxAppendBatches (5) + : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ NoopFilter (2) : : : : : +- ^ Scan parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16) : : : : +- ColumnarExchange (15) - : : : : +- VeloxAppendBatches (14) + : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ NoopFilter (11) : : : : +- ^ Scan parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33) : : : +- ColumnarExchange (32) - : : : +- VeloxAppendBatches (31) + : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ NoopFilter (28) : : : +- ^ Scan parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50) : : +- ColumnarExchange (49) - : : +- VeloxAppendBatches (48) + : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ NoopFilter (45) : : +- ^ Scan parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67) : +- ColumnarExchange (66) - : +- VeloxAppendBatches (65) + : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ NoopFilter (62) : +- ^ Scan parquet (61) @@ -147,9 +147,9 @@ Input [2]: [s_suppkey#X, s_nationkey#X] Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -184,9 +184,9 @@ Input [5]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipda Input [6]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [6]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] @@ -215,9 +215,9 @@ Input [7]: [s_suppkey#X, s_nationkey#X, l_orderkey#X, l_suppkey#X, l_extendedpri Input [6]: [hash_partition_key#X, s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [6]: [hash_partition_key#X, s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [6]: [hash_partition_key#X, s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] @@ -252,9 +252,9 @@ Input [2]: [o_orderkey#X, o_custkey#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] @@ -283,9 +283,9 @@ Input [7]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_ship Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X] Arguments: false -(39) VeloxAppendBatches +(39) VeloxResizeBatches Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X] -Arguments: X +Arguments: X, X (40) ColumnarExchange Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X] @@ -320,9 +320,9 @@ Input [2]: [c_custkey#X, c_nationkey#X] Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] @@ -351,9 +351,9 @@ Input [7]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_cust Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X] Arguments: false -(56) VeloxAppendBatches +(56) VeloxResizeBatches Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X] -Arguments: X +Arguments: X, X (57) ColumnarExchange Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X] @@ -388,9 +388,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] Arguments: false -(65) VeloxAppendBatches +(65) VeloxResizeBatches Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] -Arguments: X +Arguments: X, X (66) ColumnarExchange Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] @@ -419,9 +419,9 @@ Input [7]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nati Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X, n_name#X] Arguments: false -(73) VeloxAppendBatches +(73) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X, n_name#X] -Arguments: X +Arguments: X, X (74) ColumnarExchange Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X, n_name#X] @@ -474,9 +474,9 @@ Input [5]: [supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] Arguments: false -(87) VeloxAppendBatches +(87) VeloxResizeBatches Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (88) ColumnarExchange Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] @@ -503,9 +503,9 @@ Results [4]: [supp_nation#X, cust_nation#X, l_year#X, sum(volume#X)#X AS revenue Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] Arguments: false -(94) VeloxAppendBatches +(94) VeloxResizeBatches Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] -Arguments: X +Arguments: X, X (95) ColumnarExchange Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/8.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/8.txt index 5c9e51b95c60..ce21c6ca2cae 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/8.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/8.txt @@ -6,13 +6,13 @@ AdaptiveSparkPlan (207) +- ^ InputIteratorTransformer (138) +- ShuffleQueryStage (136) +- ColumnarExchange (135) - +- VeloxAppendBatches (134) + +- VeloxResizeBatches (134) +- ^ ProjectExecTransformer (132) +- ^ RegularHashAggregateExecTransformer (131) +- ^ InputIteratorTransformer (130) +- ShuffleQueryStage (128) +- ColumnarExchange (127) - +- VeloxAppendBatches (126) + +- VeloxResizeBatches (126) +- ^ ProjectExecTransformer (124) +- ^ FlushableHashAggregateExecTransformer (123) +- ^ ProjectExecTransformer (122) @@ -20,92 +20,92 @@ AdaptiveSparkPlan (207) :- ^ InputIteratorTransformer (111) : +- ShuffleQueryStage (109) : +- ColumnarExchange (108) - : +- VeloxAppendBatches (107) + : +- VeloxResizeBatches (107) : +- ^ ProjectExecTransformer (105) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (104) : :- ^ InputIteratorTransformer (94) : : +- ShuffleQueryStage (92) : : +- ColumnarExchange (91) - : : +- VeloxAppendBatches (90) + : : +- VeloxResizeBatches (90) : : +- ^ ProjectExecTransformer (88) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (87) : : :- ^ InputIteratorTransformer (77) : : : +- ShuffleQueryStage (75) : : : +- ColumnarExchange (74) - : : : +- VeloxAppendBatches (73) + : : : +- VeloxResizeBatches (73) : : : +- ^ ProjectExecTransformer (71) : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (70) : : : :- ^ InputIteratorTransformer (60) : : : : +- ShuffleQueryStage (58) : : : : +- ColumnarExchange (57) - : : : : +- VeloxAppendBatches (56) + : : : : +- VeloxResizeBatches (56) : : : : +- ^ ProjectExecTransformer (54) : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (53) : : : : :- ^ InputIteratorTransformer (43) : : : : : +- ShuffleQueryStage (41) : : : : : +- ColumnarExchange (40) - : : : : : +- VeloxAppendBatches (39) + : : : : : +- VeloxResizeBatches (39) : : : : : +- ^ ProjectExecTransformer (37) : : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (36) : : : : : :- ^ InputIteratorTransformer (26) : : : : : : +- ShuffleQueryStage (24) : : : : : : +- ColumnarExchange (23) - : : : : : : +- VeloxAppendBatches (22) + : : : : : : +- VeloxResizeBatches (22) : : : : : : +- ^ ProjectExecTransformer (20) : : : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) : : : : : : :- ^ InputIteratorTransformer (9) : : : : : : : +- ShuffleQueryStage (7) : : : : : : : +- ColumnarExchange (6) - : : : : : : : +- VeloxAppendBatches (5) + : : : : : : : +- VeloxResizeBatches (5) : : : : : : : +- ^ ProjectExecTransformer (3) : : : : : : : +- ^ NoopFilter (2) : : : : : : : +- ^ Scan parquet (1) : : : : : : +- ^ InputIteratorTransformer (18) : : : : : : +- ShuffleQueryStage (16) : : : : : : +- ColumnarExchange (15) - : : : : : : +- VeloxAppendBatches (14) + : : : : : : +- VeloxResizeBatches (14) : : : : : : +- ^ ProjectExecTransformer (12) : : : : : : +- ^ NoopFilter (11) : : : : : : +- ^ Scan parquet (10) : : : : : +- ^ InputIteratorTransformer (35) : : : : : +- ShuffleQueryStage (33) : : : : : +- ColumnarExchange (32) - : : : : : +- VeloxAppendBatches (31) + : : : : : +- VeloxResizeBatches (31) : : : : : +- ^ ProjectExecTransformer (29) : : : : : +- ^ NoopFilter (28) : : : : : +- ^ Scan parquet (27) : : : : +- ^ InputIteratorTransformer (52) : : : : +- ShuffleQueryStage (50) : : : : +- ColumnarExchange (49) - : : : : +- VeloxAppendBatches (48) + : : : : +- VeloxResizeBatches (48) : : : : +- ^ ProjectExecTransformer (46) : : : : +- ^ NoopFilter (45) : : : : +- ^ Scan parquet (44) : : : +- ^ InputIteratorTransformer (69) : : : +- ShuffleQueryStage (67) : : : +- ColumnarExchange (66) - : : : +- VeloxAppendBatches (65) + : : : +- VeloxResizeBatches (65) : : : +- ^ ProjectExecTransformer (63) : : : +- ^ NoopFilter (62) : : : +- ^ Scan parquet (61) : : +- ^ InputIteratorTransformer (86) : : +- ShuffleQueryStage (84) : : +- ColumnarExchange (83) - : : +- VeloxAppendBatches (82) + : : +- VeloxResizeBatches (82) : : +- ^ ProjectExecTransformer (80) : : +- ^ NoopFilter (79) : : +- ^ Scan parquet (78) : +- ^ InputIteratorTransformer (103) : +- ShuffleQueryStage (101) : +- ColumnarExchange (100) - : +- VeloxAppendBatches (99) + : +- VeloxResizeBatches (99) : +- ^ ProjectExecTransformer (97) : +- ^ NoopFilter (96) : +- ^ Scan parquet (95) +- ^ InputIteratorTransformer (120) +- ShuffleQueryStage (118) +- ColumnarExchange (117) - +- VeloxAppendBatches (116) + +- VeloxResizeBatches (116) +- ^ ProjectExecTransformer (114) +- ^ NoopFilter (113) +- ^ Scan parquet (112) @@ -196,9 +196,9 @@ Input [2]: [p_partkey#X, p_type#X] Input [2]: [hash_partition_key#X, p_partkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [2]: [hash_partition_key#X, p_partkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [2]: [hash_partition_key#X, p_partkey#X] @@ -233,9 +233,9 @@ Input [5]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discoun Input [6]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [6]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] @@ -264,9 +264,9 @@ Input [6]: [p_partkey#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] @@ -301,9 +301,9 @@ Input [2]: [s_suppkey#X, s_nationkey#X] Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -332,9 +332,9 @@ Input [6]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, s_suppke Input [5]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] Arguments: false -(39) VeloxAppendBatches +(39) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] -Arguments: X +Arguments: X, X (40) ColumnarExchange Input [5]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] @@ -369,9 +369,9 @@ Input [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Input [4]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [4]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [4]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X] @@ -400,9 +400,9 @@ Input [7]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orde Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X] Arguments: false -(56) VeloxAppendBatches +(56) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X] -Arguments: X +Arguments: X, X (57) ColumnarExchange Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X] @@ -437,9 +437,9 @@ Input [2]: [c_custkey#X, c_nationkey#X] Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] Arguments: false -(65) VeloxAppendBatches +(65) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] -Arguments: X +Arguments: X, X (66) ColumnarExchange Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] @@ -468,9 +468,9 @@ Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_order Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X] Arguments: false -(73) VeloxAppendBatches +(73) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X] -Arguments: X +Arguments: X, X (74) ColumnarExchange Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X] @@ -505,9 +505,9 @@ Input [2]: [n_nationkey#X, n_regionkey#X] Input [3]: [hash_partition_key#X, n_nationkey#X, n_regionkey#X] Arguments: false -(82) VeloxAppendBatches +(82) VeloxResizeBatches Input [3]: [hash_partition_key#X, n_nationkey#X, n_regionkey#X] -Arguments: X +Arguments: X, X (83) ColumnarExchange Input [3]: [hash_partition_key#X, n_nationkey#X, n_regionkey#X] @@ -536,9 +536,9 @@ Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nat Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X] Arguments: false -(90) VeloxAppendBatches +(90) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X] -Arguments: X +Arguments: X, X (91) ColumnarExchange Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X] @@ -573,9 +573,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] Arguments: false -(99) VeloxAppendBatches +(99) VeloxResizeBatches Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] -Arguments: X +Arguments: X, X (100) ColumnarExchange Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] @@ -604,9 +604,9 @@ Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_reg Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X] Arguments: false -(107) VeloxAppendBatches +(107) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X] -Arguments: X +Arguments: X, X (108) ColumnarExchange Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X] @@ -641,9 +641,9 @@ Input [2]: [r_regionkey#X, r_name#X] Input [2]: [hash_partition_key#X, r_regionkey#X] Arguments: false -(116) VeloxAppendBatches +(116) VeloxResizeBatches Input [2]: [hash_partition_key#X, r_regionkey#X] -Arguments: X +Arguments: X, X (117) ColumnarExchange Input [2]: [hash_partition_key#X, r_regionkey#X] @@ -683,9 +683,9 @@ Input [5]: [o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] Arguments: false -(126) VeloxAppendBatches +(126) VeloxResizeBatches Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (127) ColumnarExchange Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] @@ -716,9 +716,9 @@ Input [3]: [o_year#X, sum(CASE WHEN (nation#X = BRAZIL) THEN volume#X ELSE 0.000 Input [2]: [o_year#X, mkt_share#X] Arguments: false -(134) VeloxAppendBatches +(134) VeloxResizeBatches Input [2]: [o_year#X, mkt_share#X] -Arguments: X +Arguments: X, X (135) ColumnarExchange Input [2]: [o_year#X, mkt_share#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/9.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/9.txt index 2abb6ec215c6..06d8d6e0ef00 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/9.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/9.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (155) +- ^ InputIteratorTransformer (103) +- ShuffleQueryStage (101) +- ColumnarExchange (100) - +- VeloxAppendBatches (99) + +- VeloxResizeBatches (99) +- ^ RegularHashAggregateExecTransformer (97) +- ^ InputIteratorTransformer (96) +- ShuffleQueryStage (94) +- ColumnarExchange (93) - +- VeloxAppendBatches (92) + +- VeloxResizeBatches (92) +- ^ ProjectExecTransformer (90) +- ^ FlushableHashAggregateExecTransformer (89) +- ^ ProjectExecTransformer (88) @@ -19,66 +19,66 @@ AdaptiveSparkPlan (155) :- ^ InputIteratorTransformer (77) : +- ShuffleQueryStage (75) : +- ColumnarExchange (74) - : +- VeloxAppendBatches (73) + : +- VeloxResizeBatches (73) : +- ^ ProjectExecTransformer (71) : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (70) : :- ^ InputIteratorTransformer (60) : : +- ShuffleQueryStage (58) : : +- ColumnarExchange (57) - : : +- VeloxAppendBatches (56) + : : +- VeloxResizeBatches (56) : : +- ^ ProjectExecTransformer (54) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (53) : : :- ^ InputIteratorTransformer (43) : : : +- ShuffleQueryStage (41) : : : +- ColumnarExchange (40) - : : : +- VeloxAppendBatches (39) + : : : +- VeloxResizeBatches (39) : : : +- ^ ProjectExecTransformer (37) : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (36) : : : :- ^ InputIteratorTransformer (26) : : : : +- ShuffleQueryStage (24) : : : : +- ColumnarExchange (23) - : : : : +- VeloxAppendBatches (22) + : : : : +- VeloxResizeBatches (22) : : : : +- ^ ProjectExecTransformer (20) : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) : : : : :- ^ InputIteratorTransformer (9) : : : : : +- ShuffleQueryStage (7) : : : : : +- ColumnarExchange (6) - : : : : : +- VeloxAppendBatches (5) + : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ NoopFilter (2) : : : : : +- ^ Scan parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16) : : : : +- ColumnarExchange (15) - : : : : +- VeloxAppendBatches (14) + : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ NoopFilter (11) : : : : +- ^ Scan parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33) : : : +- ColumnarExchange (32) - : : : +- VeloxAppendBatches (31) + : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ NoopFilter (28) : : : +- ^ Scan parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50) : : +- ColumnarExchange (49) - : : +- VeloxAppendBatches (48) + : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ NoopFilter (45) : : +- ^ Scan parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67) : +- ColumnarExchange (66) - : +- VeloxAppendBatches (65) + : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ NoopFilter (62) : +- ^ Scan parquet (61) +- ^ InputIteratorTransformer (86) +- ShuffleQueryStage (84) +- ColumnarExchange (83) - +- VeloxAppendBatches (82) + +- VeloxResizeBatches (82) +- ^ ProjectExecTransformer (80) +- ^ NoopFilter (79) +- ^ Scan parquet (78) @@ -152,9 +152,9 @@ Input [2]: [p_partkey#X, p_name#X] Input [2]: [hash_partition_key#X, p_partkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [2]: [hash_partition_key#X, p_partkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [2]: [hash_partition_key#X, p_partkey#X] @@ -189,9 +189,9 @@ Input [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpric Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] @@ -220,9 +220,9 @@ Input [7]: [p_partkey#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] @@ -257,9 +257,9 @@ Input [2]: [s_suppkey#X, s_nationkey#X] Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -288,9 +288,9 @@ Input [8]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpric Input [8]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X] Arguments: false -(39) VeloxAppendBatches +(39) VeloxResizeBatches Input [8]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X] -Arguments: X +Arguments: X, X (40) ColumnarExchange Input [8]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X] @@ -325,9 +325,9 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_supplycost#X] Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_supplycost#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_supplycost#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_supplycost#X] @@ -356,9 +356,9 @@ Input [10]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpri Input [7]: [hash_partition_key#X, l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X] Arguments: false -(56) VeloxAppendBatches +(56) VeloxResizeBatches Input [7]: [hash_partition_key#X, l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X] -Arguments: X +Arguments: X, X (57) ColumnarExchange Input [7]: [hash_partition_key#X, l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X] @@ -393,9 +393,9 @@ Input [2]: [o_orderkey#X, o_orderdate#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X] Arguments: false -(65) VeloxAppendBatches +(65) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X] -Arguments: X +Arguments: X, X (66) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X] @@ -424,9 +424,9 @@ Input [8]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_natio Input [7]: [hash_partition_key#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X] Arguments: false -(73) VeloxAppendBatches +(73) VeloxResizeBatches Input [7]: [hash_partition_key#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X] -Arguments: X +Arguments: X, X (74) ColumnarExchange Input [7]: [hash_partition_key#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X] @@ -461,9 +461,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] Arguments: false -(82) VeloxAppendBatches +(82) VeloxResizeBatches Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] -Arguments: X +Arguments: X, X (83) ColumnarExchange Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] @@ -503,9 +503,9 @@ Input [4]: [nation#X, o_year#X, sum#X, isEmpty#X] Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] Arguments: false -(92) VeloxAppendBatches +(92) VeloxResizeBatches Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (93) ColumnarExchange Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] @@ -532,9 +532,9 @@ Results [3]: [nation#X, o_year#X, sum(amount#X)#X AS sum_profit#X] Input [3]: [nation#X, o_year#X, sum_profit#X] Arguments: false -(99) VeloxAppendBatches +(99) VeloxResizeBatches Input [3]: [nation#X, o_year#X, sum_profit#X] -Arguments: X +Arguments: X, X (100) ColumnarExchange Input [3]: [nation#X, o_year#X, sum_profit#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/1.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/1.txt index afea15af53d2..159a1598c9bf 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/1.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/1.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (30) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ RegularHashAggregateExecTransformer (12) +- ^ InputIteratorTransformer (11) +- ShuffleQueryStage (9), Statistics(X) +- ColumnarExchange (8) - +- VeloxAppendBatches (7) + +- VeloxResizeBatches (7) +- ^ ProjectExecTransformer (5) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) @@ -58,9 +58,9 @@ Input [17]: [l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] Arguments: false -(7) VeloxAppendBatches +(7) VeloxResizeBatches Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] -Arguments: X +Arguments: X, X (8) ColumnarExchange Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] @@ -87,9 +87,9 @@ Results [10]: [l_returnflag#X, l_linestatus#X, sum(l_quantity#X)#X AS sum_qty#X, Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/10.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/10.txt index 3be5f1996fa8..35cf0c574fde 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/10.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/10.txt @@ -8,7 +8,7 @@ AdaptiveSparkPlan (100) +- ^ InputIteratorTransformer (62) +- ShuffleQueryStage (60), Statistics(X) +- ColumnarExchange (59) - +- VeloxAppendBatches (58) + +- VeloxResizeBatches (58) +- ^ ProjectExecTransformer (56) +- ^ FlushableHashAggregateExecTransformer (55) +- ^ ProjectExecTransformer (54) @@ -16,40 +16,40 @@ AdaptiveSparkPlan (100) :- ^ InputIteratorTransformer (43) : +- ShuffleQueryStage (41), Statistics(X) : +- ColumnarExchange (40) - : +- VeloxAppendBatches (39) + : +- VeloxResizeBatches (39) : +- ^ ProjectExecTransformer (37) : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (36) : :- ^ InputIteratorTransformer (26) : : +- ShuffleQueryStage (24), Statistics(X) : : +- ColumnarExchange (23) - : : +- VeloxAppendBatches (22) + : : +- VeloxResizeBatches (22) : : +- ^ ProjectExecTransformer (20) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) : : :- ^ InputIteratorTransformer (9) : : : +- ShuffleQueryStage (7), Statistics(X) : : : +- ColumnarExchange (6) - : : : +- VeloxAppendBatches (5) + : : : +- VeloxResizeBatches (5) : : : +- ^ ProjectExecTransformer (3) : : : +- ^ NoopFilter (2) : : : +- ^ Scan parquet (1) : : +- ^ InputIteratorTransformer (18) : : +- ShuffleQueryStage (16), Statistics(X) : : +- ColumnarExchange (15) - : : +- VeloxAppendBatches (14) + : : +- VeloxResizeBatches (14) : : +- ^ ProjectExecTransformer (12) : : +- ^ NoopFilter (11) : : +- ^ Scan parquet (10) : +- ^ InputIteratorTransformer (35) : +- ShuffleQueryStage (33), Statistics(X) : +- ColumnarExchange (32) - : +- VeloxAppendBatches (31) + : +- VeloxResizeBatches (31) : +- ^ ProjectExecTransformer (29) : +- ^ NoopFilter (28) : +- ^ Scan parquet (27) +- ^ InputIteratorTransformer (52) +- ShuffleQueryStage (50), Statistics(X) +- ColumnarExchange (49) - +- VeloxAppendBatches (48) + +- VeloxResizeBatches (48) +- ^ ProjectExecTransformer (46) +- ^ NoopFilter (45) +- ^ Scan parquet (44) @@ -107,9 +107,9 @@ Input [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct Input [8]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [8]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [8]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] @@ -144,9 +144,9 @@ Input [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] @@ -175,9 +175,9 @@ Input [9]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct Input [9]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, o_orderkey#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [9]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, o_orderkey#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [9]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, o_orderkey#X] @@ -212,9 +212,9 @@ Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_returnflag#X] Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] @@ -243,9 +243,9 @@ Input [11]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acc Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, l_extendedprice#X, l_discount#X] Arguments: false -(39) VeloxAppendBatches +(39) VeloxResizeBatches Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (40) ColumnarExchange Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, l_extendedprice#X, l_discount#X] @@ -280,9 +280,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] @@ -322,9 +322,9 @@ Input [9]: [c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] Arguments: false -(58) VeloxAppendBatches +(58) VeloxResizeBatches Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (59) ColumnarExchange Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/11.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/11.txt index 2347eb9b677e..e24c94d48247 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/11.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/11.txt @@ -6,13 +6,13 @@ AdaptiveSparkPlan (82) +- ^ InputIteratorTransformer (53) +- ShuffleQueryStage (51), Statistics(X) +- ColumnarExchange (50) - +- VeloxAppendBatches (49) + +- VeloxResizeBatches (49) +- ^ FilterExecTransformer (47) +- ^ RegularHashAggregateExecTransformer (46) +- ^ InputIteratorTransformer (45) +- ShuffleQueryStage (43), Statistics(X) +- ColumnarExchange (42) - +- VeloxAppendBatches (41) + +- VeloxResizeBatches (41) +- ^ ProjectExecTransformer (39) +- ^ FlushableHashAggregateExecTransformer (38) +- ^ ProjectExecTransformer (37) @@ -20,27 +20,27 @@ AdaptiveSparkPlan (82) :- ^ InputIteratorTransformer (26) : +- ShuffleQueryStage (24), Statistics(X) : +- ColumnarExchange (23) - : +- VeloxAppendBatches (22) + : +- VeloxResizeBatches (22) : +- ^ ProjectExecTransformer (20) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7), Statistics(X) : : +- ColumnarExchange (6) - : : +- VeloxAppendBatches (5) + : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ NoopFilter (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16), Statistics(X) : +- ColumnarExchange (15) - : +- VeloxAppendBatches (14) + : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ NoopFilter (11) : +- ^ Scan parquet (10) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33), Statistics(X) +- ColumnarExchange (32) - +- VeloxAppendBatches (31) + +- VeloxResizeBatches (31) +- ^ ProjectExecTransformer (29) +- ^ NoopFilter (28) +- ^ Scan parquet (27) @@ -91,9 +91,9 @@ Input [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Input [5]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [5]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [5]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] @@ -128,9 +128,9 @@ Input [2]: [s_suppkey#X, s_nationkey#X] Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -159,9 +159,9 @@ Input [6]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X, s_suppke Input [5]: [hash_partition_key#X, ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [5]: [hash_partition_key#X, ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [5]: [hash_partition_key#X, ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] @@ -196,9 +196,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [2]: [hash_partition_key#X, n_nationkey#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [2]: [hash_partition_key#X, n_nationkey#X] @@ -238,9 +238,9 @@ Input [3]: [ps_partkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] Arguments: false -(41) VeloxAppendBatches +(41) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (42) ColumnarExchange Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] @@ -271,9 +271,9 @@ Arguments: (isnotnull(value#X) AND (cast(value#X as decimal(38,6)) > Subquery su Input [2]: [ps_partkey#X, value#X] Arguments: false -(49) VeloxAppendBatches +(49) VeloxResizeBatches Input [2]: [ps_partkey#X, value#X] -Arguments: X +Arguments: X, X (50) ColumnarExchange Input [2]: [ps_partkey#X, value#X] @@ -435,13 +435,13 @@ AdaptiveSparkPlan (136) :- ^ InputIteratorTransformer (103) : +- ShuffleQueryStage (101), Statistics(X) : +- ColumnarExchange (100) - : +- VeloxAppendBatches (99) + : +- VeloxResizeBatches (99) : +- ^ ProjectExecTransformer (97) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (96) : :- ^ InputIteratorTransformer (91) : : +- ShuffleQueryStage (89), Statistics(X) : : +- ColumnarExchange (88) - : : +- VeloxAppendBatches (87) + : : +- VeloxResizeBatches (87) : : +- ^ ProjectExecTransformer (85) : : +- ^ NoopFilter (84) : : +- ^ Scan parquet (83) @@ -494,9 +494,9 @@ Input [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Input [4]: [hash_partition_key#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Arguments: false -(87) VeloxAppendBatches +(87) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] -Arguments: X +Arguments: X, X (88) ColumnarExchange Input [4]: [hash_partition_key#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] @@ -538,9 +538,9 @@ Input [5]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X, s_suppkey#X, s_nationk Input [4]: [hash_partition_key#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] Arguments: false -(99) VeloxAppendBatches +(99) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] -Arguments: X +Arguments: X, X (100) ColumnarExchange Input [4]: [hash_partition_key#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/12.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/12.txt index b0f084e2d048..870f9ccebd86 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/12.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/12.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (55) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33), Statistics(X) +- ColumnarExchange (32) - +- VeloxAppendBatches (31) + +- VeloxResizeBatches (31) +- ^ RegularHashAggregateExecTransformer (29) +- ^ InputIteratorTransformer (28) +- ShuffleQueryStage (26), Statistics(X) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ ProjectExecTransformer (22) +- ^ FlushableHashAggregateExecTransformer (21) +- ^ ProjectExecTransformer (20) @@ -19,14 +19,14 @@ AdaptiveSparkPlan (55) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7), Statistics(X) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ NoopFilter (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ NoopFilter (11) +- ^ Scan parquet (10) @@ -68,9 +68,9 @@ Input [2]: [o_orderkey#X, o_orderpriority#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] @@ -105,9 +105,9 @@ Input [5]: [l_orderkey#X, l_shipdate#X, l_commitdate#X, l_receiptdate#X, l_shipm Input [3]: [hash_partition_key#X, l_orderkey#X, l_shipmode#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_shipmode#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_shipmode#X] @@ -147,9 +147,9 @@ Input [3]: [l_shipmode#X, sum#X, sum#X] Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] @@ -176,9 +176,9 @@ Results [3]: [l_shipmode#X, sum(CASE WHEN ((o_orderpriority#X = 1-URGENT) OR (o_ Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/13.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/13.txt index 07c32ff95fb1..ff5bfe254096 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/13.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/13.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (58) +- ^ InputIteratorTransformer (37) +- ShuffleQueryStage (35), Statistics(X) +- ColumnarExchange (34) - +- VeloxAppendBatches (33) + +- VeloxResizeBatches (33) +- ^ RegularHashAggregateExecTransformer (31) +- ^ InputIteratorTransformer (30) +- ShuffleQueryStage (28), Statistics(X) +- ColumnarExchange (27) - +- VeloxAppendBatches (26) + +- VeloxResizeBatches (26) +- ^ ProjectExecTransformer (24) +- ^ FlushableHashAggregateExecTransformer (23) +- ^ ProjectExecTransformer (22) @@ -22,13 +22,13 @@ AdaptiveSparkPlan (58) :- ^ InputIteratorTransformer (8) : +- ShuffleQueryStage (6), Statistics(X) : +- ColumnarExchange (5) - : +- VeloxAppendBatches (4) + : +- VeloxResizeBatches (4) : +- ^ ProjectExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (17) +- ShuffleQueryStage (15), Statistics(X) +- ColumnarExchange (14) - +- VeloxAppendBatches (13) + +- VeloxResizeBatches (13) +- ^ ProjectExecTransformer (11) +- ^ NoopFilter (10) +- ^ Scan parquet (9) @@ -66,9 +66,9 @@ Input [1]: [c_custkey#X] Input [2]: [hash_partition_key#X, c_custkey#X] Arguments: false -(4) VeloxAppendBatches +(4) VeloxResizeBatches Input [2]: [hash_partition_key#X, c_custkey#X] -Arguments: X +Arguments: X, X (5) ColumnarExchange Input [2]: [hash_partition_key#X, c_custkey#X] @@ -103,9 +103,9 @@ Input [3]: [o_orderkey#X, o_custkey#X, o_comment#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] Arguments: false -(13) VeloxAppendBatches +(13) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] -Arguments: X +Arguments: X, X (14) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] @@ -163,9 +163,9 @@ Input [2]: [c_count#X, count#X] Input [3]: [hash_partition_key#X, c_count#X, count#X] Arguments: false -(26) VeloxAppendBatches +(26) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_count#X, count#X] -Arguments: X +Arguments: X, X (27) ColumnarExchange Input [3]: [hash_partition_key#X, c_count#X, count#X] @@ -192,9 +192,9 @@ Results [2]: [c_count#X, count(1)#X AS custdist#X] Input [2]: [c_count#X, custdist#X] Arguments: false -(33) VeloxAppendBatches +(33) VeloxResizeBatches Input [2]: [c_count#X, custdist#X] -Arguments: X +Arguments: X, X (34) ColumnarExchange Input [2]: [c_count#X, custdist#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/14.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/14.txt index 11bbb2a71e79..247b15e33ba3 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/14.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/14.txt @@ -10,14 +10,14 @@ AdaptiveSparkPlan (39) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7), Statistics(X) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ NoopFilter (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ NoopFilter (11) +- ^ Scan parquet (10) @@ -56,9 +56,9 @@ Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Input [4]: [hash_partition_key#X, l_partkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_partkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [4]: [hash_partition_key#X, l_partkey#X, l_extendedprice#X, l_discount#X] @@ -93,9 +93,9 @@ Input [2]: [p_partkey#X, p_type#X] Input [3]: [hash_partition_key#X, p_partkey#X, p_type#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, p_partkey#X, p_type#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, p_partkey#X, p_type#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/15.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/15.txt index be97f58cf438..ecb61718853c 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/15.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/15.txt @@ -5,13 +5,13 @@ AdaptiveSparkPlan (47) +- AQEShuffleRead (29) +- ShuffleQueryStage (28), Statistics(X) +- ColumnarExchange (27) - +- VeloxAppendBatches (26) + +- VeloxResizeBatches (26) +- ^ ProjectExecTransformer (24) +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (23) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7), Statistics(X) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ NoopFilter (2) : +- ^ Scan parquet (1) @@ -20,7 +20,7 @@ AdaptiveSparkPlan (47) +- ^ InputIteratorTransformer (20) +- ShuffleQueryStage (18), Statistics(X) +- ColumnarExchange (17) - +- VeloxAppendBatches (16) + +- VeloxResizeBatches (16) +- ^ ProjectExecTransformer (14) +- ^ FlushableHashAggregateExecTransformer (13) +- ^ ProjectExecTransformer (12) @@ -64,9 +64,9 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_phone#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_phone#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_phone#X] @@ -112,9 +112,9 @@ Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] Arguments: false -(16) VeloxAppendBatches +(16) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (17) ColumnarExchange Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] @@ -154,9 +154,9 @@ Input [6]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, supplier_no#X, total_ Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] Arguments: false -(26) VeloxAppendBatches +(26) VeloxResizeBatches Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] -Arguments: X +Arguments: X, X (27) ColumnarExchange Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] @@ -267,7 +267,7 @@ AdaptiveSparkPlan (73) +- ^ InputIteratorTransformer (58) +- ShuffleQueryStage (56), Statistics(X) +- ColumnarExchange (55) - +- VeloxAppendBatches (54) + +- VeloxResizeBatches (54) +- ^ ProjectExecTransformer (52) +- ^ FlushableHashAggregateExecTransformer (51) +- ^ ProjectExecTransformer (50) @@ -314,9 +314,9 @@ Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] Arguments: false -(54) VeloxAppendBatches +(54) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (55) ColumnarExchange Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/16.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/16.txt index 86d2f321f653..50a27d401534 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/16.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/16.txt @@ -6,19 +6,19 @@ AdaptiveSparkPlan (71) +- ^ InputIteratorTransformer (44) +- ShuffleQueryStage (42), Statistics(X) +- ColumnarExchange (41) - +- VeloxAppendBatches (40) + +- VeloxResizeBatches (40) +- ^ RegularHashAggregateExecTransformer (38) +- ^ InputIteratorTransformer (37) +- ShuffleQueryStage (35), Statistics(X) +- ColumnarExchange (34) - +- VeloxAppendBatches (33) + +- VeloxResizeBatches (33) +- ^ ProjectExecTransformer (31) +- ^ FlushableHashAggregateExecTransformer (30) +- ^ RegularHashAggregateExecTransformer (29) +- ^ InputIteratorTransformer (28) +- ShuffleQueryStage (26), Statistics(X) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ ProjectExecTransformer (22) +- ^ FlushableHashAggregateExecTransformer (21) +- ^ ProjectExecTransformer (20) @@ -26,14 +26,14 @@ AdaptiveSparkPlan (71) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7), Statistics(X) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ NoopFilter (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ NoopFilter (11) +- ^ Scan parquet (10) @@ -82,9 +82,9 @@ Input [2]: [ps_partkey#X, ps_suppkey#X] Input [3]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X] @@ -119,9 +119,9 @@ Input [4]: [p_partkey#X, p_brand#X, p_type#X, p_size#X] Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_type#X, p_size#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_type#X, p_size#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_type#X, p_size#X] @@ -161,9 +161,9 @@ Input [4]: [p_brand#X, p_type#X, p_size#X, ps_suppkey#X] Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] @@ -201,9 +201,9 @@ Input [4]: [p_brand#X, p_type#X, p_size#X, count#X] Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] Arguments: false -(33) VeloxAppendBatches +(33) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] -Arguments: X +Arguments: X, X (34) ColumnarExchange Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] @@ -230,9 +230,9 @@ Results [4]: [p_brand#X, p_type#X, p_size#X, count(ps_suppkey#X)#X AS supplier_c Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] Arguments: false -(40) VeloxAppendBatches +(40) VeloxResizeBatches Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] -Arguments: X +Arguments: X, X (41) ColumnarExchange Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/17.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/17.txt index 6a2e47576cad..e3596c36ce38 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/17.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/17.txt @@ -12,14 +12,14 @@ AdaptiveSparkPlan (63) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7), Statistics(X) : : +- ColumnarExchange (6) - : : +- VeloxAppendBatches (5) + : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ NoopFilter (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16), Statistics(X) : +- ColumnarExchange (15) - : +- VeloxAppendBatches (14) + : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ NoopFilter (11) : +- ^ Scan parquet (10) @@ -29,7 +29,7 @@ AdaptiveSparkPlan (63) +- ^ InputIteratorTransformer (30) +- ShuffleQueryStage (28), Statistics(X) +- ColumnarExchange (27) - +- VeloxAppendBatches (26) + +- VeloxResizeBatches (26) +- ^ ProjectExecTransformer (24) +- ^ FlushableHashAggregateExecTransformer (23) +- ^ NoopFilter (22) @@ -78,9 +78,9 @@ Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Input [4]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [4]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X] @@ -115,9 +115,9 @@ Input [3]: [p_partkey#X, p_brand#X, p_container#X] Input [2]: [hash_partition_key#X, p_partkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [2]: [hash_partition_key#X, p_partkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [2]: [hash_partition_key#X, p_partkey#X] @@ -168,9 +168,9 @@ Input [3]: [l_partkey#X, sum#X, count#X] Input [4]: [hash_partition_key#X, l_partkey#X, sum#X, count#X] Arguments: false -(26) VeloxAppendBatches +(26) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_partkey#X, sum#X, count#X] -Arguments: X +Arguments: X, X (27) ColumnarExchange Input [4]: [hash_partition_key#X, l_partkey#X, sum#X, count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/18.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/18.txt index 7fe13a003017..e84c68a092d6 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/18.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/18.txt @@ -10,26 +10,26 @@ AdaptiveSparkPlan (110) :- ^ InputIteratorTransformer (46) : +- ShuffleQueryStage (44), Statistics(X) : +- ColumnarExchange (43) - : +- VeloxAppendBatches (42) + : +- VeloxResizeBatches (42) : +- ^ ProjectExecTransformer (40) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (39) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7), Statistics(X) : : +- ColumnarExchange (6) - : : +- VeloxAppendBatches (5) + : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ NoopFilter (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (38) : +- ShuffleQueryStage (36), Statistics(X) : +- ColumnarExchange (35) - : +- VeloxAppendBatches (34) + : +- VeloxResizeBatches (34) : +- ^ ProjectExecTransformer (32) : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (31) : :- ^ InputIteratorTransformer (18) : : +- ShuffleQueryStage (16), Statistics(X) : : +- ColumnarExchange (15) - : : +- VeloxAppendBatches (14) + : : +- VeloxResizeBatches (14) : : +- ^ ProjectExecTransformer (12) : : +- ^ NoopFilter (11) : : +- ^ Scan parquet (10) @@ -39,7 +39,7 @@ AdaptiveSparkPlan (110) : +- ^ InputIteratorTransformer (27) : +- ShuffleQueryStage (25), Statistics(X) : +- ColumnarExchange (24) - : +- VeloxAppendBatches (23) + : +- VeloxResizeBatches (23) : +- ^ ProjectExecTransformer (21) : +- ^ FlushableHashAggregateExecTransformer (20) : +- ^ Scan parquet (19) @@ -47,7 +47,7 @@ AdaptiveSparkPlan (110) :- ^ InputIteratorTransformer (55) : +- ShuffleQueryStage (53), Statistics(X) : +- ColumnarExchange (52) - : +- VeloxAppendBatches (51) + : +- VeloxResizeBatches (51) : +- ^ ProjectExecTransformer (49) : +- ^ NoopFilter (48) : +- ^ Scan parquet (47) @@ -118,9 +118,9 @@ Input [2]: [c_custkey#X, c_name#X] Input [3]: [hash_partition_key#X, c_custkey#X, c_name#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_custkey#X, c_name#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, c_custkey#X, c_name#X] @@ -155,9 +155,9 @@ Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] @@ -194,9 +194,9 @@ Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] Arguments: false -(23) VeloxAppendBatches +(23) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (24) ColumnarExchange Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] @@ -240,9 +240,9 @@ Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: false -(34) VeloxAppendBatches +(34) VeloxResizeBatches Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] -Arguments: X +Arguments: X, X (35) ColumnarExchange Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] @@ -271,9 +271,9 @@ Input [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_ Input [6]: [hash_partition_key#X, c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] Arguments: false -(42) VeloxAppendBatches +(42) VeloxResizeBatches Input [6]: [hash_partition_key#X, c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] -Arguments: X +Arguments: X, X (43) ColumnarExchange Input [6]: [hash_partition_key#X, c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] @@ -308,9 +308,9 @@ Input [2]: [l_orderkey#X, l_quantity#X] Input [3]: [hash_partition_key#X, l_orderkey#X, l_quantity#X] Arguments: false -(51) VeloxAppendBatches +(51) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_quantity#X] -Arguments: X +Arguments: X, X (52) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_quantity#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/19.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/19.txt index 34abb726b85a..ec373f6a4c88 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/19.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/19.txt @@ -9,14 +9,14 @@ AdaptiveSparkPlan (38) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7), Statistics(X) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ NoopFilter (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ NoopFilter (11) +- ^ Scan parquet (10) @@ -55,9 +55,9 @@ Input [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipin Input [5]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [5]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] @@ -92,9 +92,9 @@ Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_size#X, p_container#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_size#X, p_container#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_size#X, p_container#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/20.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/20.txt index 2cf50b2a3a98..bb3a76ec6bb6 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/20.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/20.txt @@ -5,52 +5,52 @@ AdaptiveSparkPlan (143) +- AQEShuffleRead (92) +- ShuffleQueryStage (91), Statistics(X) +- ColumnarExchange (90) - +- VeloxAppendBatches (89) + +- VeloxResizeBatches (89) +- ^ ProjectExecTransformer (87) +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (86) :- ^ InputIteratorTransformer (76) : +- ShuffleQueryStage (74), Statistics(X) : +- ColumnarExchange (73) - : +- VeloxAppendBatches (72) + : +- VeloxResizeBatches (72) : +- ^ ProjectExecTransformer (70) : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (69) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7), Statistics(X) : : +- ColumnarExchange (6) - : : +- VeloxAppendBatches (5) + : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ NoopFilter (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (68) : +- ShuffleQueryStage (66), Statistics(X) : +- ColumnarExchange (65) - : +- VeloxAppendBatches (64) + : +- VeloxResizeBatches (64) : +- ^ ProjectExecTransformer (62) : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (61) : :- ^ InputIteratorTransformer (35) : : +- ShuffleQueryStage (33), Statistics(X) : : +- ColumnarExchange (32) - : : +- VeloxAppendBatches (31) + : : +- VeloxResizeBatches (31) : : +- ^ ProjectExecTransformer (29) : : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (28) : : :- ^ InputIteratorTransformer (18) : : : +- ShuffleQueryStage (16), Statistics(X) : : : +- ColumnarExchange (15) - : : : +- VeloxAppendBatches (14) + : : : +- VeloxResizeBatches (14) : : : +- ^ ProjectExecTransformer (12) : : : +- ^ NoopFilter (11) : : : +- ^ Scan parquet (10) : : +- ^ InputIteratorTransformer (27) : : +- ShuffleQueryStage (25), Statistics(X) : : +- ColumnarExchange (24) - : : +- VeloxAppendBatches (23) + : : +- VeloxResizeBatches (23) : : +- ^ ProjectExecTransformer (21) : : +- ^ NoopFilter (20) : : +- ^ Scan parquet (19) : +- ^ InputIteratorTransformer (60) : +- ShuffleQueryStage (58), Statistics(X) : +- ColumnarExchange (57) - : +- VeloxAppendBatches (56) + : +- VeloxResizeBatches (56) : +- ^ ProjectExecTransformer (54) : +- ^ FilterExecTransformer (53) : +- ^ ProjectExecTransformer (52) @@ -60,7 +60,7 @@ AdaptiveSparkPlan (143) : :- ^ InputIteratorTransformer (44) : : +- ShuffleQueryStage (42), Statistics(X) : : +- ColumnarExchange (41) - : : +- VeloxAppendBatches (40) + : : +- VeloxResizeBatches (40) : : +- ^ ProjectExecTransformer (38) : : +- ^ NoopFilter (37) : : +- ^ Scan parquet (36) @@ -70,7 +70,7 @@ AdaptiveSparkPlan (143) +- ^ InputIteratorTransformer (85) +- ShuffleQueryStage (83), Statistics(X) +- ColumnarExchange (82) - +- VeloxAppendBatches (81) + +- VeloxResizeBatches (81) +- ^ ProjectExecTransformer (79) +- ^ NoopFilter (78) +- ^ Scan parquet (77) @@ -145,9 +145,9 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] @@ -182,9 +182,9 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] @@ -219,9 +219,9 @@ Input [2]: [p_partkey#X, p_name#X] Input [2]: [hash_partition_key#X, p_partkey#X] Arguments: false -(23) VeloxAppendBatches +(23) VeloxResizeBatches Input [2]: [hash_partition_key#X, p_partkey#X] -Arguments: X +Arguments: X, X (24) ColumnarExchange Input [2]: [hash_partition_key#X, p_partkey#X] @@ -250,9 +250,9 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] @@ -287,9 +287,9 @@ Input [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] Input [4]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, l_quantity#X] Arguments: false -(40) VeloxAppendBatches +(40) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, l_quantity#X] -Arguments: X +Arguments: X, X (41) ColumnarExchange Input [4]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, l_quantity#X] @@ -353,9 +353,9 @@ Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: false -(56) VeloxAppendBatches +(56) VeloxResizeBatches Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -Arguments: X +Arguments: X, X (57) ColumnarExchange Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] @@ -384,9 +384,9 @@ Input [6]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, (0.5 * sum(l_quantity))#X Input [2]: [hash_partition_key#X, ps_suppkey#X] Arguments: false -(64) VeloxAppendBatches +(64) VeloxResizeBatches Input [2]: [hash_partition_key#X, ps_suppkey#X] -Arguments: X +Arguments: X, X (65) ColumnarExchange Input [2]: [hash_partition_key#X, ps_suppkey#X] @@ -415,9 +415,9 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] Arguments: false -(72) VeloxAppendBatches +(72) VeloxResizeBatches Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] -Arguments: X +Arguments: X, X (73) ColumnarExchange Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] @@ -452,9 +452,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: false -(81) VeloxAppendBatches +(81) VeloxResizeBatches Input [2]: [hash_partition_key#X, n_nationkey#X] -Arguments: X +Arguments: X, X (82) ColumnarExchange Input [2]: [hash_partition_key#X, n_nationkey#X] @@ -483,9 +483,9 @@ Input [4]: [s_name#X, s_address#X, s_nationkey#X, n_nationkey#X] Input [2]: [s_name#X, s_address#X] Arguments: false -(89) VeloxAppendBatches +(89) VeloxResizeBatches Input [2]: [s_name#X, s_address#X] -Arguments: X +Arguments: X, X (90) ColumnarExchange Input [2]: [s_name#X, s_address#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/21.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/21.txt index f92684b17b15..428acf4c9122 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/21.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/21.txt @@ -6,7 +6,7 @@ AdaptiveSparkPlan (137) +- ^ InputIteratorTransformer (88) +- ShuffleQueryStage (86), Statistics(X) +- ColumnarExchange (85) - +- VeloxAppendBatches (84) + +- VeloxResizeBatches (84) +- ^ ProjectExecTransformer (82) +- ^ FlushableHashAggregateExecTransformer (81) +- ^ ProjectExecTransformer (80) @@ -14,60 +14,60 @@ AdaptiveSparkPlan (137) :- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67), Statistics(X) : +- ColumnarExchange (66) - : +- VeloxAppendBatches (65) + : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (62) : :- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50), Statistics(X) : : +- ColumnarExchange (49) - : : +- VeloxAppendBatches (48) + : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (45) : : :- ^ InputIteratorTransformer (9) : : : +- ShuffleQueryStage (7), Statistics(X) : : : +- ColumnarExchange (6) - : : : +- VeloxAppendBatches (5) + : : : +- VeloxResizeBatches (5) : : : +- ^ ProjectExecTransformer (3) : : : +- ^ NoopFilter (2) : : : +- ^ Scan parquet (1) : : +- ^ InputIteratorTransformer (44) : : +- ShuffleQueryStage (42), Statistics(X) : : +- ColumnarExchange (41) - : : +- VeloxAppendBatches (40) + : : +- VeloxResizeBatches (40) : : +- ^ ProjectExecTransformer (38) : : +- ^ ShuffledHashJoinExecTransformer LeftAnti BuildRight (37) : : :- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (27) : : : :- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : +- ColumnarExchange (15) - : : : : +- VeloxAppendBatches (14) + : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ NoopFilter (11) : : : : +- ^ Scan parquet (10) : : : +- ^ InputIteratorTransformer (26) : : : +- ShuffleQueryStage (24), Statistics(X) : : : +- ColumnarExchange (23) - : : : +- VeloxAppendBatches (22) + : : : +- VeloxResizeBatches (22) : : : +- ^ ProjectExecTransformer (20) : : : +- ^ Scan parquet (19) : : +- ^ InputIteratorTransformer (36) : : +- ShuffleQueryStage (34), Statistics(X) : : +- ColumnarExchange (33) - : : +- VeloxAppendBatches (32) + : : +- VeloxResizeBatches (32) : : +- ^ ProjectExecTransformer (30) : : +- ^ NoopFilter (29) : : +- ^ Scan parquet (28) : +- ^ InputIteratorTransformer (61) : +- ShuffleQueryStage (59), Statistics(X) : +- ColumnarExchange (58) - : +- VeloxAppendBatches (57) + : +- VeloxResizeBatches (57) : +- ^ ProjectExecTransformer (55) : +- ^ NoopFilter (54) : +- ^ Scan parquet (53) +- ^ InputIteratorTransformer (78) +- ShuffleQueryStage (76), Statistics(X) +- ColumnarExchange (75) - +- VeloxAppendBatches (74) + +- VeloxResizeBatches (74) +- ^ ProjectExecTransformer (72) +- ^ NoopFilter (71) +- ^ Scan parquet (70) @@ -138,9 +138,9 @@ Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X] Input [4]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_nationkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [4]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_nationkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [4]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_nationkey#X] @@ -175,9 +175,9 @@ Input [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] @@ -207,9 +207,9 @@ Input [2]: [l_orderkey#X, l_suppkey#X] Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] @@ -249,9 +249,9 @@ Input [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] Arguments: false -(32) VeloxAppendBatches +(32) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] -Arguments: X +Arguments: X, X (33) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] @@ -280,9 +280,9 @@ Input [2]: [l_orderkey#X, l_suppkey#X] Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] Arguments: false -(40) VeloxAppendBatches +(40) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] -Arguments: X +Arguments: X, X (41) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] @@ -311,9 +311,9 @@ Input [5]: [s_suppkey#X, s_name#X, s_nationkey#X, l_orderkey#X, l_suppkey#X] Input [4]: [hash_partition_key#X, s_name#X, s_nationkey#X, l_orderkey#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [4]: [hash_partition_key#X, s_name#X, s_nationkey#X, l_orderkey#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [4]: [hash_partition_key#X, s_name#X, s_nationkey#X, l_orderkey#X] @@ -348,9 +348,9 @@ Input [2]: [o_orderkey#X, o_orderstatus#X] Input [2]: [hash_partition_key#X, o_orderkey#X] Arguments: false -(57) VeloxAppendBatches +(57) VeloxResizeBatches Input [2]: [hash_partition_key#X, o_orderkey#X] -Arguments: X +Arguments: X, X (58) ColumnarExchange Input [2]: [hash_partition_key#X, o_orderkey#X] @@ -379,9 +379,9 @@ Input [4]: [s_name#X, s_nationkey#X, l_orderkey#X, o_orderkey#X] Input [3]: [hash_partition_key#X, s_name#X, s_nationkey#X] Arguments: false -(65) VeloxAppendBatches +(65) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_name#X, s_nationkey#X] -Arguments: X +Arguments: X, X (66) ColumnarExchange Input [3]: [hash_partition_key#X, s_name#X, s_nationkey#X] @@ -416,9 +416,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: false -(74) VeloxAppendBatches +(74) VeloxResizeBatches Input [2]: [hash_partition_key#X, n_nationkey#X] -Arguments: X +Arguments: X, X (75) ColumnarExchange Input [2]: [hash_partition_key#X, n_nationkey#X] @@ -458,9 +458,9 @@ Input [2]: [s_name#X, count#X] Input [3]: [hash_partition_key#X, s_name#X, count#X] Arguments: false -(84) VeloxAppendBatches +(84) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_name#X, count#X] -Arguments: X +Arguments: X, X (85) ColumnarExchange Input [3]: [hash_partition_key#X, s_name#X, count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/22.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/22.txt index 1c2790a4a999..86272c2a803e 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/22.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/22.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (52) +- ^ InputIteratorTransformer (34) +- ShuffleQueryStage (32), Statistics(X) +- ColumnarExchange (31) - +- VeloxAppendBatches (30) + +- VeloxResizeBatches (30) +- ^ RegularHashAggregateExecTransformer (28) +- ^ InputIteratorTransformer (27) +- ShuffleQueryStage (25), Statistics(X) +- ColumnarExchange (24) - +- VeloxAppendBatches (23) + +- VeloxResizeBatches (23) +- ^ ProjectExecTransformer (21) +- ^ FlushableHashAggregateExecTransformer (20) +- ^ ProjectExecTransformer (19) @@ -19,14 +19,14 @@ AdaptiveSparkPlan (52) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7), Statistics(X) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ NoopFilter (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (17) +- ShuffleQueryStage (15), Statistics(X) +- ColumnarExchange (14) - +- VeloxAppendBatches (13) + +- VeloxResizeBatches (13) +- ^ ProjectExecTransformer (11) +- ^ Scan parquet (10) +- == Initial Plan == @@ -65,9 +65,9 @@ Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X] Input [4]: [hash_partition_key#X, c_custkey#X, c_phone#X, c_acctbal#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [4]: [hash_partition_key#X, c_custkey#X, c_phone#X, c_acctbal#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [4]: [hash_partition_key#X, c_custkey#X, c_phone#X, c_acctbal#X] @@ -97,9 +97,9 @@ Input [1]: [o_custkey#X] Input [2]: [hash_partition_key#X, o_custkey#X] Arguments: false -(13) VeloxAppendBatches +(13) VeloxResizeBatches Input [2]: [hash_partition_key#X, o_custkey#X] -Arguments: X +Arguments: X, X (14) ColumnarExchange Input [2]: [hash_partition_key#X, o_custkey#X] @@ -139,9 +139,9 @@ Input [4]: [cntrycode#X, count#X, sum#X, isEmpty#X] Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] Arguments: false -(23) VeloxAppendBatches +(23) VeloxResizeBatches Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (24) ColumnarExchange Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] @@ -168,9 +168,9 @@ Results [3]: [cntrycode#X, count(1)#X AS numcust#X, sum(c_acctbal#X)#X AS totacc Input [3]: [cntrycode#X, numcust#X, totacctbal#X] Arguments: false -(30) VeloxAppendBatches +(30) VeloxResizeBatches Input [3]: [cntrycode#X, numcust#X, totacctbal#X] -Arguments: X +Arguments: X, X (31) ColumnarExchange Input [3]: [cntrycode#X, numcust#X, totacctbal#X] @@ -279,7 +279,7 @@ AdaptiveSparkPlan (72) +- ^ InputIteratorTransformer (62) +- ShuffleQueryStage (60), Statistics(X) +- ColumnarExchange (59) - +- VeloxAppendBatches (58) + +- VeloxResizeBatches (58) +- ^ FlushableHashAggregateExecTransformer (56) +- ^ ProjectExecTransformer (55) +- ^ NoopFilter (54) @@ -319,9 +319,9 @@ Results [2]: [sum#X, count#X] Input [2]: [sum#X, count#X] Arguments: false -(58) VeloxAppendBatches +(58) VeloxResizeBatches Input [2]: [sum#X, count#X] -Arguments: X +Arguments: X, X (59) ColumnarExchange Input [2]: [sum#X, count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/3.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/3.txt index 1f9905294144..40f7aeb35749 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/3.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/3.txt @@ -11,27 +11,27 @@ AdaptiveSparkPlan (67) :- ^ InputIteratorTransformer (26) : +- ShuffleQueryStage (24), Statistics(X) : +- ColumnarExchange (23) - : +- VeloxAppendBatches (22) + : +- VeloxResizeBatches (22) : +- ^ ProjectExecTransformer (20) : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7), Statistics(X) : : +- ColumnarExchange (6) - : : +- VeloxAppendBatches (5) + : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ NoopFilter (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16), Statistics(X) : +- ColumnarExchange (15) - : +- VeloxAppendBatches (14) + : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ NoopFilter (11) : +- ^ Scan parquet (10) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33), Statistics(X) +- ColumnarExchange (32) - +- VeloxAppendBatches (31) + +- VeloxResizeBatches (31) +- ^ ProjectExecTransformer (29) +- ^ NoopFilter (28) +- ^ Scan parquet (27) @@ -80,9 +80,9 @@ Input [2]: [c_custkey#X, c_mktsegment#X] Input [2]: [hash_partition_key#X, c_custkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [2]: [hash_partition_key#X, c_custkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [2]: [hash_partition_key#X, c_custkey#X] @@ -117,9 +117,9 @@ Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] @@ -148,9 +148,9 @@ Input [5]: [c_custkey#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriorit Input [4]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X, o_shippriority#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [4]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X, o_shippriority#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [4]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X, o_shippriority#X] @@ -185,9 +185,9 @@ Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/4.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/4.txt index 130bc2983040..eb6a3e27f9f3 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/4.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/4.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (56) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33), Statistics(X) +- ColumnarExchange (32) - +- VeloxAppendBatches (31) + +- VeloxResizeBatches (31) +- ^ RegularHashAggregateExecTransformer (29) +- ^ InputIteratorTransformer (28) +- ShuffleQueryStage (26), Statistics(X) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ ProjectExecTransformer (22) +- ^ FlushableHashAggregateExecTransformer (21) +- ^ ProjectExecTransformer (20) @@ -19,14 +19,14 @@ AdaptiveSparkPlan (56) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7), Statistics(X) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ NoopFilter (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ NoopFilter (11) +- ^ Scan parquet (10) @@ -69,9 +69,9 @@ Input [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] @@ -106,9 +106,9 @@ Input [3]: [l_orderkey#X, l_commitdate#X, l_receiptdate#X] Input [2]: [hash_partition_key#X, l_orderkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [2]: [hash_partition_key#X, l_orderkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [2]: [hash_partition_key#X, l_orderkey#X] @@ -148,9 +148,9 @@ Input [2]: [o_orderpriority#X, count#X] Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] @@ -177,9 +177,9 @@ Results [2]: [o_orderpriority#X, count(1)#X AS order_count#X] Input [2]: [o_orderpriority#X, order_count#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [2]: [o_orderpriority#X, order_count#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [2]: [o_orderpriority#X, order_count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/5.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/5.txt index 10ce074fd760..20d0aa276e59 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/5.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/5.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (156) +- ^ InputIteratorTransformer (103) +- ShuffleQueryStage (101), Statistics(X) +- ColumnarExchange (100) - +- VeloxAppendBatches (99) + +- VeloxResizeBatches (99) +- ^ RegularHashAggregateExecTransformer (97) +- ^ InputIteratorTransformer (96) +- ShuffleQueryStage (94), Statistics(X) +- ColumnarExchange (93) - +- VeloxAppendBatches (92) + +- VeloxResizeBatches (92) +- ^ ProjectExecTransformer (90) +- ^ FlushableHashAggregateExecTransformer (89) +- ^ ProjectExecTransformer (88) @@ -19,66 +19,66 @@ AdaptiveSparkPlan (156) :- ^ InputIteratorTransformer (77) : +- ShuffleQueryStage (75), Statistics(X) : +- ColumnarExchange (74) - : +- VeloxAppendBatches (73) + : +- VeloxResizeBatches (73) : +- ^ ProjectExecTransformer (71) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (70) : :- ^ InputIteratorTransformer (60) : : +- ShuffleQueryStage (58), Statistics(X) : : +- ColumnarExchange (57) - : : +- VeloxAppendBatches (56) + : : +- VeloxResizeBatches (56) : : +- ^ ProjectExecTransformer (54) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (53) : : :- ^ InputIteratorTransformer (43) : : : +- ShuffleQueryStage (41), Statistics(X) : : : +- ColumnarExchange (40) - : : : +- VeloxAppendBatches (39) + : : : +- VeloxResizeBatches (39) : : : +- ^ ProjectExecTransformer (37) : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (36) : : : :- ^ InputIteratorTransformer (26) : : : : +- ShuffleQueryStage (24), Statistics(X) : : : : +- ColumnarExchange (23) - : : : : +- VeloxAppendBatches (22) + : : : : +- VeloxResizeBatches (22) : : : : +- ^ ProjectExecTransformer (20) : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) : : : : :- ^ InputIteratorTransformer (9) : : : : : +- ShuffleQueryStage (7), Statistics(X) : : : : : +- ColumnarExchange (6) - : : : : : +- VeloxAppendBatches (5) + : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ NoopFilter (2) : : : : : +- ^ Scan parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : +- ColumnarExchange (15) - : : : : +- VeloxAppendBatches (14) + : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ NoopFilter (11) : : : : +- ^ Scan parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33), Statistics(X) : : : +- ColumnarExchange (32) - : : : +- VeloxAppendBatches (31) + : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ NoopFilter (28) : : : +- ^ Scan parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50), Statistics(X) : : +- ColumnarExchange (49) - : : +- VeloxAppendBatches (48) + : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ NoopFilter (45) : : +- ^ Scan parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67), Statistics(X) : +- ColumnarExchange (66) - : +- VeloxAppendBatches (65) + : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ NoopFilter (62) : +- ^ Scan parquet (61) +- ^ InputIteratorTransformer (86) +- ShuffleQueryStage (84), Statistics(X) +- ColumnarExchange (83) - +- VeloxAppendBatches (82) + +- VeloxResizeBatches (82) +- ^ ProjectExecTransformer (80) +- ^ NoopFilter (79) +- ^ Scan parquet (78) @@ -153,9 +153,9 @@ Input [2]: [c_custkey#X, c_nationkey#X] Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] @@ -190,9 +190,9 @@ Input [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] @@ -221,9 +221,9 @@ Input [4]: [c_custkey#X, c_nationkey#X, o_orderkey#X, o_custkey#X] Input [3]: [hash_partition_key#X, c_nationkey#X, o_orderkey#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_nationkey#X, o_orderkey#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [3]: [hash_partition_key#X, c_nationkey#X, o_orderkey#X] @@ -258,9 +258,9 @@ Input [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] @@ -289,9 +289,9 @@ Input [6]: [c_nationkey#X, o_orderkey#X, l_orderkey#X, l_suppkey#X, l_extendedpr Input [5]: [hash_partition_key#X, c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(39) VeloxAppendBatches +(39) VeloxResizeBatches Input [5]: [hash_partition_key#X, c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (40) ColumnarExchange Input [5]: [hash_partition_key#X, c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] @@ -326,9 +326,9 @@ Input [2]: [s_suppkey#X, s_nationkey#X] Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -357,9 +357,9 @@ Input [6]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, s_suppk Input [4]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X] Arguments: false -(56) VeloxAppendBatches +(56) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X] -Arguments: X +Arguments: X, X (57) ColumnarExchange Input [4]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X] @@ -394,9 +394,9 @@ Input [3]: [n_nationkey#X, n_name#X, n_regionkey#X] Input [4]: [hash_partition_key#X, n_nationkey#X, n_name#X, n_regionkey#X] Arguments: false -(65) VeloxAppendBatches +(65) VeloxResizeBatches Input [4]: [hash_partition_key#X, n_nationkey#X, n_name#X, n_regionkey#X] -Arguments: X +Arguments: X, X (66) ColumnarExchange Input [4]: [hash_partition_key#X, n_nationkey#X, n_name#X, n_regionkey#X] @@ -425,9 +425,9 @@ Input [6]: [l_extendedprice#X, l_discount#X, s_nationkey#X, n_nationkey#X, n_nam Input [5]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] Arguments: false -(73) VeloxAppendBatches +(73) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] -Arguments: X +Arguments: X, X (74) ColumnarExchange Input [5]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] @@ -462,9 +462,9 @@ Input [2]: [r_regionkey#X, r_name#X] Input [2]: [hash_partition_key#X, r_regionkey#X] Arguments: false -(82) VeloxAppendBatches +(82) VeloxResizeBatches Input [2]: [hash_partition_key#X, r_regionkey#X] -Arguments: X +Arguments: X, X (83) ColumnarExchange Input [2]: [hash_partition_key#X, r_regionkey#X] @@ -504,9 +504,9 @@ Input [3]: [n_name#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] Arguments: false -(92) VeloxAppendBatches +(92) VeloxResizeBatches Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (93) ColumnarExchange Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] @@ -533,9 +533,9 @@ Results [2]: [n_name#X, sum(CheckOverflow((promote_precision(cast(l_extendedpric Input [2]: [n_name#X, revenue#X] Arguments: false -(99) VeloxAppendBatches +(99) VeloxResizeBatches Input [2]: [n_name#X, revenue#X] -Arguments: X +Arguments: X, X (100) ColumnarExchange Input [2]: [n_name#X, revenue#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/6.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/6.txt index fa9c936a0ca1..1aaed506d7e0 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/6.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/6.txt @@ -6,7 +6,7 @@ AdaptiveSparkPlan (20) +- ^ InputIteratorTransformer (10) +- ShuffleQueryStage (8), Statistics(X) +- ColumnarExchange (7) - +- VeloxAppendBatches (6) + +- VeloxResizeBatches (6) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ NoopFilter (2) @@ -46,9 +46,9 @@ Results [2]: [sum#X, isEmpty#X] Input [2]: [sum#X, isEmpty#X] Arguments: false -(6) VeloxAppendBatches +(6) VeloxResizeBatches Input [2]: [sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (7) ColumnarExchange Input [2]: [sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/7.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/7.txt index 651cfa840be8..2261654d4a50 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/7.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/7.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (149) +- ^ InputIteratorTransformer (98) +- ShuffleQueryStage (96), Statistics(X) +- ColumnarExchange (95) - +- VeloxAppendBatches (94) + +- VeloxResizeBatches (94) +- ^ RegularHashAggregateExecTransformer (92) +- ^ InputIteratorTransformer (91) +- ShuffleQueryStage (89), Statistics(X) +- ColumnarExchange (88) - +- VeloxAppendBatches (87) + +- VeloxResizeBatches (87) +- ^ ProjectExecTransformer (85) +- ^ FlushableHashAggregateExecTransformer (84) +- ^ ProjectExecTransformer (83) @@ -19,59 +19,59 @@ AdaptiveSparkPlan (149) :- ^ InputIteratorTransformer (77) : +- ShuffleQueryStage (75), Statistics(X) : +- ColumnarExchange (74) - : +- VeloxAppendBatches (73) + : +- VeloxResizeBatches (73) : +- ^ ProjectExecTransformer (71) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (70) : :- ^ InputIteratorTransformer (60) : : +- ShuffleQueryStage (58), Statistics(X) : : +- ColumnarExchange (57) - : : +- VeloxAppendBatches (56) + : : +- VeloxResizeBatches (56) : : +- ^ ProjectExecTransformer (54) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (53) : : :- ^ InputIteratorTransformer (43) : : : +- ShuffleQueryStage (41), Statistics(X) : : : +- ColumnarExchange (40) - : : : +- VeloxAppendBatches (39) + : : : +- VeloxResizeBatches (39) : : : +- ^ ProjectExecTransformer (37) : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (36) : : : :- ^ InputIteratorTransformer (26) : : : : +- ShuffleQueryStage (24), Statistics(X) : : : : +- ColumnarExchange (23) - : : : : +- VeloxAppendBatches (22) + : : : : +- VeloxResizeBatches (22) : : : : +- ^ ProjectExecTransformer (20) : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) : : : : :- ^ InputIteratorTransformer (9) : : : : : +- ShuffleQueryStage (7), Statistics(X) : : : : : +- ColumnarExchange (6) - : : : : : +- VeloxAppendBatches (5) + : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ NoopFilter (2) : : : : : +- ^ Scan parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : +- ColumnarExchange (15) - : : : : +- VeloxAppendBatches (14) + : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ NoopFilter (11) : : : : +- ^ Scan parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33), Statistics(X) : : : +- ColumnarExchange (32) - : : : +- VeloxAppendBatches (31) + : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ NoopFilter (28) : : : +- ^ Scan parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50), Statistics(X) : : +- ColumnarExchange (49) - : : +- VeloxAppendBatches (48) + : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ NoopFilter (45) : : +- ^ Scan parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67), Statistics(X) : +- ColumnarExchange (66) - : +- VeloxAppendBatches (65) + : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ NoopFilter (62) : +- ^ Scan parquet (61) @@ -147,9 +147,9 @@ Input [2]: [s_suppkey#X, s_nationkey#X] Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -184,9 +184,9 @@ Input [5]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipda Input [6]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [6]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] @@ -215,9 +215,9 @@ Input [7]: [s_suppkey#X, s_nationkey#X, l_orderkey#X, l_suppkey#X, l_extendedpri Input [6]: [hash_partition_key#X, s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [6]: [hash_partition_key#X, s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [6]: [hash_partition_key#X, s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] @@ -252,9 +252,9 @@ Input [2]: [o_orderkey#X, o_custkey#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] @@ -283,9 +283,9 @@ Input [7]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_ship Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X] Arguments: false -(39) VeloxAppendBatches +(39) VeloxResizeBatches Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X] -Arguments: X +Arguments: X, X (40) ColumnarExchange Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X] @@ -320,9 +320,9 @@ Input [2]: [c_custkey#X, c_nationkey#X] Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] @@ -351,9 +351,9 @@ Input [7]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_cust Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X] Arguments: false -(56) VeloxAppendBatches +(56) VeloxResizeBatches Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X] -Arguments: X +Arguments: X, X (57) ColumnarExchange Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X] @@ -388,9 +388,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] Arguments: false -(65) VeloxAppendBatches +(65) VeloxResizeBatches Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] -Arguments: X +Arguments: X, X (66) ColumnarExchange Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] @@ -419,9 +419,9 @@ Input [7]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nati Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X, n_name#X] Arguments: false -(73) VeloxAppendBatches +(73) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X, n_name#X] -Arguments: X +Arguments: X, X (74) ColumnarExchange Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X, n_name#X] @@ -474,9 +474,9 @@ Input [5]: [supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] Arguments: false -(87) VeloxAppendBatches +(87) VeloxResizeBatches Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (88) ColumnarExchange Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] @@ -503,9 +503,9 @@ Results [4]: [supp_nation#X, cust_nation#X, l_year#X, sum(volume#X)#X AS revenue Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] Arguments: false -(94) VeloxAppendBatches +(94) VeloxResizeBatches Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] -Arguments: X +Arguments: X, X (95) ColumnarExchange Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/8.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/8.txt index e359f4c944e7..3e836d995cb9 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/8.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/8.txt @@ -6,13 +6,13 @@ AdaptiveSparkPlan (207) +- ^ InputIteratorTransformer (138) +- ShuffleQueryStage (136), Statistics(X) +- ColumnarExchange (135) - +- VeloxAppendBatches (134) + +- VeloxResizeBatches (134) +- ^ ProjectExecTransformer (132) +- ^ RegularHashAggregateExecTransformer (131) +- ^ InputIteratorTransformer (130) +- ShuffleQueryStage (128), Statistics(X) +- ColumnarExchange (127) - +- VeloxAppendBatches (126) + +- VeloxResizeBatches (126) +- ^ ProjectExecTransformer (124) +- ^ FlushableHashAggregateExecTransformer (123) +- ^ ProjectExecTransformer (122) @@ -20,92 +20,92 @@ AdaptiveSparkPlan (207) :- ^ InputIteratorTransformer (111) : +- ShuffleQueryStage (109), Statistics(X) : +- ColumnarExchange (108) - : +- VeloxAppendBatches (107) + : +- VeloxResizeBatches (107) : +- ^ ProjectExecTransformer (105) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (104) : :- ^ InputIteratorTransformer (94) : : +- ShuffleQueryStage (92), Statistics(X) : : +- ColumnarExchange (91) - : : +- VeloxAppendBatches (90) + : : +- VeloxResizeBatches (90) : : +- ^ ProjectExecTransformer (88) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (87) : : :- ^ InputIteratorTransformer (77) : : : +- ShuffleQueryStage (75), Statistics(X) : : : +- ColumnarExchange (74) - : : : +- VeloxAppendBatches (73) + : : : +- VeloxResizeBatches (73) : : : +- ^ ProjectExecTransformer (71) : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (70) : : : :- ^ InputIteratorTransformer (60) : : : : +- ShuffleQueryStage (58), Statistics(X) : : : : +- ColumnarExchange (57) - : : : : +- VeloxAppendBatches (56) + : : : : +- VeloxResizeBatches (56) : : : : +- ^ ProjectExecTransformer (54) : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (53) : : : : :- ^ InputIteratorTransformer (43) : : : : : +- ShuffleQueryStage (41), Statistics(X) : : : : : +- ColumnarExchange (40) - : : : : : +- VeloxAppendBatches (39) + : : : : : +- VeloxResizeBatches (39) : : : : : +- ^ ProjectExecTransformer (37) : : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (36) : : : : : :- ^ InputIteratorTransformer (26) : : : : : : +- ShuffleQueryStage (24), Statistics(X) : : : : : : +- ColumnarExchange (23) - : : : : : : +- VeloxAppendBatches (22) + : : : : : : +- VeloxResizeBatches (22) : : : : : : +- ^ ProjectExecTransformer (20) : : : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) : : : : : : :- ^ InputIteratorTransformer (9) : : : : : : : +- ShuffleQueryStage (7), Statistics(X) : : : : : : : +- ColumnarExchange (6) - : : : : : : : +- VeloxAppendBatches (5) + : : : : : : : +- VeloxResizeBatches (5) : : : : : : : +- ^ ProjectExecTransformer (3) : : : : : : : +- ^ NoopFilter (2) : : : : : : : +- ^ Scan parquet (1) : : : : : : +- ^ InputIteratorTransformer (18) : : : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : : : +- ColumnarExchange (15) - : : : : : : +- VeloxAppendBatches (14) + : : : : : : +- VeloxResizeBatches (14) : : : : : : +- ^ ProjectExecTransformer (12) : : : : : : +- ^ NoopFilter (11) : : : : : : +- ^ Scan parquet (10) : : : : : +- ^ InputIteratorTransformer (35) : : : : : +- ShuffleQueryStage (33), Statistics(X) : : : : : +- ColumnarExchange (32) - : : : : : +- VeloxAppendBatches (31) + : : : : : +- VeloxResizeBatches (31) : : : : : +- ^ ProjectExecTransformer (29) : : : : : +- ^ NoopFilter (28) : : : : : +- ^ Scan parquet (27) : : : : +- ^ InputIteratorTransformer (52) : : : : +- ShuffleQueryStage (50), Statistics(X) : : : : +- ColumnarExchange (49) - : : : : +- VeloxAppendBatches (48) + : : : : +- VeloxResizeBatches (48) : : : : +- ^ ProjectExecTransformer (46) : : : : +- ^ NoopFilter (45) : : : : +- ^ Scan parquet (44) : : : +- ^ InputIteratorTransformer (69) : : : +- ShuffleQueryStage (67), Statistics(X) : : : +- ColumnarExchange (66) - : : : +- VeloxAppendBatches (65) + : : : +- VeloxResizeBatches (65) : : : +- ^ ProjectExecTransformer (63) : : : +- ^ NoopFilter (62) : : : +- ^ Scan parquet (61) : : +- ^ InputIteratorTransformer (86) : : +- ShuffleQueryStage (84), Statistics(X) : : +- ColumnarExchange (83) - : : +- VeloxAppendBatches (82) + : : +- VeloxResizeBatches (82) : : +- ^ ProjectExecTransformer (80) : : +- ^ NoopFilter (79) : : +- ^ Scan parquet (78) : +- ^ InputIteratorTransformer (103) : +- ShuffleQueryStage (101), Statistics(X) : +- ColumnarExchange (100) - : +- VeloxAppendBatches (99) + : +- VeloxResizeBatches (99) : +- ^ ProjectExecTransformer (97) : +- ^ NoopFilter (96) : +- ^ Scan parquet (95) +- ^ InputIteratorTransformer (120) +- ShuffleQueryStage (118), Statistics(X) +- ColumnarExchange (117) - +- VeloxAppendBatches (116) + +- VeloxResizeBatches (116) +- ^ ProjectExecTransformer (114) +- ^ NoopFilter (113) +- ^ Scan parquet (112) @@ -196,9 +196,9 @@ Input [2]: [p_partkey#X, p_type#X] Input [2]: [hash_partition_key#X, p_partkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [2]: [hash_partition_key#X, p_partkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [2]: [hash_partition_key#X, p_partkey#X] @@ -233,9 +233,9 @@ Input [5]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discoun Input [6]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [6]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] @@ -264,9 +264,9 @@ Input [6]: [p_partkey#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] @@ -301,9 +301,9 @@ Input [2]: [s_suppkey#X, s_nationkey#X] Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -332,9 +332,9 @@ Input [6]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, s_suppke Input [5]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] Arguments: false -(39) VeloxAppendBatches +(39) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] -Arguments: X +Arguments: X, X (40) ColumnarExchange Input [5]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] @@ -369,9 +369,9 @@ Input [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Input [4]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [4]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [4]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X] @@ -400,9 +400,9 @@ Input [7]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orde Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X] Arguments: false -(56) VeloxAppendBatches +(56) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X] -Arguments: X +Arguments: X, X (57) ColumnarExchange Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X] @@ -437,9 +437,9 @@ Input [2]: [c_custkey#X, c_nationkey#X] Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] Arguments: false -(65) VeloxAppendBatches +(65) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] -Arguments: X +Arguments: X, X (66) ColumnarExchange Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] @@ -468,9 +468,9 @@ Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_order Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X] Arguments: false -(73) VeloxAppendBatches +(73) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X] -Arguments: X +Arguments: X, X (74) ColumnarExchange Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X] @@ -505,9 +505,9 @@ Input [2]: [n_nationkey#X, n_regionkey#X] Input [3]: [hash_partition_key#X, n_nationkey#X, n_regionkey#X] Arguments: false -(82) VeloxAppendBatches +(82) VeloxResizeBatches Input [3]: [hash_partition_key#X, n_nationkey#X, n_regionkey#X] -Arguments: X +Arguments: X, X (83) ColumnarExchange Input [3]: [hash_partition_key#X, n_nationkey#X, n_regionkey#X] @@ -536,9 +536,9 @@ Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nat Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X] Arguments: false -(90) VeloxAppendBatches +(90) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X] -Arguments: X +Arguments: X, X (91) ColumnarExchange Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X] @@ -573,9 +573,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] Arguments: false -(99) VeloxAppendBatches +(99) VeloxResizeBatches Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] -Arguments: X +Arguments: X, X (100) ColumnarExchange Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] @@ -604,9 +604,9 @@ Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_reg Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X] Arguments: false -(107) VeloxAppendBatches +(107) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X] -Arguments: X +Arguments: X, X (108) ColumnarExchange Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X] @@ -641,9 +641,9 @@ Input [2]: [r_regionkey#X, r_name#X] Input [2]: [hash_partition_key#X, r_regionkey#X] Arguments: false -(116) VeloxAppendBatches +(116) VeloxResizeBatches Input [2]: [hash_partition_key#X, r_regionkey#X] -Arguments: X +Arguments: X, X (117) ColumnarExchange Input [2]: [hash_partition_key#X, r_regionkey#X] @@ -683,9 +683,9 @@ Input [5]: [o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] Arguments: false -(126) VeloxAppendBatches +(126) VeloxResizeBatches Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (127) ColumnarExchange Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] @@ -716,9 +716,9 @@ Input [3]: [o_year#X, sum(CASE WHEN (nation#X = BRAZIL) THEN volume#X ELSE 0.000 Input [2]: [o_year#X, mkt_share#X] Arguments: false -(134) VeloxAppendBatches +(134) VeloxResizeBatches Input [2]: [o_year#X, mkt_share#X] -Arguments: X +Arguments: X, X (135) ColumnarExchange Input [2]: [o_year#X, mkt_share#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/9.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/9.txt index 21c91ca14180..8ed229096ba6 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/9.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/9.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (155) +- ^ InputIteratorTransformer (103) +- ShuffleQueryStage (101), Statistics(X) +- ColumnarExchange (100) - +- VeloxAppendBatches (99) + +- VeloxResizeBatches (99) +- ^ RegularHashAggregateExecTransformer (97) +- ^ InputIteratorTransformer (96) +- ShuffleQueryStage (94), Statistics(X) +- ColumnarExchange (93) - +- VeloxAppendBatches (92) + +- VeloxResizeBatches (92) +- ^ ProjectExecTransformer (90) +- ^ FlushableHashAggregateExecTransformer (89) +- ^ ProjectExecTransformer (88) @@ -19,66 +19,66 @@ AdaptiveSparkPlan (155) :- ^ InputIteratorTransformer (77) : +- ShuffleQueryStage (75), Statistics(X) : +- ColumnarExchange (74) - : +- VeloxAppendBatches (73) + : +- VeloxResizeBatches (73) : +- ^ ProjectExecTransformer (71) : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (70) : :- ^ InputIteratorTransformer (60) : : +- ShuffleQueryStage (58), Statistics(X) : : +- ColumnarExchange (57) - : : +- VeloxAppendBatches (56) + : : +- VeloxResizeBatches (56) : : +- ^ ProjectExecTransformer (54) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (53) : : :- ^ InputIteratorTransformer (43) : : : +- ShuffleQueryStage (41), Statistics(X) : : : +- ColumnarExchange (40) - : : : +- VeloxAppendBatches (39) + : : : +- VeloxResizeBatches (39) : : : +- ^ ProjectExecTransformer (37) : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (36) : : : :- ^ InputIteratorTransformer (26) : : : : +- ShuffleQueryStage (24), Statistics(X) : : : : +- ColumnarExchange (23) - : : : : +- VeloxAppendBatches (22) + : : : : +- VeloxResizeBatches (22) : : : : +- ^ ProjectExecTransformer (20) : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) : : : : :- ^ InputIteratorTransformer (9) : : : : : +- ShuffleQueryStage (7), Statistics(X) : : : : : +- ColumnarExchange (6) - : : : : : +- VeloxAppendBatches (5) + : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ NoopFilter (2) : : : : : +- ^ Scan parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : +- ColumnarExchange (15) - : : : : +- VeloxAppendBatches (14) + : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ NoopFilter (11) : : : : +- ^ Scan parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33), Statistics(X) : : : +- ColumnarExchange (32) - : : : +- VeloxAppendBatches (31) + : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ NoopFilter (28) : : : +- ^ Scan parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50), Statistics(X) : : +- ColumnarExchange (49) - : : +- VeloxAppendBatches (48) + : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ NoopFilter (45) : : +- ^ Scan parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67), Statistics(X) : +- ColumnarExchange (66) - : +- VeloxAppendBatches (65) + : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ NoopFilter (62) : +- ^ Scan parquet (61) +- ^ InputIteratorTransformer (86) +- ShuffleQueryStage (84), Statistics(X) +- ColumnarExchange (83) - +- VeloxAppendBatches (82) + +- VeloxResizeBatches (82) +- ^ ProjectExecTransformer (80) +- ^ NoopFilter (79) +- ^ Scan parquet (78) @@ -152,9 +152,9 @@ Input [2]: [p_partkey#X, p_name#X] Input [2]: [hash_partition_key#X, p_partkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [2]: [hash_partition_key#X, p_partkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [2]: [hash_partition_key#X, p_partkey#X] @@ -189,9 +189,9 @@ Input [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpric Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] @@ -220,9 +220,9 @@ Input [7]: [p_partkey#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] @@ -257,9 +257,9 @@ Input [2]: [s_suppkey#X, s_nationkey#X] Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -288,9 +288,9 @@ Input [8]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpric Input [8]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X] Arguments: false -(39) VeloxAppendBatches +(39) VeloxResizeBatches Input [8]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X] -Arguments: X +Arguments: X, X (40) ColumnarExchange Input [8]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X] @@ -325,9 +325,9 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_supplycost#X] Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_supplycost#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_supplycost#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_supplycost#X] @@ -356,9 +356,9 @@ Input [10]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpri Input [7]: [hash_partition_key#X, l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X] Arguments: false -(56) VeloxAppendBatches +(56) VeloxResizeBatches Input [7]: [hash_partition_key#X, l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X] -Arguments: X +Arguments: X, X (57) ColumnarExchange Input [7]: [hash_partition_key#X, l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X] @@ -393,9 +393,9 @@ Input [2]: [o_orderkey#X, o_orderdate#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X] Arguments: false -(65) VeloxAppendBatches +(65) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X] -Arguments: X +Arguments: X, X (66) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X] @@ -424,9 +424,9 @@ Input [8]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_natio Input [7]: [hash_partition_key#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X] Arguments: false -(73) VeloxAppendBatches +(73) VeloxResizeBatches Input [7]: [hash_partition_key#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X] -Arguments: X +Arguments: X, X (74) ColumnarExchange Input [7]: [hash_partition_key#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X] @@ -461,9 +461,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] Arguments: false -(82) VeloxAppendBatches +(82) VeloxResizeBatches Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] -Arguments: X +Arguments: X, X (83) ColumnarExchange Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] @@ -503,9 +503,9 @@ Input [4]: [nation#X, o_year#X, sum#X, isEmpty#X] Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] Arguments: false -(92) VeloxAppendBatches +(92) VeloxResizeBatches Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (93) ColumnarExchange Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] @@ -532,9 +532,9 @@ Results [3]: [nation#X, o_year#X, sum(amount#X)#X AS sum_profit#X] Input [3]: [nation#X, o_year#X, sum_profit#X] Arguments: false -(99) VeloxAppendBatches +(99) VeloxResizeBatches Input [3]: [nation#X, o_year#X, sum_profit#X] -Arguments: X +Arguments: X, X (100) ColumnarExchange Input [3]: [nation#X, o_year#X, sum_profit#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/1.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/1.txt index 545f2e7e086d..2db104cfeb12 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/1.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/1.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (30) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ RegularHashAggregateExecTransformer (12) +- ^ InputIteratorTransformer (11) +- ShuffleQueryStage (9), Statistics(X) +- ColumnarExchange (8) - +- VeloxAppendBatches (7) + +- VeloxResizeBatches (7) +- ^ ProjectExecTransformer (5) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) @@ -58,9 +58,9 @@ Input [17]: [l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] Arguments: false -(7) VeloxAppendBatches +(7) VeloxResizeBatches Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] -Arguments: X +Arguments: X, X (8) ColumnarExchange Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] @@ -87,9 +87,9 @@ Results [10]: [l_returnflag#X, l_linestatus#X, sum(l_quantity#X)#X AS sum_qty#X, Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/10.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/10.txt index 5e48ceb742d7..02aaa69a3ea8 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/10.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/10.txt @@ -8,7 +8,7 @@ AdaptiveSparkPlan (100) +- ^ InputIteratorTransformer (62) +- ShuffleQueryStage (60), Statistics(X) +- ColumnarExchange (59) - +- VeloxAppendBatches (58) + +- VeloxResizeBatches (58) +- ^ ProjectExecTransformer (56) +- ^ FlushableHashAggregateExecTransformer (55) +- ^ ProjectExecTransformer (54) @@ -16,40 +16,40 @@ AdaptiveSparkPlan (100) :- ^ InputIteratorTransformer (43) : +- ShuffleQueryStage (41), Statistics(X) : +- ColumnarExchange (40) - : +- VeloxAppendBatches (39) + : +- VeloxResizeBatches (39) : +- ^ ProjectExecTransformer (37) : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (36) : :- ^ InputIteratorTransformer (26) : : +- ShuffleQueryStage (24), Statistics(X) : : +- ColumnarExchange (23) - : : +- VeloxAppendBatches (22) + : : +- VeloxResizeBatches (22) : : +- ^ ProjectExecTransformer (20) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) : : :- ^ InputIteratorTransformer (9) : : : +- ShuffleQueryStage (7), Statistics(X) : : : +- ColumnarExchange (6) - : : : +- VeloxAppendBatches (5) + : : : +- VeloxResizeBatches (5) : : : +- ^ ProjectExecTransformer (3) : : : +- ^ NoopFilter (2) : : : +- ^ Scan parquet (1) : : +- ^ InputIteratorTransformer (18) : : +- ShuffleQueryStage (16), Statistics(X) : : +- ColumnarExchange (15) - : : +- VeloxAppendBatches (14) + : : +- VeloxResizeBatches (14) : : +- ^ ProjectExecTransformer (12) : : +- ^ NoopFilter (11) : : +- ^ Scan parquet (10) : +- ^ InputIteratorTransformer (35) : +- ShuffleQueryStage (33), Statistics(X) : +- ColumnarExchange (32) - : +- VeloxAppendBatches (31) + : +- VeloxResizeBatches (31) : +- ^ ProjectExecTransformer (29) : +- ^ NoopFilter (28) : +- ^ Scan parquet (27) +- ^ InputIteratorTransformer (52) +- ShuffleQueryStage (50), Statistics(X) +- ColumnarExchange (49) - +- VeloxAppendBatches (48) + +- VeloxResizeBatches (48) +- ^ ProjectExecTransformer (46) +- ^ NoopFilter (45) +- ^ Scan parquet (44) @@ -107,9 +107,9 @@ Input [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct Input [8]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [8]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [8]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] @@ -144,9 +144,9 @@ Input [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] @@ -176,9 +176,9 @@ Input [9]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct Input [9]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, o_orderkey#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [9]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, o_orderkey#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [9]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, o_orderkey#X] @@ -213,9 +213,9 @@ Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_returnflag#X] Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] @@ -245,9 +245,9 @@ Input [11]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acc Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, l_extendedprice#X, l_discount#X] Arguments: false -(39) VeloxAppendBatches +(39) VeloxResizeBatches Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (40) ColumnarExchange Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, l_extendedprice#X, l_discount#X] @@ -282,9 +282,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] @@ -325,9 +325,9 @@ Input [9]: [c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] Arguments: false -(58) VeloxAppendBatches +(58) VeloxResizeBatches Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (59) ColumnarExchange Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/11.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/11.txt index 6c9eef6b1f1f..39187c1a1c26 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/11.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/11.txt @@ -6,13 +6,13 @@ AdaptiveSparkPlan (82) +- ^ InputIteratorTransformer (53) +- ShuffleQueryStage (51), Statistics(X) +- ColumnarExchange (50) - +- VeloxAppendBatches (49) + +- VeloxResizeBatches (49) +- ^ FilterExecTransformer (47) +- ^ RegularHashAggregateExecTransformer (46) +- ^ InputIteratorTransformer (45) +- ShuffleQueryStage (43), Statistics(X) +- ColumnarExchange (42) - +- VeloxAppendBatches (41) + +- VeloxResizeBatches (41) +- ^ ProjectExecTransformer (39) +- ^ FlushableHashAggregateExecTransformer (38) +- ^ ProjectExecTransformer (37) @@ -20,27 +20,27 @@ AdaptiveSparkPlan (82) :- ^ InputIteratorTransformer (26) : +- ShuffleQueryStage (24), Statistics(X) : +- ColumnarExchange (23) - : +- VeloxAppendBatches (22) + : +- VeloxResizeBatches (22) : +- ^ ProjectExecTransformer (20) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7), Statistics(X) : : +- ColumnarExchange (6) - : : +- VeloxAppendBatches (5) + : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ NoopFilter (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16), Statistics(X) : +- ColumnarExchange (15) - : +- VeloxAppendBatches (14) + : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ NoopFilter (11) : +- ^ Scan parquet (10) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33), Statistics(X) +- ColumnarExchange (32) - +- VeloxAppendBatches (31) + +- VeloxResizeBatches (31) +- ^ ProjectExecTransformer (29) +- ^ NoopFilter (28) +- ^ Scan parquet (27) @@ -91,9 +91,9 @@ Input [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Input [5]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [5]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [5]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] @@ -128,9 +128,9 @@ Input [2]: [s_suppkey#X, s_nationkey#X] Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -160,9 +160,9 @@ Input [6]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X, s_suppke Input [5]: [hash_partition_key#X, ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [5]: [hash_partition_key#X, ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [5]: [hash_partition_key#X, ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] @@ -197,9 +197,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [2]: [hash_partition_key#X, n_nationkey#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [2]: [hash_partition_key#X, n_nationkey#X] @@ -240,9 +240,9 @@ Input [3]: [ps_partkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] Arguments: false -(41) VeloxAppendBatches +(41) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (42) ColumnarExchange Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] @@ -273,9 +273,9 @@ Arguments: (isnotnull(value#X) AND (cast(value#X as decimal(38,6)) > Subquery su Input [2]: [ps_partkey#X, value#X] Arguments: false -(49) VeloxAppendBatches +(49) VeloxResizeBatches Input [2]: [ps_partkey#X, value#X] -Arguments: X +Arguments: X, X (50) ColumnarExchange Input [2]: [ps_partkey#X, value#X] @@ -439,13 +439,13 @@ AdaptiveSparkPlan (136) :- ^ InputIteratorTransformer (103) : +- ShuffleQueryStage (101), Statistics(X) : +- ColumnarExchange (100) - : +- VeloxAppendBatches (99) + : +- VeloxResizeBatches (99) : +- ^ ProjectExecTransformer (97) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (96) : :- ^ InputIteratorTransformer (91) : : +- ShuffleQueryStage (89), Statistics(X) : : +- ColumnarExchange (88) - : : +- VeloxAppendBatches (87) + : : +- VeloxResizeBatches (87) : : +- ^ ProjectExecTransformer (85) : : +- ^ NoopFilter (84) : : +- ^ Scan parquet (83) @@ -498,9 +498,9 @@ Input [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Input [4]: [hash_partition_key#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Arguments: false -(87) VeloxAppendBatches +(87) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] -Arguments: X +Arguments: X, X (88) ColumnarExchange Input [4]: [hash_partition_key#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] @@ -543,9 +543,9 @@ Input [5]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X, s_suppkey#X, s_nationk Input [4]: [hash_partition_key#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] Arguments: false -(99) VeloxAppendBatches +(99) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] -Arguments: X +Arguments: X, X (100) ColumnarExchange Input [4]: [hash_partition_key#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/12.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/12.txt index 8f963c49aba8..2ce16d0a2db5 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/12.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/12.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (55) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33), Statistics(X) +- ColumnarExchange (32) - +- VeloxAppendBatches (31) + +- VeloxResizeBatches (31) +- ^ RegularHashAggregateExecTransformer (29) +- ^ InputIteratorTransformer (28) +- ShuffleQueryStage (26), Statistics(X) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ ProjectExecTransformer (22) +- ^ FlushableHashAggregateExecTransformer (21) +- ^ ProjectExecTransformer (20) @@ -19,14 +19,14 @@ AdaptiveSparkPlan (55) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7), Statistics(X) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ NoopFilter (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ NoopFilter (11) +- ^ Scan parquet (10) @@ -68,9 +68,9 @@ Input [2]: [o_orderkey#X, o_orderpriority#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] @@ -105,9 +105,9 @@ Input [5]: [l_orderkey#X, l_shipdate#X, l_commitdate#X, l_receiptdate#X, l_shipm Input [3]: [hash_partition_key#X, l_orderkey#X, l_shipmode#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_shipmode#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_shipmode#X] @@ -148,9 +148,9 @@ Input [3]: [l_shipmode#X, sum#X, sum#X] Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] @@ -177,9 +177,9 @@ Results [3]: [l_shipmode#X, sum(CASE WHEN ((o_orderpriority#X = 1-URGENT) OR (o_ Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/13.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/13.txt index 9584f92628cd..8989431cc83c 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/13.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/13.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (58) +- ^ InputIteratorTransformer (37) +- ShuffleQueryStage (35), Statistics(X) +- ColumnarExchange (34) - +- VeloxAppendBatches (33) + +- VeloxResizeBatches (33) +- ^ RegularHashAggregateExecTransformer (31) +- ^ InputIteratorTransformer (30) +- ShuffleQueryStage (28), Statistics(X) +- ColumnarExchange (27) - +- VeloxAppendBatches (26) + +- VeloxResizeBatches (26) +- ^ ProjectExecTransformer (24) +- ^ FlushableHashAggregateExecTransformer (23) +- ^ ProjectExecTransformer (22) @@ -22,13 +22,13 @@ AdaptiveSparkPlan (58) :- ^ InputIteratorTransformer (8) : +- ShuffleQueryStage (6), Statistics(X) : +- ColumnarExchange (5) - : +- VeloxAppendBatches (4) + : +- VeloxResizeBatches (4) : +- ^ ProjectExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (17) +- ShuffleQueryStage (15), Statistics(X) +- ColumnarExchange (14) - +- VeloxAppendBatches (13) + +- VeloxResizeBatches (13) +- ^ ProjectExecTransformer (11) +- ^ NoopFilter (10) +- ^ Scan parquet (9) @@ -66,9 +66,9 @@ Input [1]: [c_custkey#X] Input [2]: [hash_partition_key#X, c_custkey#X] Arguments: false -(4) VeloxAppendBatches +(4) VeloxResizeBatches Input [2]: [hash_partition_key#X, c_custkey#X] -Arguments: X +Arguments: X, X (5) ColumnarExchange Input [2]: [hash_partition_key#X, c_custkey#X] @@ -103,9 +103,9 @@ Input [3]: [o_orderkey#X, o_custkey#X, o_comment#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] Arguments: false -(13) VeloxAppendBatches +(13) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] -Arguments: X +Arguments: X, X (14) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] @@ -164,9 +164,9 @@ Input [2]: [c_count#X, count#X] Input [3]: [hash_partition_key#X, c_count#X, count#X] Arguments: false -(26) VeloxAppendBatches +(26) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_count#X, count#X] -Arguments: X +Arguments: X, X (27) ColumnarExchange Input [3]: [hash_partition_key#X, c_count#X, count#X] @@ -193,9 +193,9 @@ Results [2]: [c_count#X, count(1)#X AS custdist#X] Input [2]: [c_count#X, custdist#X] Arguments: false -(33) VeloxAppendBatches +(33) VeloxResizeBatches Input [2]: [c_count#X, custdist#X] -Arguments: X +Arguments: X, X (34) ColumnarExchange Input [2]: [c_count#X, custdist#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/14.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/14.txt index 175a0e5a97cc..f9e3f564c782 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/14.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/14.txt @@ -10,14 +10,14 @@ AdaptiveSparkPlan (39) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7), Statistics(X) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ NoopFilter (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ NoopFilter (11) +- ^ Scan parquet (10) @@ -56,9 +56,9 @@ Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Input [4]: [hash_partition_key#X, l_partkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_partkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [4]: [hash_partition_key#X, l_partkey#X, l_extendedprice#X, l_discount#X] @@ -93,9 +93,9 @@ Input [2]: [p_partkey#X, p_type#X] Input [3]: [hash_partition_key#X, p_partkey#X, p_type#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, p_partkey#X, p_type#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, p_partkey#X, p_type#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/15.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/15.txt index 130d9036b4a2..ca94a960e811 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/15.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/15.txt @@ -5,13 +5,13 @@ AdaptiveSparkPlan (47) +- AQEShuffleRead (29) +- ShuffleQueryStage (28), Statistics(X) +- ColumnarExchange (27) - +- VeloxAppendBatches (26) + +- VeloxResizeBatches (26) +- ^ ProjectExecTransformer (24) +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (23) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7), Statistics(X) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ NoopFilter (2) : +- ^ Scan parquet (1) @@ -20,7 +20,7 @@ AdaptiveSparkPlan (47) +- ^ InputIteratorTransformer (20) +- ShuffleQueryStage (18), Statistics(X) +- ColumnarExchange (17) - +- VeloxAppendBatches (16) + +- VeloxResizeBatches (16) +- ^ ProjectExecTransformer (14) +- ^ FlushableHashAggregateExecTransformer (13) +- ^ ProjectExecTransformer (12) @@ -64,9 +64,9 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_phone#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_phone#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_phone#X] @@ -112,9 +112,9 @@ Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] Arguments: false -(16) VeloxAppendBatches +(16) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (17) ColumnarExchange Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] @@ -155,9 +155,9 @@ Input [6]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, supplier_no#X, total_ Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] Arguments: false -(26) VeloxAppendBatches +(26) VeloxResizeBatches Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] -Arguments: X +Arguments: X, X (27) ColumnarExchange Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] @@ -269,7 +269,7 @@ AdaptiveSparkPlan (73) +- ^ InputIteratorTransformer (58) +- ShuffleQueryStage (56), Statistics(X) +- ColumnarExchange (55) - +- VeloxAppendBatches (54) + +- VeloxResizeBatches (54) +- ^ ProjectExecTransformer (52) +- ^ FlushableHashAggregateExecTransformer (51) +- ^ ProjectExecTransformer (50) @@ -316,9 +316,9 @@ Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] Arguments: false -(54) VeloxAppendBatches +(54) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (55) ColumnarExchange Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/16.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/16.txt index 32a24beb94b9..5813bbd1af9b 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/16.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/16.txt @@ -6,19 +6,19 @@ AdaptiveSparkPlan (71) +- ^ InputIteratorTransformer (44) +- ShuffleQueryStage (42), Statistics(X) +- ColumnarExchange (41) - +- VeloxAppendBatches (40) + +- VeloxResizeBatches (40) +- ^ RegularHashAggregateExecTransformer (38) +- ^ InputIteratorTransformer (37) +- ShuffleQueryStage (35), Statistics(X) +- ColumnarExchange (34) - +- VeloxAppendBatches (33) + +- VeloxResizeBatches (33) +- ^ ProjectExecTransformer (31) +- ^ FlushableHashAggregateExecTransformer (30) +- ^ RegularHashAggregateExecTransformer (29) +- ^ InputIteratorTransformer (28) +- ShuffleQueryStage (26), Statistics(X) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ ProjectExecTransformer (22) +- ^ FlushableHashAggregateExecTransformer (21) +- ^ ProjectExecTransformer (20) @@ -26,14 +26,14 @@ AdaptiveSparkPlan (71) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7), Statistics(X) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ NoopFilter (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ NoopFilter (11) +- ^ Scan parquet (10) @@ -82,9 +82,9 @@ Input [2]: [ps_partkey#X, ps_suppkey#X] Input [3]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X] @@ -119,9 +119,9 @@ Input [4]: [p_partkey#X, p_brand#X, p_type#X, p_size#X] Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_type#X, p_size#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_type#X, p_size#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_type#X, p_size#X] @@ -162,9 +162,9 @@ Input [4]: [p_brand#X, p_type#X, p_size#X, ps_suppkey#X] Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] @@ -202,9 +202,9 @@ Input [4]: [p_brand#X, p_type#X, p_size#X, count#X] Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] Arguments: false -(33) VeloxAppendBatches +(33) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] -Arguments: X +Arguments: X, X (34) ColumnarExchange Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] @@ -231,9 +231,9 @@ Results [4]: [p_brand#X, p_type#X, p_size#X, count(ps_suppkey#X)#X AS supplier_c Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] Arguments: false -(40) VeloxAppendBatches +(40) VeloxResizeBatches Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] -Arguments: X +Arguments: X, X (41) ColumnarExchange Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/17.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/17.txt index 59baa2d7a08c..8b4e121891f2 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/17.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/17.txt @@ -12,14 +12,14 @@ AdaptiveSparkPlan (63) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7), Statistics(X) : : +- ColumnarExchange (6) - : : +- VeloxAppendBatches (5) + : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ NoopFilter (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16), Statistics(X) : +- ColumnarExchange (15) - : +- VeloxAppendBatches (14) + : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ NoopFilter (11) : +- ^ Scan parquet (10) @@ -29,7 +29,7 @@ AdaptiveSparkPlan (63) +- ^ InputIteratorTransformer (30) +- ShuffleQueryStage (28), Statistics(X) +- ColumnarExchange (27) - +- VeloxAppendBatches (26) + +- VeloxResizeBatches (26) +- ^ ProjectExecTransformer (24) +- ^ FlushableHashAggregateExecTransformer (23) +- ^ NoopFilter (22) @@ -78,9 +78,9 @@ Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Input [4]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [4]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X] @@ -115,9 +115,9 @@ Input [3]: [p_partkey#X, p_brand#X, p_container#X] Input [2]: [hash_partition_key#X, p_partkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [2]: [hash_partition_key#X, p_partkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [2]: [hash_partition_key#X, p_partkey#X] @@ -169,9 +169,9 @@ Input [3]: [l_partkey#X, sum#X, count#X] Input [4]: [hash_partition_key#X, l_partkey#X, sum#X, count#X] Arguments: false -(26) VeloxAppendBatches +(26) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_partkey#X, sum#X, count#X] -Arguments: X +Arguments: X, X (27) ColumnarExchange Input [4]: [hash_partition_key#X, l_partkey#X, sum#X, count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/18.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/18.txt index c78e265e54d9..3f7c919c6212 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/18.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/18.txt @@ -10,26 +10,26 @@ AdaptiveSparkPlan (110) :- ^ InputIteratorTransformer (46) : +- ShuffleQueryStage (44), Statistics(X) : +- ColumnarExchange (43) - : +- VeloxAppendBatches (42) + : +- VeloxResizeBatches (42) : +- ^ ProjectExecTransformer (40) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (39) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7), Statistics(X) : : +- ColumnarExchange (6) - : : +- VeloxAppendBatches (5) + : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ NoopFilter (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (38) : +- ShuffleQueryStage (36), Statistics(X) : +- ColumnarExchange (35) - : +- VeloxAppendBatches (34) + : +- VeloxResizeBatches (34) : +- ^ ProjectExecTransformer (32) : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (31) : :- ^ InputIteratorTransformer (18) : : +- ShuffleQueryStage (16), Statistics(X) : : +- ColumnarExchange (15) - : : +- VeloxAppendBatches (14) + : : +- VeloxResizeBatches (14) : : +- ^ ProjectExecTransformer (12) : : +- ^ NoopFilter (11) : : +- ^ Scan parquet (10) @@ -39,7 +39,7 @@ AdaptiveSparkPlan (110) : +- ^ InputIteratorTransformer (27) : +- ShuffleQueryStage (25), Statistics(X) : +- ColumnarExchange (24) - : +- VeloxAppendBatches (23) + : +- VeloxResizeBatches (23) : +- ^ ProjectExecTransformer (21) : +- ^ FlushableHashAggregateExecTransformer (20) : +- ^ Scan parquet (19) @@ -47,7 +47,7 @@ AdaptiveSparkPlan (110) :- ^ InputIteratorTransformer (55) : +- ShuffleQueryStage (53), Statistics(X) : +- ColumnarExchange (52) - : +- VeloxAppendBatches (51) + : +- VeloxResizeBatches (51) : +- ^ ProjectExecTransformer (49) : +- ^ NoopFilter (48) : +- ^ Scan parquet (47) @@ -118,9 +118,9 @@ Input [2]: [c_custkey#X, c_name#X] Input [3]: [hash_partition_key#X, c_custkey#X, c_name#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_custkey#X, c_name#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, c_custkey#X, c_name#X] @@ -155,9 +155,9 @@ Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] @@ -194,9 +194,9 @@ Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] Arguments: false -(23) VeloxAppendBatches +(23) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (24) ColumnarExchange Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] @@ -241,9 +241,9 @@ Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: false -(34) VeloxAppendBatches +(34) VeloxResizeBatches Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] -Arguments: X +Arguments: X, X (35) ColumnarExchange Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] @@ -273,9 +273,9 @@ Input [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_ Input [6]: [hash_partition_key#X, c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] Arguments: false -(42) VeloxAppendBatches +(42) VeloxResizeBatches Input [6]: [hash_partition_key#X, c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] -Arguments: X +Arguments: X, X (43) ColumnarExchange Input [6]: [hash_partition_key#X, c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] @@ -310,9 +310,9 @@ Input [2]: [l_orderkey#X, l_quantity#X] Input [3]: [hash_partition_key#X, l_orderkey#X, l_quantity#X] Arguments: false -(51) VeloxAppendBatches +(51) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_quantity#X] -Arguments: X +Arguments: X, X (52) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_quantity#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/19.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/19.txt index a9c629524fb7..f29554305a6d 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/19.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/19.txt @@ -9,14 +9,14 @@ AdaptiveSparkPlan (38) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7), Statistics(X) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ NoopFilter (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ NoopFilter (11) +- ^ Scan parquet (10) @@ -55,9 +55,9 @@ Input [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipin Input [5]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [5]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] @@ -92,9 +92,9 @@ Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_size#X, p_container#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_size#X, p_container#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_size#X, p_container#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/20.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/20.txt index 8e929ff7b296..227d18bd0f21 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/20.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/20.txt @@ -5,52 +5,52 @@ AdaptiveSparkPlan (143) +- AQEShuffleRead (92) +- ShuffleQueryStage (91), Statistics(X) +- ColumnarExchange (90) - +- VeloxAppendBatches (89) + +- VeloxResizeBatches (89) +- ^ ProjectExecTransformer (87) +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (86) :- ^ InputIteratorTransformer (76) : +- ShuffleQueryStage (74), Statistics(X) : +- ColumnarExchange (73) - : +- VeloxAppendBatches (72) + : +- VeloxResizeBatches (72) : +- ^ ProjectExecTransformer (70) : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (69) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7), Statistics(X) : : +- ColumnarExchange (6) - : : +- VeloxAppendBatches (5) + : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ NoopFilter (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (68) : +- ShuffleQueryStage (66), Statistics(X) : +- ColumnarExchange (65) - : +- VeloxAppendBatches (64) + : +- VeloxResizeBatches (64) : +- ^ ProjectExecTransformer (62) : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (61) : :- ^ InputIteratorTransformer (35) : : +- ShuffleQueryStage (33), Statistics(X) : : +- ColumnarExchange (32) - : : +- VeloxAppendBatches (31) + : : +- VeloxResizeBatches (31) : : +- ^ ProjectExecTransformer (29) : : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (28) : : :- ^ InputIteratorTransformer (18) : : : +- ShuffleQueryStage (16), Statistics(X) : : : +- ColumnarExchange (15) - : : : +- VeloxAppendBatches (14) + : : : +- VeloxResizeBatches (14) : : : +- ^ ProjectExecTransformer (12) : : : +- ^ NoopFilter (11) : : : +- ^ Scan parquet (10) : : +- ^ InputIteratorTransformer (27) : : +- ShuffleQueryStage (25), Statistics(X) : : +- ColumnarExchange (24) - : : +- VeloxAppendBatches (23) + : : +- VeloxResizeBatches (23) : : +- ^ ProjectExecTransformer (21) : : +- ^ NoopFilter (20) : : +- ^ Scan parquet (19) : +- ^ InputIteratorTransformer (60) : +- ShuffleQueryStage (58), Statistics(X) : +- ColumnarExchange (57) - : +- VeloxAppendBatches (56) + : +- VeloxResizeBatches (56) : +- ^ ProjectExecTransformer (54) : +- ^ FilterExecTransformer (53) : +- ^ ProjectExecTransformer (52) @@ -60,7 +60,7 @@ AdaptiveSparkPlan (143) : :- ^ InputIteratorTransformer (44) : : +- ShuffleQueryStage (42), Statistics(X) : : +- ColumnarExchange (41) - : : +- VeloxAppendBatches (40) + : : +- VeloxResizeBatches (40) : : +- ^ ProjectExecTransformer (38) : : +- ^ NoopFilter (37) : : +- ^ Scan parquet (36) @@ -70,7 +70,7 @@ AdaptiveSparkPlan (143) +- ^ InputIteratorTransformer (85) +- ShuffleQueryStage (83), Statistics(X) +- ColumnarExchange (82) - +- VeloxAppendBatches (81) + +- VeloxResizeBatches (81) +- ^ ProjectExecTransformer (79) +- ^ NoopFilter (78) +- ^ Scan parquet (77) @@ -145,9 +145,9 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] @@ -182,9 +182,9 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] @@ -219,9 +219,9 @@ Input [2]: [p_partkey#X, p_name#X] Input [2]: [hash_partition_key#X, p_partkey#X] Arguments: false -(23) VeloxAppendBatches +(23) VeloxResizeBatches Input [2]: [hash_partition_key#X, p_partkey#X] -Arguments: X +Arguments: X, X (24) ColumnarExchange Input [2]: [hash_partition_key#X, p_partkey#X] @@ -251,9 +251,9 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] @@ -288,9 +288,9 @@ Input [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] Input [4]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, l_quantity#X] Arguments: false -(40) VeloxAppendBatches +(40) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, l_quantity#X] -Arguments: X +Arguments: X, X (41) ColumnarExchange Input [4]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, l_quantity#X] @@ -355,9 +355,9 @@ Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: false -(56) VeloxAppendBatches +(56) VeloxResizeBatches Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -Arguments: X +Arguments: X, X (57) ColumnarExchange Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] @@ -387,9 +387,9 @@ Input [6]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, (0.5 * sum(l_quantity))#X Input [2]: [hash_partition_key#X, ps_suppkey#X] Arguments: false -(64) VeloxAppendBatches +(64) VeloxResizeBatches Input [2]: [hash_partition_key#X, ps_suppkey#X] -Arguments: X +Arguments: X, X (65) ColumnarExchange Input [2]: [hash_partition_key#X, ps_suppkey#X] @@ -419,9 +419,9 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] Arguments: false -(72) VeloxAppendBatches +(72) VeloxResizeBatches Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] -Arguments: X +Arguments: X, X (73) ColumnarExchange Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] @@ -456,9 +456,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: false -(81) VeloxAppendBatches +(81) VeloxResizeBatches Input [2]: [hash_partition_key#X, n_nationkey#X] -Arguments: X +Arguments: X, X (82) ColumnarExchange Input [2]: [hash_partition_key#X, n_nationkey#X] @@ -488,9 +488,9 @@ Input [4]: [s_name#X, s_address#X, s_nationkey#X, n_nationkey#X] Input [2]: [s_name#X, s_address#X] Arguments: false -(89) VeloxAppendBatches +(89) VeloxResizeBatches Input [2]: [s_name#X, s_address#X] -Arguments: X +Arguments: X, X (90) ColumnarExchange Input [2]: [s_name#X, s_address#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/21.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/21.txt index 279f4f096692..1ffbdd45fef9 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/21.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/21.txt @@ -6,7 +6,7 @@ AdaptiveSparkPlan (137) +- ^ InputIteratorTransformer (88) +- ShuffleQueryStage (86), Statistics(X) +- ColumnarExchange (85) - +- VeloxAppendBatches (84) + +- VeloxResizeBatches (84) +- ^ ProjectExecTransformer (82) +- ^ FlushableHashAggregateExecTransformer (81) +- ^ ProjectExecTransformer (80) @@ -14,60 +14,60 @@ AdaptiveSparkPlan (137) :- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67), Statistics(X) : +- ColumnarExchange (66) - : +- VeloxAppendBatches (65) + : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (62) : :- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50), Statistics(X) : : +- ColumnarExchange (49) - : : +- VeloxAppendBatches (48) + : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (45) : : :- ^ InputIteratorTransformer (9) : : : +- ShuffleQueryStage (7), Statistics(X) : : : +- ColumnarExchange (6) - : : : +- VeloxAppendBatches (5) + : : : +- VeloxResizeBatches (5) : : : +- ^ ProjectExecTransformer (3) : : : +- ^ NoopFilter (2) : : : +- ^ Scan parquet (1) : : +- ^ InputIteratorTransformer (44) : : +- ShuffleQueryStage (42), Statistics(X) : : +- ColumnarExchange (41) - : : +- VeloxAppendBatches (40) + : : +- VeloxResizeBatches (40) : : +- ^ ProjectExecTransformer (38) : : +- ^ ShuffledHashJoinExecTransformer LeftAnti BuildRight (37) : : :- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (27) : : : :- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : +- ColumnarExchange (15) - : : : : +- VeloxAppendBatches (14) + : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ NoopFilter (11) : : : : +- ^ Scan parquet (10) : : : +- ^ InputIteratorTransformer (26) : : : +- ShuffleQueryStage (24), Statistics(X) : : : +- ColumnarExchange (23) - : : : +- VeloxAppendBatches (22) + : : : +- VeloxResizeBatches (22) : : : +- ^ ProjectExecTransformer (20) : : : +- ^ Scan parquet (19) : : +- ^ InputIteratorTransformer (36) : : +- ShuffleQueryStage (34), Statistics(X) : : +- ColumnarExchange (33) - : : +- VeloxAppendBatches (32) + : : +- VeloxResizeBatches (32) : : +- ^ ProjectExecTransformer (30) : : +- ^ NoopFilter (29) : : +- ^ Scan parquet (28) : +- ^ InputIteratorTransformer (61) : +- ShuffleQueryStage (59), Statistics(X) : +- ColumnarExchange (58) - : +- VeloxAppendBatches (57) + : +- VeloxResizeBatches (57) : +- ^ ProjectExecTransformer (55) : +- ^ NoopFilter (54) : +- ^ Scan parquet (53) +- ^ InputIteratorTransformer (78) +- ShuffleQueryStage (76), Statistics(X) +- ColumnarExchange (75) - +- VeloxAppendBatches (74) + +- VeloxResizeBatches (74) +- ^ ProjectExecTransformer (72) +- ^ NoopFilter (71) +- ^ Scan parquet (70) @@ -138,9 +138,9 @@ Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X] Input [4]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_nationkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [4]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_nationkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [4]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_nationkey#X] @@ -175,9 +175,9 @@ Input [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] @@ -207,9 +207,9 @@ Input [2]: [l_orderkey#X, l_suppkey#X] Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] @@ -250,9 +250,9 @@ Input [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] Arguments: false -(32) VeloxAppendBatches +(32) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] -Arguments: X +Arguments: X, X (33) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] @@ -282,9 +282,9 @@ Input [2]: [l_orderkey#X, l_suppkey#X] Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] Arguments: false -(40) VeloxAppendBatches +(40) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] -Arguments: X +Arguments: X, X (41) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] @@ -314,9 +314,9 @@ Input [5]: [s_suppkey#X, s_name#X, s_nationkey#X, l_orderkey#X, l_suppkey#X] Input [4]: [hash_partition_key#X, s_name#X, s_nationkey#X, l_orderkey#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [4]: [hash_partition_key#X, s_name#X, s_nationkey#X, l_orderkey#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [4]: [hash_partition_key#X, s_name#X, s_nationkey#X, l_orderkey#X] @@ -351,9 +351,9 @@ Input [2]: [o_orderkey#X, o_orderstatus#X] Input [2]: [hash_partition_key#X, o_orderkey#X] Arguments: false -(57) VeloxAppendBatches +(57) VeloxResizeBatches Input [2]: [hash_partition_key#X, o_orderkey#X] -Arguments: X +Arguments: X, X (58) ColumnarExchange Input [2]: [hash_partition_key#X, o_orderkey#X] @@ -383,9 +383,9 @@ Input [4]: [s_name#X, s_nationkey#X, l_orderkey#X, o_orderkey#X] Input [3]: [hash_partition_key#X, s_name#X, s_nationkey#X] Arguments: false -(65) VeloxAppendBatches +(65) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_name#X, s_nationkey#X] -Arguments: X +Arguments: X, X (66) ColumnarExchange Input [3]: [hash_partition_key#X, s_name#X, s_nationkey#X] @@ -420,9 +420,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: false -(74) VeloxAppendBatches +(74) VeloxResizeBatches Input [2]: [hash_partition_key#X, n_nationkey#X] -Arguments: X +Arguments: X, X (75) ColumnarExchange Input [2]: [hash_partition_key#X, n_nationkey#X] @@ -463,9 +463,9 @@ Input [2]: [s_name#X, count#X] Input [3]: [hash_partition_key#X, s_name#X, count#X] Arguments: false -(84) VeloxAppendBatches +(84) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_name#X, count#X] -Arguments: X +Arguments: X, X (85) ColumnarExchange Input [3]: [hash_partition_key#X, s_name#X, count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/22.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/22.txt index 2b93055014bd..fd4141aec0af 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/22.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/22.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (52) +- ^ InputIteratorTransformer (34) +- ShuffleQueryStage (32), Statistics(X) +- ColumnarExchange (31) - +- VeloxAppendBatches (30) + +- VeloxResizeBatches (30) +- ^ RegularHashAggregateExecTransformer (28) +- ^ InputIteratorTransformer (27) +- ShuffleQueryStage (25), Statistics(X) +- ColumnarExchange (24) - +- VeloxAppendBatches (23) + +- VeloxResizeBatches (23) +- ^ ProjectExecTransformer (21) +- ^ FlushableHashAggregateExecTransformer (20) +- ^ ProjectExecTransformer (19) @@ -19,14 +19,14 @@ AdaptiveSparkPlan (52) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7), Statistics(X) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ NoopFilter (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (17) +- ShuffleQueryStage (15), Statistics(X) +- ColumnarExchange (14) - +- VeloxAppendBatches (13) + +- VeloxResizeBatches (13) +- ^ ProjectExecTransformer (11) +- ^ Scan parquet (10) +- == Initial Plan == @@ -65,9 +65,9 @@ Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X] Input [4]: [hash_partition_key#X, c_custkey#X, c_phone#X, c_acctbal#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [4]: [hash_partition_key#X, c_custkey#X, c_phone#X, c_acctbal#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [4]: [hash_partition_key#X, c_custkey#X, c_phone#X, c_acctbal#X] @@ -97,9 +97,9 @@ Input [1]: [o_custkey#X] Input [2]: [hash_partition_key#X, o_custkey#X] Arguments: false -(13) VeloxAppendBatches +(13) VeloxResizeBatches Input [2]: [hash_partition_key#X, o_custkey#X] -Arguments: X +Arguments: X, X (14) ColumnarExchange Input [2]: [hash_partition_key#X, o_custkey#X] @@ -140,9 +140,9 @@ Input [4]: [cntrycode#X, count#X, sum#X, isEmpty#X] Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] Arguments: false -(23) VeloxAppendBatches +(23) VeloxResizeBatches Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (24) ColumnarExchange Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] @@ -169,9 +169,9 @@ Results [3]: [cntrycode#X, count(1)#X AS numcust#X, sum(c_acctbal#X)#X AS totacc Input [3]: [cntrycode#X, numcust#X, totacctbal#X] Arguments: false -(30) VeloxAppendBatches +(30) VeloxResizeBatches Input [3]: [cntrycode#X, numcust#X, totacctbal#X] -Arguments: X +Arguments: X, X (31) ColumnarExchange Input [3]: [cntrycode#X, numcust#X, totacctbal#X] @@ -281,7 +281,7 @@ AdaptiveSparkPlan (72) +- ^ InputIteratorTransformer (62) +- ShuffleQueryStage (60), Statistics(X) +- ColumnarExchange (59) - +- VeloxAppendBatches (58) + +- VeloxResizeBatches (58) +- ^ FlushableHashAggregateExecTransformer (56) +- ^ ProjectExecTransformer (55) +- ^ NoopFilter (54) @@ -321,9 +321,9 @@ Results [2]: [sum#X, count#X] Input [2]: [sum#X, count#X] Arguments: false -(58) VeloxAppendBatches +(58) VeloxResizeBatches Input [2]: [sum#X, count#X] -Arguments: X +Arguments: X, X (59) ColumnarExchange Input [2]: [sum#X, count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/3.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/3.txt index aa679861da7c..c7aed3993470 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/3.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/3.txt @@ -11,27 +11,27 @@ AdaptiveSparkPlan (67) :- ^ InputIteratorTransformer (26) : +- ShuffleQueryStage (24), Statistics(X) : +- ColumnarExchange (23) - : +- VeloxAppendBatches (22) + : +- VeloxResizeBatches (22) : +- ^ ProjectExecTransformer (20) : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7), Statistics(X) : : +- ColumnarExchange (6) - : : +- VeloxAppendBatches (5) + : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ NoopFilter (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16), Statistics(X) : +- ColumnarExchange (15) - : +- VeloxAppendBatches (14) + : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ NoopFilter (11) : +- ^ Scan parquet (10) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33), Statistics(X) +- ColumnarExchange (32) - +- VeloxAppendBatches (31) + +- VeloxResizeBatches (31) +- ^ ProjectExecTransformer (29) +- ^ NoopFilter (28) +- ^ Scan parquet (27) @@ -80,9 +80,9 @@ Input [2]: [c_custkey#X, c_mktsegment#X] Input [2]: [hash_partition_key#X, c_custkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [2]: [hash_partition_key#X, c_custkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [2]: [hash_partition_key#X, c_custkey#X] @@ -117,9 +117,9 @@ Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] @@ -149,9 +149,9 @@ Input [5]: [c_custkey#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriorit Input [4]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X, o_shippriority#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [4]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X, o_shippriority#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [4]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X, o_shippriority#X] @@ -186,9 +186,9 @@ Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/4.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/4.txt index 02c494288f95..72999dce97b3 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/4.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/4.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (56) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33), Statistics(X) +- ColumnarExchange (32) - +- VeloxAppendBatches (31) + +- VeloxResizeBatches (31) +- ^ RegularHashAggregateExecTransformer (29) +- ^ InputIteratorTransformer (28) +- ShuffleQueryStage (26), Statistics(X) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ ProjectExecTransformer (22) +- ^ FlushableHashAggregateExecTransformer (21) +- ^ ProjectExecTransformer (20) @@ -19,14 +19,14 @@ AdaptiveSparkPlan (56) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7), Statistics(X) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ NoopFilter (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ NoopFilter (11) +- ^ Scan parquet (10) @@ -69,9 +69,9 @@ Input [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] @@ -106,9 +106,9 @@ Input [3]: [l_orderkey#X, l_commitdate#X, l_receiptdate#X] Input [2]: [hash_partition_key#X, l_orderkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [2]: [hash_partition_key#X, l_orderkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [2]: [hash_partition_key#X, l_orderkey#X] @@ -149,9 +149,9 @@ Input [2]: [o_orderpriority#X, count#X] Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] @@ -178,9 +178,9 @@ Results [2]: [o_orderpriority#X, count(1)#X AS order_count#X] Input [2]: [o_orderpriority#X, order_count#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [2]: [o_orderpriority#X, order_count#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [2]: [o_orderpriority#X, order_count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/5.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/5.txt index 67150984ab61..388d18a0314d 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/5.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/5.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (156) +- ^ InputIteratorTransformer (103) +- ShuffleQueryStage (101), Statistics(X) +- ColumnarExchange (100) - +- VeloxAppendBatches (99) + +- VeloxResizeBatches (99) +- ^ RegularHashAggregateExecTransformer (97) +- ^ InputIteratorTransformer (96) +- ShuffleQueryStage (94), Statistics(X) +- ColumnarExchange (93) - +- VeloxAppendBatches (92) + +- VeloxResizeBatches (92) +- ^ ProjectExecTransformer (90) +- ^ FlushableHashAggregateExecTransformer (89) +- ^ ProjectExecTransformer (88) @@ -19,66 +19,66 @@ AdaptiveSparkPlan (156) :- ^ InputIteratorTransformer (77) : +- ShuffleQueryStage (75), Statistics(X) : +- ColumnarExchange (74) - : +- VeloxAppendBatches (73) + : +- VeloxResizeBatches (73) : +- ^ ProjectExecTransformer (71) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (70) : :- ^ InputIteratorTransformer (60) : : +- ShuffleQueryStage (58), Statistics(X) : : +- ColumnarExchange (57) - : : +- VeloxAppendBatches (56) + : : +- VeloxResizeBatches (56) : : +- ^ ProjectExecTransformer (54) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (53) : : :- ^ InputIteratorTransformer (43) : : : +- ShuffleQueryStage (41), Statistics(X) : : : +- ColumnarExchange (40) - : : : +- VeloxAppendBatches (39) + : : : +- VeloxResizeBatches (39) : : : +- ^ ProjectExecTransformer (37) : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (36) : : : :- ^ InputIteratorTransformer (26) : : : : +- ShuffleQueryStage (24), Statistics(X) : : : : +- ColumnarExchange (23) - : : : : +- VeloxAppendBatches (22) + : : : : +- VeloxResizeBatches (22) : : : : +- ^ ProjectExecTransformer (20) : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) : : : : :- ^ InputIteratorTransformer (9) : : : : : +- ShuffleQueryStage (7), Statistics(X) : : : : : +- ColumnarExchange (6) - : : : : : +- VeloxAppendBatches (5) + : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ NoopFilter (2) : : : : : +- ^ Scan parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : +- ColumnarExchange (15) - : : : : +- VeloxAppendBatches (14) + : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ NoopFilter (11) : : : : +- ^ Scan parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33), Statistics(X) : : : +- ColumnarExchange (32) - : : : +- VeloxAppendBatches (31) + : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ NoopFilter (28) : : : +- ^ Scan parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50), Statistics(X) : : +- ColumnarExchange (49) - : : +- VeloxAppendBatches (48) + : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ NoopFilter (45) : : +- ^ Scan parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67), Statistics(X) : +- ColumnarExchange (66) - : +- VeloxAppendBatches (65) + : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ NoopFilter (62) : +- ^ Scan parquet (61) +- ^ InputIteratorTransformer (86) +- ShuffleQueryStage (84), Statistics(X) +- ColumnarExchange (83) - +- VeloxAppendBatches (82) + +- VeloxResizeBatches (82) +- ^ ProjectExecTransformer (80) +- ^ NoopFilter (79) +- ^ Scan parquet (78) @@ -153,9 +153,9 @@ Input [2]: [c_custkey#X, c_nationkey#X] Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] @@ -190,9 +190,9 @@ Input [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] @@ -222,9 +222,9 @@ Input [4]: [c_custkey#X, c_nationkey#X, o_orderkey#X, o_custkey#X] Input [3]: [hash_partition_key#X, c_nationkey#X, o_orderkey#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_nationkey#X, o_orderkey#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [3]: [hash_partition_key#X, c_nationkey#X, o_orderkey#X] @@ -259,9 +259,9 @@ Input [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] @@ -291,9 +291,9 @@ Input [6]: [c_nationkey#X, o_orderkey#X, l_orderkey#X, l_suppkey#X, l_extendedpr Input [5]: [hash_partition_key#X, c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(39) VeloxAppendBatches +(39) VeloxResizeBatches Input [5]: [hash_partition_key#X, c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (40) ColumnarExchange Input [5]: [hash_partition_key#X, c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] @@ -328,9 +328,9 @@ Input [2]: [s_suppkey#X, s_nationkey#X] Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -360,9 +360,9 @@ Input [6]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, s_suppk Input [4]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X] Arguments: false -(56) VeloxAppendBatches +(56) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X] -Arguments: X +Arguments: X, X (57) ColumnarExchange Input [4]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X] @@ -397,9 +397,9 @@ Input [3]: [n_nationkey#X, n_name#X, n_regionkey#X] Input [4]: [hash_partition_key#X, n_nationkey#X, n_name#X, n_regionkey#X] Arguments: false -(65) VeloxAppendBatches +(65) VeloxResizeBatches Input [4]: [hash_partition_key#X, n_nationkey#X, n_name#X, n_regionkey#X] -Arguments: X +Arguments: X, X (66) ColumnarExchange Input [4]: [hash_partition_key#X, n_nationkey#X, n_name#X, n_regionkey#X] @@ -429,9 +429,9 @@ Input [6]: [l_extendedprice#X, l_discount#X, s_nationkey#X, n_nationkey#X, n_nam Input [5]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] Arguments: false -(73) VeloxAppendBatches +(73) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] -Arguments: X +Arguments: X, X (74) ColumnarExchange Input [5]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] @@ -466,9 +466,9 @@ Input [2]: [r_regionkey#X, r_name#X] Input [2]: [hash_partition_key#X, r_regionkey#X] Arguments: false -(82) VeloxAppendBatches +(82) VeloxResizeBatches Input [2]: [hash_partition_key#X, r_regionkey#X] -Arguments: X +Arguments: X, X (83) ColumnarExchange Input [2]: [hash_partition_key#X, r_regionkey#X] @@ -509,9 +509,9 @@ Input [3]: [n_name#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] Arguments: false -(92) VeloxAppendBatches +(92) VeloxResizeBatches Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (93) ColumnarExchange Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] @@ -538,9 +538,9 @@ Results [2]: [n_name#X, sum((l_extendedprice#X * (1 - l_discount#X)))#X AS reven Input [2]: [n_name#X, revenue#X] Arguments: false -(99) VeloxAppendBatches +(99) VeloxResizeBatches Input [2]: [n_name#X, revenue#X] -Arguments: X +Arguments: X, X (100) ColumnarExchange Input [2]: [n_name#X, revenue#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/6.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/6.txt index 64624c791f72..89cf24d87474 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/6.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/6.txt @@ -6,7 +6,7 @@ AdaptiveSparkPlan (20) +- ^ InputIteratorTransformer (10) +- ShuffleQueryStage (8), Statistics(X) +- ColumnarExchange (7) - +- VeloxAppendBatches (6) + +- VeloxResizeBatches (6) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ NoopFilter (2) @@ -46,9 +46,9 @@ Results [2]: [sum#X, isEmpty#X] Input [2]: [sum#X, isEmpty#X] Arguments: false -(6) VeloxAppendBatches +(6) VeloxResizeBatches Input [2]: [sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (7) ColumnarExchange Input [2]: [sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/7.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/7.txt index 65dfab993c3c..6284f06e5d2c 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/7.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/7.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (149) +- ^ InputIteratorTransformer (98) +- ShuffleQueryStage (96), Statistics(X) +- ColumnarExchange (95) - +- VeloxAppendBatches (94) + +- VeloxResizeBatches (94) +- ^ RegularHashAggregateExecTransformer (92) +- ^ InputIteratorTransformer (91) +- ShuffleQueryStage (89), Statistics(X) +- ColumnarExchange (88) - +- VeloxAppendBatches (87) + +- VeloxResizeBatches (87) +- ^ ProjectExecTransformer (85) +- ^ FlushableHashAggregateExecTransformer (84) +- ^ ProjectExecTransformer (83) @@ -19,59 +19,59 @@ AdaptiveSparkPlan (149) :- ^ InputIteratorTransformer (77) : +- ShuffleQueryStage (75), Statistics(X) : +- ColumnarExchange (74) - : +- VeloxAppendBatches (73) + : +- VeloxResizeBatches (73) : +- ^ ProjectExecTransformer (71) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (70) : :- ^ InputIteratorTransformer (60) : : +- ShuffleQueryStage (58), Statistics(X) : : +- ColumnarExchange (57) - : : +- VeloxAppendBatches (56) + : : +- VeloxResizeBatches (56) : : +- ^ ProjectExecTransformer (54) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (53) : : :- ^ InputIteratorTransformer (43) : : : +- ShuffleQueryStage (41), Statistics(X) : : : +- ColumnarExchange (40) - : : : +- VeloxAppendBatches (39) + : : : +- VeloxResizeBatches (39) : : : +- ^ ProjectExecTransformer (37) : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (36) : : : :- ^ InputIteratorTransformer (26) : : : : +- ShuffleQueryStage (24), Statistics(X) : : : : +- ColumnarExchange (23) - : : : : +- VeloxAppendBatches (22) + : : : : +- VeloxResizeBatches (22) : : : : +- ^ ProjectExecTransformer (20) : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) : : : : :- ^ InputIteratorTransformer (9) : : : : : +- ShuffleQueryStage (7), Statistics(X) : : : : : +- ColumnarExchange (6) - : : : : : +- VeloxAppendBatches (5) + : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ NoopFilter (2) : : : : : +- ^ Scan parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : +- ColumnarExchange (15) - : : : : +- VeloxAppendBatches (14) + : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ NoopFilter (11) : : : : +- ^ Scan parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33), Statistics(X) : : : +- ColumnarExchange (32) - : : : +- VeloxAppendBatches (31) + : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ NoopFilter (28) : : : +- ^ Scan parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50), Statistics(X) : : +- ColumnarExchange (49) - : : +- VeloxAppendBatches (48) + : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ NoopFilter (45) : : +- ^ Scan parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67), Statistics(X) : +- ColumnarExchange (66) - : +- VeloxAppendBatches (65) + : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ NoopFilter (62) : +- ^ Scan parquet (61) @@ -147,9 +147,9 @@ Input [2]: [s_suppkey#X, s_nationkey#X] Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -184,9 +184,9 @@ Input [5]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipda Input [6]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [6]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] @@ -216,9 +216,9 @@ Input [7]: [s_suppkey#X, s_nationkey#X, l_orderkey#X, l_suppkey#X, l_extendedpri Input [6]: [hash_partition_key#X, s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [6]: [hash_partition_key#X, s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [6]: [hash_partition_key#X, s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] @@ -253,9 +253,9 @@ Input [2]: [o_orderkey#X, o_custkey#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] @@ -285,9 +285,9 @@ Input [7]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_ship Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X] Arguments: false -(39) VeloxAppendBatches +(39) VeloxResizeBatches Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X] -Arguments: X +Arguments: X, X (40) ColumnarExchange Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X] @@ -322,9 +322,9 @@ Input [2]: [c_custkey#X, c_nationkey#X] Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] @@ -354,9 +354,9 @@ Input [7]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_cust Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X] Arguments: false -(56) VeloxAppendBatches +(56) VeloxResizeBatches Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X] -Arguments: X +Arguments: X, X (57) ColumnarExchange Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X] @@ -391,9 +391,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] Arguments: false -(65) VeloxAppendBatches +(65) VeloxResizeBatches Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] -Arguments: X +Arguments: X, X (66) ColumnarExchange Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] @@ -423,9 +423,9 @@ Input [7]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nati Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X, n_name#X] Arguments: false -(73) VeloxAppendBatches +(73) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X, n_name#X] -Arguments: X +Arguments: X, X (74) ColumnarExchange Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X, n_name#X] @@ -479,9 +479,9 @@ Input [5]: [supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] Arguments: false -(87) VeloxAppendBatches +(87) VeloxResizeBatches Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (88) ColumnarExchange Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] @@ -508,9 +508,9 @@ Results [4]: [supp_nation#X, cust_nation#X, l_year#X, sum(volume#X)#X AS revenue Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] Arguments: false -(94) VeloxAppendBatches +(94) VeloxResizeBatches Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] -Arguments: X +Arguments: X, X (95) ColumnarExchange Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/8.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/8.txt index 61f6287c2429..2dc7abc112bd 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/8.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/8.txt @@ -6,13 +6,13 @@ AdaptiveSparkPlan (207) +- ^ InputIteratorTransformer (138) +- ShuffleQueryStage (136), Statistics(X) +- ColumnarExchange (135) - +- VeloxAppendBatches (134) + +- VeloxResizeBatches (134) +- ^ ProjectExecTransformer (132) +- ^ RegularHashAggregateExecTransformer (131) +- ^ InputIteratorTransformer (130) +- ShuffleQueryStage (128), Statistics(X) +- ColumnarExchange (127) - +- VeloxAppendBatches (126) + +- VeloxResizeBatches (126) +- ^ ProjectExecTransformer (124) +- ^ FlushableHashAggregateExecTransformer (123) +- ^ ProjectExecTransformer (122) @@ -20,92 +20,92 @@ AdaptiveSparkPlan (207) :- ^ InputIteratorTransformer (111) : +- ShuffleQueryStage (109), Statistics(X) : +- ColumnarExchange (108) - : +- VeloxAppendBatches (107) + : +- VeloxResizeBatches (107) : +- ^ ProjectExecTransformer (105) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (104) : :- ^ InputIteratorTransformer (94) : : +- ShuffleQueryStage (92), Statistics(X) : : +- ColumnarExchange (91) - : : +- VeloxAppendBatches (90) + : : +- VeloxResizeBatches (90) : : +- ^ ProjectExecTransformer (88) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (87) : : :- ^ InputIteratorTransformer (77) : : : +- ShuffleQueryStage (75), Statistics(X) : : : +- ColumnarExchange (74) - : : : +- VeloxAppendBatches (73) + : : : +- VeloxResizeBatches (73) : : : +- ^ ProjectExecTransformer (71) : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (70) : : : :- ^ InputIteratorTransformer (60) : : : : +- ShuffleQueryStage (58), Statistics(X) : : : : +- ColumnarExchange (57) - : : : : +- VeloxAppendBatches (56) + : : : : +- VeloxResizeBatches (56) : : : : +- ^ ProjectExecTransformer (54) : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (53) : : : : :- ^ InputIteratorTransformer (43) : : : : : +- ShuffleQueryStage (41), Statistics(X) : : : : : +- ColumnarExchange (40) - : : : : : +- VeloxAppendBatches (39) + : : : : : +- VeloxResizeBatches (39) : : : : : +- ^ ProjectExecTransformer (37) : : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (36) : : : : : :- ^ InputIteratorTransformer (26) : : : : : : +- ShuffleQueryStage (24), Statistics(X) : : : : : : +- ColumnarExchange (23) - : : : : : : +- VeloxAppendBatches (22) + : : : : : : +- VeloxResizeBatches (22) : : : : : : +- ^ ProjectExecTransformer (20) : : : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) : : : : : : :- ^ InputIteratorTransformer (9) : : : : : : : +- ShuffleQueryStage (7), Statistics(X) : : : : : : : +- ColumnarExchange (6) - : : : : : : : +- VeloxAppendBatches (5) + : : : : : : : +- VeloxResizeBatches (5) : : : : : : : +- ^ ProjectExecTransformer (3) : : : : : : : +- ^ NoopFilter (2) : : : : : : : +- ^ Scan parquet (1) : : : : : : +- ^ InputIteratorTransformer (18) : : : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : : : +- ColumnarExchange (15) - : : : : : : +- VeloxAppendBatches (14) + : : : : : : +- VeloxResizeBatches (14) : : : : : : +- ^ ProjectExecTransformer (12) : : : : : : +- ^ NoopFilter (11) : : : : : : +- ^ Scan parquet (10) : : : : : +- ^ InputIteratorTransformer (35) : : : : : +- ShuffleQueryStage (33), Statistics(X) : : : : : +- ColumnarExchange (32) - : : : : : +- VeloxAppendBatches (31) + : : : : : +- VeloxResizeBatches (31) : : : : : +- ^ ProjectExecTransformer (29) : : : : : +- ^ NoopFilter (28) : : : : : +- ^ Scan parquet (27) : : : : +- ^ InputIteratorTransformer (52) : : : : +- ShuffleQueryStage (50), Statistics(X) : : : : +- ColumnarExchange (49) - : : : : +- VeloxAppendBatches (48) + : : : : +- VeloxResizeBatches (48) : : : : +- ^ ProjectExecTransformer (46) : : : : +- ^ NoopFilter (45) : : : : +- ^ Scan parquet (44) : : : +- ^ InputIteratorTransformer (69) : : : +- ShuffleQueryStage (67), Statistics(X) : : : +- ColumnarExchange (66) - : : : +- VeloxAppendBatches (65) + : : : +- VeloxResizeBatches (65) : : : +- ^ ProjectExecTransformer (63) : : : +- ^ NoopFilter (62) : : : +- ^ Scan parquet (61) : : +- ^ InputIteratorTransformer (86) : : +- ShuffleQueryStage (84), Statistics(X) : : +- ColumnarExchange (83) - : : +- VeloxAppendBatches (82) + : : +- VeloxResizeBatches (82) : : +- ^ ProjectExecTransformer (80) : : +- ^ NoopFilter (79) : : +- ^ Scan parquet (78) : +- ^ InputIteratorTransformer (103) : +- ShuffleQueryStage (101), Statistics(X) : +- ColumnarExchange (100) - : +- VeloxAppendBatches (99) + : +- VeloxResizeBatches (99) : +- ^ ProjectExecTransformer (97) : +- ^ NoopFilter (96) : +- ^ Scan parquet (95) +- ^ InputIteratorTransformer (120) +- ShuffleQueryStage (118), Statistics(X) +- ColumnarExchange (117) - +- VeloxAppendBatches (116) + +- VeloxResizeBatches (116) +- ^ ProjectExecTransformer (114) +- ^ NoopFilter (113) +- ^ Scan parquet (112) @@ -196,9 +196,9 @@ Input [2]: [p_partkey#X, p_type#X] Input [2]: [hash_partition_key#X, p_partkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [2]: [hash_partition_key#X, p_partkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [2]: [hash_partition_key#X, p_partkey#X] @@ -233,9 +233,9 @@ Input [5]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discoun Input [6]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [6]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] @@ -265,9 +265,9 @@ Input [6]: [p_partkey#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] @@ -302,9 +302,9 @@ Input [2]: [s_suppkey#X, s_nationkey#X] Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -334,9 +334,9 @@ Input [6]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, s_suppke Input [5]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] Arguments: false -(39) VeloxAppendBatches +(39) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] -Arguments: X +Arguments: X, X (40) ColumnarExchange Input [5]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] @@ -371,9 +371,9 @@ Input [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Input [4]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [4]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [4]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X] @@ -403,9 +403,9 @@ Input [7]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orde Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X] Arguments: false -(56) VeloxAppendBatches +(56) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X] -Arguments: X +Arguments: X, X (57) ColumnarExchange Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X] @@ -440,9 +440,9 @@ Input [2]: [c_custkey#X, c_nationkey#X] Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] Arguments: false -(65) VeloxAppendBatches +(65) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] -Arguments: X +Arguments: X, X (66) ColumnarExchange Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] @@ -472,9 +472,9 @@ Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_order Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X] Arguments: false -(73) VeloxAppendBatches +(73) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X] -Arguments: X +Arguments: X, X (74) ColumnarExchange Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X] @@ -509,9 +509,9 @@ Input [2]: [n_nationkey#X, n_regionkey#X] Input [3]: [hash_partition_key#X, n_nationkey#X, n_regionkey#X] Arguments: false -(82) VeloxAppendBatches +(82) VeloxResizeBatches Input [3]: [hash_partition_key#X, n_nationkey#X, n_regionkey#X] -Arguments: X +Arguments: X, X (83) ColumnarExchange Input [3]: [hash_partition_key#X, n_nationkey#X, n_regionkey#X] @@ -541,9 +541,9 @@ Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nat Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X] Arguments: false -(90) VeloxAppendBatches +(90) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X] -Arguments: X +Arguments: X, X (91) ColumnarExchange Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X] @@ -578,9 +578,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] Arguments: false -(99) VeloxAppendBatches +(99) VeloxResizeBatches Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] -Arguments: X +Arguments: X, X (100) ColumnarExchange Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] @@ -610,9 +610,9 @@ Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_reg Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X] Arguments: false -(107) VeloxAppendBatches +(107) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X] -Arguments: X +Arguments: X, X (108) ColumnarExchange Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X] @@ -647,9 +647,9 @@ Input [2]: [r_regionkey#X, r_name#X] Input [2]: [hash_partition_key#X, r_regionkey#X] Arguments: false -(116) VeloxAppendBatches +(116) VeloxResizeBatches Input [2]: [hash_partition_key#X, r_regionkey#X] -Arguments: X +Arguments: X, X (117) ColumnarExchange Input [2]: [hash_partition_key#X, r_regionkey#X] @@ -690,9 +690,9 @@ Input [5]: [o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] Arguments: false -(126) VeloxAppendBatches +(126) VeloxResizeBatches Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (127) ColumnarExchange Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] @@ -723,9 +723,9 @@ Input [3]: [o_year#X, sum(CASE WHEN (nation#X = BRAZIL) THEN volume#X ELSE 0.000 Input [2]: [o_year#X, mkt_share#X] Arguments: false -(134) VeloxAppendBatches +(134) VeloxResizeBatches Input [2]: [o_year#X, mkt_share#X] -Arguments: X +Arguments: X, X (135) ColumnarExchange Input [2]: [o_year#X, mkt_share#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/9.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/9.txt index 4b983de23fde..ce095ca705bc 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/9.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/9.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (155) +- ^ InputIteratorTransformer (103) +- ShuffleQueryStage (101), Statistics(X) +- ColumnarExchange (100) - +- VeloxAppendBatches (99) + +- VeloxResizeBatches (99) +- ^ RegularHashAggregateExecTransformer (97) +- ^ InputIteratorTransformer (96) +- ShuffleQueryStage (94), Statistics(X) +- ColumnarExchange (93) - +- VeloxAppendBatches (92) + +- VeloxResizeBatches (92) +- ^ ProjectExecTransformer (90) +- ^ FlushableHashAggregateExecTransformer (89) +- ^ ProjectExecTransformer (88) @@ -19,66 +19,66 @@ AdaptiveSparkPlan (155) :- ^ InputIteratorTransformer (77) : +- ShuffleQueryStage (75), Statistics(X) : +- ColumnarExchange (74) - : +- VeloxAppendBatches (73) + : +- VeloxResizeBatches (73) : +- ^ ProjectExecTransformer (71) : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (70) : :- ^ InputIteratorTransformer (60) : : +- ShuffleQueryStage (58), Statistics(X) : : +- ColumnarExchange (57) - : : +- VeloxAppendBatches (56) + : : +- VeloxResizeBatches (56) : : +- ^ ProjectExecTransformer (54) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (53) : : :- ^ InputIteratorTransformer (43) : : : +- ShuffleQueryStage (41), Statistics(X) : : : +- ColumnarExchange (40) - : : : +- VeloxAppendBatches (39) + : : : +- VeloxResizeBatches (39) : : : +- ^ ProjectExecTransformer (37) : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (36) : : : :- ^ InputIteratorTransformer (26) : : : : +- ShuffleQueryStage (24), Statistics(X) : : : : +- ColumnarExchange (23) - : : : : +- VeloxAppendBatches (22) + : : : : +- VeloxResizeBatches (22) : : : : +- ^ ProjectExecTransformer (20) : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) : : : : :- ^ InputIteratorTransformer (9) : : : : : +- ShuffleQueryStage (7), Statistics(X) : : : : : +- ColumnarExchange (6) - : : : : : +- VeloxAppendBatches (5) + : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ NoopFilter (2) : : : : : +- ^ Scan parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : +- ColumnarExchange (15) - : : : : +- VeloxAppendBatches (14) + : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ NoopFilter (11) : : : : +- ^ Scan parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33), Statistics(X) : : : +- ColumnarExchange (32) - : : : +- VeloxAppendBatches (31) + : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ NoopFilter (28) : : : +- ^ Scan parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50), Statistics(X) : : +- ColumnarExchange (49) - : : +- VeloxAppendBatches (48) + : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ NoopFilter (45) : : +- ^ Scan parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67), Statistics(X) : +- ColumnarExchange (66) - : +- VeloxAppendBatches (65) + : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ NoopFilter (62) : +- ^ Scan parquet (61) +- ^ InputIteratorTransformer (86) +- ShuffleQueryStage (84), Statistics(X) +- ColumnarExchange (83) - +- VeloxAppendBatches (82) + +- VeloxResizeBatches (82) +- ^ ProjectExecTransformer (80) +- ^ NoopFilter (79) +- ^ Scan parquet (78) @@ -152,9 +152,9 @@ Input [2]: [p_partkey#X, p_name#X] Input [2]: [hash_partition_key#X, p_partkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [2]: [hash_partition_key#X, p_partkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [2]: [hash_partition_key#X, p_partkey#X] @@ -189,9 +189,9 @@ Input [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpric Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] @@ -221,9 +221,9 @@ Input [7]: [p_partkey#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] @@ -258,9 +258,9 @@ Input [2]: [s_suppkey#X, s_nationkey#X] Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -290,9 +290,9 @@ Input [8]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpric Input [8]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X] Arguments: false -(39) VeloxAppendBatches +(39) VeloxResizeBatches Input [8]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X] -Arguments: X +Arguments: X, X (40) ColumnarExchange Input [8]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X] @@ -327,9 +327,9 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_supplycost#X] Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_supplycost#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_supplycost#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_supplycost#X] @@ -359,9 +359,9 @@ Input [10]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpri Input [7]: [hash_partition_key#X, l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X] Arguments: false -(56) VeloxAppendBatches +(56) VeloxResizeBatches Input [7]: [hash_partition_key#X, l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X] -Arguments: X +Arguments: X, X (57) ColumnarExchange Input [7]: [hash_partition_key#X, l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X] @@ -396,9 +396,9 @@ Input [2]: [o_orderkey#X, o_orderdate#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X] Arguments: false -(65) VeloxAppendBatches +(65) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X] -Arguments: X +Arguments: X, X (66) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X] @@ -428,9 +428,9 @@ Input [8]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_natio Input [7]: [hash_partition_key#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X] Arguments: false -(73) VeloxAppendBatches +(73) VeloxResizeBatches Input [7]: [hash_partition_key#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X] -Arguments: X +Arguments: X, X (74) ColumnarExchange Input [7]: [hash_partition_key#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X] @@ -465,9 +465,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] Arguments: false -(82) VeloxAppendBatches +(82) VeloxResizeBatches Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] -Arguments: X +Arguments: X, X (83) ColumnarExchange Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] @@ -508,9 +508,9 @@ Input [4]: [nation#X, o_year#X, sum#X, isEmpty#X] Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] Arguments: false -(92) VeloxAppendBatches +(92) VeloxResizeBatches Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (93) ColumnarExchange Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] @@ -537,9 +537,9 @@ Results [3]: [nation#X, o_year#X, sum(amount#X)#X AS sum_profit#X] Input [3]: [nation#X, o_year#X, sum_profit#X] Arguments: false -(99) VeloxAppendBatches +(99) VeloxResizeBatches Input [3]: [nation#X, o_year#X, sum_profit#X] -Arguments: X +Arguments: X, X (100) ColumnarExchange Input [3]: [nation#X, o_year#X, sum_profit#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/1.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/1.txt index 63b7d317f3cf..5ceb73b301db 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/1.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/1.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (30) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ RegularHashAggregateExecTransformer (12) +- ^ InputIteratorTransformer (11) +- ShuffleQueryStage (9) +- ColumnarExchange (8) - +- VeloxAppendBatches (7) + +- VeloxResizeBatches (7) +- ^ ProjectExecTransformer (5) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) @@ -58,9 +58,9 @@ Input [17]: [l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] Arguments: false -(7) VeloxAppendBatches +(7) VeloxResizeBatches Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] -Arguments: X +Arguments: X, X (8) ColumnarExchange Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] @@ -87,9 +87,9 @@ Results [10]: [l_returnflag#X, l_linestatus#X, sum(l_quantity#X)#X AS sum_qty#X, Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/10.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/10.txt index 993884df3f3a..b6c3dc67a1a5 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/10.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/10.txt @@ -8,7 +8,7 @@ AdaptiveSparkPlan (100) +- ^ InputIteratorTransformer (62) +- ShuffleQueryStage (60) +- ColumnarExchange (59) - +- VeloxAppendBatches (58) + +- VeloxResizeBatches (58) +- ^ ProjectExecTransformer (56) +- ^ FlushableHashAggregateExecTransformer (55) +- ^ ProjectExecTransformer (54) @@ -16,40 +16,40 @@ AdaptiveSparkPlan (100) :- ^ InputIteratorTransformer (43) : +- ShuffleQueryStage (41) : +- ColumnarExchange (40) - : +- VeloxAppendBatches (39) + : +- VeloxResizeBatches (39) : +- ^ ProjectExecTransformer (37) : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (36) : :- ^ InputIteratorTransformer (26) : : +- ShuffleQueryStage (24) : : +- ColumnarExchange (23) - : : +- VeloxAppendBatches (22) + : : +- VeloxResizeBatches (22) : : +- ^ ProjectExecTransformer (20) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) : : :- ^ InputIteratorTransformer (9) : : : +- ShuffleQueryStage (7) : : : +- ColumnarExchange (6) - : : : +- VeloxAppendBatches (5) + : : : +- VeloxResizeBatches (5) : : : +- ^ ProjectExecTransformer (3) : : : +- ^ FilterExecTransformer (2) : : : +- ^ Scan parquet (1) : : +- ^ InputIteratorTransformer (18) : : +- ShuffleQueryStage (16) : : +- ColumnarExchange (15) - : : +- VeloxAppendBatches (14) + : : +- VeloxResizeBatches (14) : : +- ^ ProjectExecTransformer (12) : : +- ^ FilterExecTransformer (11) : : +- ^ Scan parquet (10) : +- ^ InputIteratorTransformer (35) : +- ShuffleQueryStage (33) : +- ColumnarExchange (32) - : +- VeloxAppendBatches (31) + : +- VeloxResizeBatches (31) : +- ^ ProjectExecTransformer (29) : +- ^ FilterExecTransformer (28) : +- ^ Scan parquet (27) +- ^ InputIteratorTransformer (52) +- ShuffleQueryStage (50) +- ColumnarExchange (49) - +- VeloxAppendBatches (48) + +- VeloxResizeBatches (48) +- ^ ProjectExecTransformer (46) +- ^ FilterExecTransformer (45) +- ^ Scan parquet (44) @@ -107,9 +107,9 @@ Input [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct Input [8]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [8]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [8]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] @@ -144,9 +144,9 @@ Input [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] @@ -175,9 +175,9 @@ Input [9]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct Input [9]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, o_orderkey#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [9]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, o_orderkey#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [9]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, o_orderkey#X] @@ -212,9 +212,9 @@ Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_returnflag#X] Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] @@ -243,9 +243,9 @@ Input [11]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acc Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, l_extendedprice#X, l_discount#X] Arguments: false -(39) VeloxAppendBatches +(39) VeloxResizeBatches Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (40) ColumnarExchange Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, l_extendedprice#X, l_discount#X] @@ -280,9 +280,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] @@ -322,9 +322,9 @@ Input [9]: [c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] Arguments: false -(58) VeloxAppendBatches +(58) VeloxResizeBatches Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (59) ColumnarExchange Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/11.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/11.txt index 8142375d9ead..48d14d1812d1 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/11.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/11.txt @@ -6,13 +6,13 @@ AdaptiveSparkPlan (82) +- ^ InputIteratorTransformer (53) +- ShuffleQueryStage (51) +- ColumnarExchange (50) - +- VeloxAppendBatches (49) + +- VeloxResizeBatches (49) +- ^ FilterExecTransformer (47) +- ^ RegularHashAggregateExecTransformer (46) +- ^ InputIteratorTransformer (45) +- ShuffleQueryStage (43) +- ColumnarExchange (42) - +- VeloxAppendBatches (41) + +- VeloxResizeBatches (41) +- ^ ProjectExecTransformer (39) +- ^ FlushableHashAggregateExecTransformer (38) +- ^ ProjectExecTransformer (37) @@ -20,27 +20,27 @@ AdaptiveSparkPlan (82) :- ^ InputIteratorTransformer (26) : +- ShuffleQueryStage (24) : +- ColumnarExchange (23) - : +- VeloxAppendBatches (22) + : +- VeloxResizeBatches (22) : +- ^ ProjectExecTransformer (20) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7) : : +- ColumnarExchange (6) - : : +- VeloxAppendBatches (5) + : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16) : +- ColumnarExchange (15) - : +- VeloxAppendBatches (14) + : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ FilterExecTransformer (11) : +- ^ Scan parquet (10) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33) +- ColumnarExchange (32) - +- VeloxAppendBatches (31) + +- VeloxResizeBatches (31) +- ^ ProjectExecTransformer (29) +- ^ FilterExecTransformer (28) +- ^ Scan parquet (27) @@ -91,9 +91,9 @@ Input [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Input [5]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [5]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [5]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] @@ -128,9 +128,9 @@ Input [2]: [s_suppkey#X, s_nationkey#X] Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -159,9 +159,9 @@ Input [6]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X, s_suppke Input [5]: [hash_partition_key#X, ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [5]: [hash_partition_key#X, ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [5]: [hash_partition_key#X, ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] @@ -196,9 +196,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [2]: [hash_partition_key#X, n_nationkey#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [2]: [hash_partition_key#X, n_nationkey#X] @@ -238,9 +238,9 @@ Input [3]: [ps_partkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] Arguments: false -(41) VeloxAppendBatches +(41) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (42) ColumnarExchange Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] @@ -271,9 +271,9 @@ Arguments: (isnotnull(value#X) AND (cast(value#X as decimal(38,6)) > Subquery su Input [2]: [ps_partkey#X, value#X] Arguments: false -(49) VeloxAppendBatches +(49) VeloxResizeBatches Input [2]: [ps_partkey#X, value#X] -Arguments: X +Arguments: X, X (50) ColumnarExchange Input [2]: [ps_partkey#X, value#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/12.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/12.txt index 802a79759235..0a836a6d8ed7 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/12.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/12.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (55) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33) +- ColumnarExchange (32) - +- VeloxAppendBatches (31) + +- VeloxResizeBatches (31) +- ^ RegularHashAggregateExecTransformer (29) +- ^ InputIteratorTransformer (28) +- ShuffleQueryStage (26) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ ProjectExecTransformer (22) +- ^ FlushableHashAggregateExecTransformer (21) +- ^ ProjectExecTransformer (20) @@ -19,14 +19,14 @@ AdaptiveSparkPlan (55) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) +- ^ Scan parquet (10) @@ -68,9 +68,9 @@ Input [2]: [o_orderkey#X, o_orderpriority#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] @@ -105,9 +105,9 @@ Input [5]: [l_orderkey#X, l_shipdate#X, l_commitdate#X, l_receiptdate#X, l_shipm Input [3]: [hash_partition_key#X, l_orderkey#X, l_shipmode#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_shipmode#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_shipmode#X] @@ -147,9 +147,9 @@ Input [3]: [l_shipmode#X, sum#X, sum#X] Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] @@ -176,9 +176,9 @@ Results [3]: [l_shipmode#X, sum(CASE WHEN ((o_orderpriority#X = 1-URGENT) OR (o_ Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/13.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/13.txt index b9bf0f1fad60..522a695f73b1 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/13.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/13.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (58) +- ^ InputIteratorTransformer (37) +- ShuffleQueryStage (35) +- ColumnarExchange (34) - +- VeloxAppendBatches (33) + +- VeloxResizeBatches (33) +- ^ RegularHashAggregateExecTransformer (31) +- ^ InputIteratorTransformer (30) +- ShuffleQueryStage (28) +- ColumnarExchange (27) - +- VeloxAppendBatches (26) + +- VeloxResizeBatches (26) +- ^ ProjectExecTransformer (24) +- ^ FlushableHashAggregateExecTransformer (23) +- ^ ProjectExecTransformer (22) @@ -22,13 +22,13 @@ AdaptiveSparkPlan (58) :- ^ InputIteratorTransformer (8) : +- ShuffleQueryStage (6) : +- ColumnarExchange (5) - : +- VeloxAppendBatches (4) + : +- VeloxResizeBatches (4) : +- ^ ProjectExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (17) +- ShuffleQueryStage (15) +- ColumnarExchange (14) - +- VeloxAppendBatches (13) + +- VeloxResizeBatches (13) +- ^ ProjectExecTransformer (11) +- ^ FilterExecTransformer (10) +- ^ Scan parquet (9) @@ -66,9 +66,9 @@ Input [1]: [c_custkey#X] Input [2]: [hash_partition_key#X, c_custkey#X] Arguments: false -(4) VeloxAppendBatches +(4) VeloxResizeBatches Input [2]: [hash_partition_key#X, c_custkey#X] -Arguments: X +Arguments: X, X (5) ColumnarExchange Input [2]: [hash_partition_key#X, c_custkey#X] @@ -103,9 +103,9 @@ Input [3]: [o_orderkey#X, o_custkey#X, o_comment#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] Arguments: false -(13) VeloxAppendBatches +(13) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] -Arguments: X +Arguments: X, X (14) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] @@ -163,9 +163,9 @@ Input [2]: [c_count#X, count#X] Input [3]: [hash_partition_key#X, c_count#X, count#X] Arguments: false -(26) VeloxAppendBatches +(26) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_count#X, count#X] -Arguments: X +Arguments: X, X (27) ColumnarExchange Input [3]: [hash_partition_key#X, c_count#X, count#X] @@ -192,9 +192,9 @@ Results [2]: [c_count#X, count(1)#X AS custdist#X] Input [2]: [c_count#X, custdist#X] Arguments: false -(33) VeloxAppendBatches +(33) VeloxResizeBatches Input [2]: [c_count#X, custdist#X] -Arguments: X +Arguments: X, X (34) ColumnarExchange Input [2]: [c_count#X, custdist#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/14.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/14.txt index 425c55f5a4ce..d050850332af 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/14.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/14.txt @@ -10,14 +10,14 @@ AdaptiveSparkPlan (39) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) +- ^ Scan parquet (10) @@ -56,9 +56,9 @@ Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Input [4]: [hash_partition_key#X, l_partkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_partkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [4]: [hash_partition_key#X, l_partkey#X, l_extendedprice#X, l_discount#X] @@ -93,9 +93,9 @@ Input [2]: [p_partkey#X, p_type#X] Input [3]: [hash_partition_key#X, p_partkey#X, p_type#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, p_partkey#X, p_type#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, p_partkey#X, p_type#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/15.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/15.txt index f003eed994d0..1d085e52a5d4 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/15.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/15.txt @@ -6,13 +6,13 @@ AdaptiveSparkPlan (50) +- ^ InputIteratorTransformer (30) +- ShuffleQueryStage (28) +- ColumnarExchange (27) - +- VeloxAppendBatches (26) + +- VeloxResizeBatches (26) +- ^ ProjectExecTransformer (24) +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (23) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) @@ -21,7 +21,7 @@ AdaptiveSparkPlan (50) +- ^ InputIteratorTransformer (20) +- ShuffleQueryStage (18) +- ColumnarExchange (17) - +- VeloxAppendBatches (16) + +- VeloxResizeBatches (16) +- ^ ProjectExecTransformer (14) +- ^ FlushableHashAggregateExecTransformer (13) +- ^ ProjectExecTransformer (12) @@ -65,9 +65,9 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_phone#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_phone#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_phone#X] @@ -113,9 +113,9 @@ Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] Arguments: false -(16) VeloxAppendBatches +(16) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (17) ColumnarExchange Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] @@ -155,9 +155,9 @@ Input [6]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, supplier_no#X, total_ Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] Arguments: false -(26) VeloxAppendBatches +(26) VeloxResizeBatches Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] -Arguments: X +Arguments: X, X (27) ColumnarExchange Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/16.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/16.txt index c9374b01ff02..4b6a18e96860 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/16.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/16.txt @@ -6,19 +6,19 @@ AdaptiveSparkPlan (71) +- ^ InputIteratorTransformer (44) +- ShuffleQueryStage (42) +- ColumnarExchange (41) - +- VeloxAppendBatches (40) + +- VeloxResizeBatches (40) +- ^ RegularHashAggregateExecTransformer (38) +- ^ InputIteratorTransformer (37) +- ShuffleQueryStage (35) +- ColumnarExchange (34) - +- VeloxAppendBatches (33) + +- VeloxResizeBatches (33) +- ^ ProjectExecTransformer (31) +- ^ FlushableHashAggregateExecTransformer (30) +- ^ RegularHashAggregateExecTransformer (29) +- ^ InputIteratorTransformer (28) +- ShuffleQueryStage (26) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ ProjectExecTransformer (22) +- ^ FlushableHashAggregateExecTransformer (21) +- ^ ProjectExecTransformer (20) @@ -26,14 +26,14 @@ AdaptiveSparkPlan (71) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) +- ^ Scan parquet (10) @@ -82,9 +82,9 @@ Input [2]: [ps_partkey#X, ps_suppkey#X] Input [3]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X] @@ -119,9 +119,9 @@ Input [4]: [p_partkey#X, p_brand#X, p_type#X, p_size#X] Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_type#X, p_size#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_type#X, p_size#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_type#X, p_size#X] @@ -161,9 +161,9 @@ Input [4]: [p_brand#X, p_type#X, p_size#X, ps_suppkey#X] Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] @@ -201,9 +201,9 @@ Input [4]: [p_brand#X, p_type#X, p_size#X, count#X] Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] Arguments: false -(33) VeloxAppendBatches +(33) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] -Arguments: X +Arguments: X, X (34) ColumnarExchange Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] @@ -230,9 +230,9 @@ Results [4]: [p_brand#X, p_type#X, p_size#X, count(ps_suppkey#X)#X AS supplier_c Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] Arguments: false -(40) VeloxAppendBatches +(40) VeloxResizeBatches Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] -Arguments: X +Arguments: X, X (41) ColumnarExchange Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/17.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/17.txt index 69f50fa16ab0..6ebaa345a5a9 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/17.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/17.txt @@ -12,14 +12,14 @@ AdaptiveSparkPlan (63) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7) : : +- ColumnarExchange (6) - : : +- VeloxAppendBatches (5) + : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16) : +- ColumnarExchange (15) - : +- VeloxAppendBatches (14) + : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ FilterExecTransformer (11) : +- ^ Scan parquet (10) @@ -29,7 +29,7 @@ AdaptiveSparkPlan (63) +- ^ InputIteratorTransformer (30) +- ShuffleQueryStage (28) +- ColumnarExchange (27) - +- VeloxAppendBatches (26) + +- VeloxResizeBatches (26) +- ^ ProjectExecTransformer (24) +- ^ FlushableHashAggregateExecTransformer (23) +- ^ FilterExecTransformer (22) @@ -78,9 +78,9 @@ Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Input [4]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [4]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X] @@ -115,9 +115,9 @@ Input [3]: [p_partkey#X, p_brand#X, p_container#X] Input [2]: [hash_partition_key#X, p_partkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [2]: [hash_partition_key#X, p_partkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [2]: [hash_partition_key#X, p_partkey#X] @@ -168,9 +168,9 @@ Input [3]: [l_partkey#X, sum#X, count#X] Input [4]: [hash_partition_key#X, l_partkey#X, sum#X, count#X] Arguments: false -(26) VeloxAppendBatches +(26) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_partkey#X, sum#X, count#X] -Arguments: X +Arguments: X, X (27) ColumnarExchange Input [4]: [hash_partition_key#X, l_partkey#X, sum#X, count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/18.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/18.txt index 96fa1cd82606..de38e4d98b65 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/18.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/18.txt @@ -10,26 +10,26 @@ AdaptiveSparkPlan (110) :- ^ InputIteratorTransformer (46) : +- ShuffleQueryStage (44) : +- ColumnarExchange (43) - : +- VeloxAppendBatches (42) + : +- VeloxResizeBatches (42) : +- ^ ProjectExecTransformer (40) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (39) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7) : : +- ColumnarExchange (6) - : : +- VeloxAppendBatches (5) + : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (38) : +- ShuffleQueryStage (36) : +- ColumnarExchange (35) - : +- VeloxAppendBatches (34) + : +- VeloxResizeBatches (34) : +- ^ ProjectExecTransformer (32) : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (31) : :- ^ InputIteratorTransformer (18) : : +- ShuffleQueryStage (16) : : +- ColumnarExchange (15) - : : +- VeloxAppendBatches (14) + : : +- VeloxResizeBatches (14) : : +- ^ ProjectExecTransformer (12) : : +- ^ FilterExecTransformer (11) : : +- ^ Scan parquet (10) @@ -39,7 +39,7 @@ AdaptiveSparkPlan (110) : +- ^ InputIteratorTransformer (27) : +- ShuffleQueryStage (25) : +- ColumnarExchange (24) - : +- VeloxAppendBatches (23) + : +- VeloxResizeBatches (23) : +- ^ ProjectExecTransformer (21) : +- ^ FlushableHashAggregateExecTransformer (20) : +- ^ Scan parquet (19) @@ -47,7 +47,7 @@ AdaptiveSparkPlan (110) :- ^ InputIteratorTransformer (55) : +- ShuffleQueryStage (53) : +- ColumnarExchange (52) - : +- VeloxAppendBatches (51) + : +- VeloxResizeBatches (51) : +- ^ ProjectExecTransformer (49) : +- ^ FilterExecTransformer (48) : +- ^ Scan parquet (47) @@ -118,9 +118,9 @@ Input [2]: [c_custkey#X, c_name#X] Input [3]: [hash_partition_key#X, c_custkey#X, c_name#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_custkey#X, c_name#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, c_custkey#X, c_name#X] @@ -155,9 +155,9 @@ Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] @@ -194,9 +194,9 @@ Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] Arguments: false -(23) VeloxAppendBatches +(23) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (24) ColumnarExchange Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] @@ -240,9 +240,9 @@ Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: false -(34) VeloxAppendBatches +(34) VeloxResizeBatches Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] -Arguments: X +Arguments: X, X (35) ColumnarExchange Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] @@ -271,9 +271,9 @@ Input [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_ Input [6]: [hash_partition_key#X, c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] Arguments: false -(42) VeloxAppendBatches +(42) VeloxResizeBatches Input [6]: [hash_partition_key#X, c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] -Arguments: X +Arguments: X, X (43) ColumnarExchange Input [6]: [hash_partition_key#X, c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] @@ -308,9 +308,9 @@ Input [2]: [l_orderkey#X, l_quantity#X] Input [3]: [hash_partition_key#X, l_orderkey#X, l_quantity#X] Arguments: false -(51) VeloxAppendBatches +(51) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_quantity#X] -Arguments: X +Arguments: X, X (52) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_quantity#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/19.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/19.txt index ae0feb5dfd56..84520a969902 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/19.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/19.txt @@ -9,14 +9,14 @@ AdaptiveSparkPlan (38) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) +- ^ Scan parquet (10) @@ -55,9 +55,9 @@ Input [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipin Input [5]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [5]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] @@ -92,9 +92,9 @@ Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_size#X, p_container#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_size#X, p_container#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_size#X, p_container#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/20.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/20.txt index bab785551636..d581db858a0c 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/20.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/20.txt @@ -6,52 +6,52 @@ AdaptiveSparkPlan (146) +- ^ InputIteratorTransformer (93) +- ShuffleQueryStage (91) +- ColumnarExchange (90) - +- VeloxAppendBatches (89) + +- VeloxResizeBatches (89) +- ^ ProjectExecTransformer (87) +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (86) :- ^ InputIteratorTransformer (76) : +- ShuffleQueryStage (74) : +- ColumnarExchange (73) - : +- VeloxAppendBatches (72) + : +- VeloxResizeBatches (72) : +- ^ ProjectExecTransformer (70) : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (69) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7) : : +- ColumnarExchange (6) - : : +- VeloxAppendBatches (5) + : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (68) : +- ShuffleQueryStage (66) : +- ColumnarExchange (65) - : +- VeloxAppendBatches (64) + : +- VeloxResizeBatches (64) : +- ^ ProjectExecTransformer (62) : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (61) : :- ^ InputIteratorTransformer (35) : : +- ShuffleQueryStage (33) : : +- ColumnarExchange (32) - : : +- VeloxAppendBatches (31) + : : +- VeloxResizeBatches (31) : : +- ^ ProjectExecTransformer (29) : : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (28) : : :- ^ InputIteratorTransformer (18) : : : +- ShuffleQueryStage (16) : : : +- ColumnarExchange (15) - : : : +- VeloxAppendBatches (14) + : : : +- VeloxResizeBatches (14) : : : +- ^ ProjectExecTransformer (12) : : : +- ^ FilterExecTransformer (11) : : : +- ^ Scan parquet (10) : : +- ^ InputIteratorTransformer (27) : : +- ShuffleQueryStage (25) : : +- ColumnarExchange (24) - : : +- VeloxAppendBatches (23) + : : +- VeloxResizeBatches (23) : : +- ^ ProjectExecTransformer (21) : : +- ^ FilterExecTransformer (20) : : +- ^ Scan parquet (19) : +- ^ InputIteratorTransformer (60) : +- ShuffleQueryStage (58) : +- ColumnarExchange (57) - : +- VeloxAppendBatches (56) + : +- VeloxResizeBatches (56) : +- ^ ProjectExecTransformer (54) : +- ^ FilterExecTransformer (53) : +- ^ ProjectExecTransformer (52) @@ -61,7 +61,7 @@ AdaptiveSparkPlan (146) : :- ^ InputIteratorTransformer (44) : : +- ShuffleQueryStage (42) : : +- ColumnarExchange (41) - : : +- VeloxAppendBatches (40) + : : +- VeloxResizeBatches (40) : : +- ^ ProjectExecTransformer (38) : : +- ^ FilterExecTransformer (37) : : +- ^ Scan parquet (36) @@ -71,7 +71,7 @@ AdaptiveSparkPlan (146) +- ^ InputIteratorTransformer (85) +- ShuffleQueryStage (83) +- ColumnarExchange (82) - +- VeloxAppendBatches (81) + +- VeloxResizeBatches (81) +- ^ ProjectExecTransformer (79) +- ^ FilterExecTransformer (78) +- ^ Scan parquet (77) @@ -146,9 +146,9 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] @@ -183,9 +183,9 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] @@ -220,9 +220,9 @@ Input [2]: [p_partkey#X, p_name#X] Input [2]: [hash_partition_key#X, p_partkey#X] Arguments: false -(23) VeloxAppendBatches +(23) VeloxResizeBatches Input [2]: [hash_partition_key#X, p_partkey#X] -Arguments: X +Arguments: X, X (24) ColumnarExchange Input [2]: [hash_partition_key#X, p_partkey#X] @@ -251,9 +251,9 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] @@ -288,9 +288,9 @@ Input [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] Input [4]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, l_quantity#X] Arguments: false -(40) VeloxAppendBatches +(40) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, l_quantity#X] -Arguments: X +Arguments: X, X (41) ColumnarExchange Input [4]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, l_quantity#X] @@ -354,9 +354,9 @@ Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: false -(56) VeloxAppendBatches +(56) VeloxResizeBatches Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -Arguments: X +Arguments: X, X (57) ColumnarExchange Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] @@ -385,9 +385,9 @@ Input [6]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, (0.5 * sum(l_quantity))#X Input [2]: [hash_partition_key#X, ps_suppkey#X] Arguments: false -(64) VeloxAppendBatches +(64) VeloxResizeBatches Input [2]: [hash_partition_key#X, ps_suppkey#X] -Arguments: X +Arguments: X, X (65) ColumnarExchange Input [2]: [hash_partition_key#X, ps_suppkey#X] @@ -416,9 +416,9 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] Arguments: false -(72) VeloxAppendBatches +(72) VeloxResizeBatches Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] -Arguments: X +Arguments: X, X (73) ColumnarExchange Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] @@ -453,9 +453,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: false -(81) VeloxAppendBatches +(81) VeloxResizeBatches Input [2]: [hash_partition_key#X, n_nationkey#X] -Arguments: X +Arguments: X, X (82) ColumnarExchange Input [2]: [hash_partition_key#X, n_nationkey#X] @@ -484,9 +484,9 @@ Input [4]: [s_name#X, s_address#X, s_nationkey#X, n_nationkey#X] Input [2]: [s_name#X, s_address#X] Arguments: false -(89) VeloxAppendBatches +(89) VeloxResizeBatches Input [2]: [s_name#X, s_address#X] -Arguments: X +Arguments: X, X (90) ColumnarExchange Input [2]: [s_name#X, s_address#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/21.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/21.txt index ef4e87bb1de4..c9375e9823a3 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/21.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/21.txt @@ -7,7 +7,7 @@ AdaptiveSparkPlan (138) +- ^ InputIteratorTransformer (88) +- ShuffleQueryStage (86) +- ColumnarExchange (85) - +- VeloxAppendBatches (84) + +- VeloxResizeBatches (84) +- ^ ProjectExecTransformer (82) +- ^ FlushableHashAggregateExecTransformer (81) +- ^ ProjectExecTransformer (80) @@ -15,60 +15,60 @@ AdaptiveSparkPlan (138) :- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67) : +- ColumnarExchange (66) - : +- VeloxAppendBatches (65) + : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (62) : :- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50) : : +- ColumnarExchange (49) - : : +- VeloxAppendBatches (48) + : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (45) : : :- ^ InputIteratorTransformer (9) : : : +- ShuffleQueryStage (7) : : : +- ColumnarExchange (6) - : : : +- VeloxAppendBatches (5) + : : : +- VeloxResizeBatches (5) : : : +- ^ ProjectExecTransformer (3) : : : +- ^ FilterExecTransformer (2) : : : +- ^ Scan parquet (1) : : +- ^ InputIteratorTransformer (44) : : +- ShuffleQueryStage (42) : : +- ColumnarExchange (41) - : : +- VeloxAppendBatches (40) + : : +- VeloxResizeBatches (40) : : +- ^ ProjectExecTransformer (38) : : +- ^ ShuffledHashJoinExecTransformer LeftAnti BuildRight (37) : : :- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (27) : : : :- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16) : : : : +- ColumnarExchange (15) - : : : : +- VeloxAppendBatches (14) + : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) : : : : +- ^ Scan parquet (10) : : : +- ^ InputIteratorTransformer (26) : : : +- ShuffleQueryStage (24) : : : +- ColumnarExchange (23) - : : : +- VeloxAppendBatches (22) + : : : +- VeloxResizeBatches (22) : : : +- ^ ProjectExecTransformer (20) : : : +- ^ Scan parquet (19) : : +- ^ InputIteratorTransformer (36) : : +- ShuffleQueryStage (34) : : +- ColumnarExchange (33) - : : +- VeloxAppendBatches (32) + : : +- VeloxResizeBatches (32) : : +- ^ ProjectExecTransformer (30) : : +- ^ FilterExecTransformer (29) : : +- ^ Scan parquet (28) : +- ^ InputIteratorTransformer (61) : +- ShuffleQueryStage (59) : +- ColumnarExchange (58) - : +- VeloxAppendBatches (57) + : +- VeloxResizeBatches (57) : +- ^ ProjectExecTransformer (55) : +- ^ FilterExecTransformer (54) : +- ^ Scan parquet (53) +- ^ InputIteratorTransformer (78) +- ShuffleQueryStage (76) +- ColumnarExchange (75) - +- VeloxAppendBatches (74) + +- VeloxResizeBatches (74) +- ^ ProjectExecTransformer (72) +- ^ FilterExecTransformer (71) +- ^ Scan parquet (70) @@ -139,9 +139,9 @@ Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X] Input [4]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_nationkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [4]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_nationkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [4]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_nationkey#X] @@ -176,9 +176,9 @@ Input [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] @@ -208,9 +208,9 @@ Input [2]: [l_orderkey#X, l_suppkey#X] Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] @@ -250,9 +250,9 @@ Input [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] Arguments: false -(32) VeloxAppendBatches +(32) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] -Arguments: X +Arguments: X, X (33) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] @@ -281,9 +281,9 @@ Input [2]: [l_orderkey#X, l_suppkey#X] Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] Arguments: false -(40) VeloxAppendBatches +(40) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] -Arguments: X +Arguments: X, X (41) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] @@ -312,9 +312,9 @@ Input [5]: [s_suppkey#X, s_name#X, s_nationkey#X, l_orderkey#X, l_suppkey#X] Input [4]: [hash_partition_key#X, s_name#X, s_nationkey#X, l_orderkey#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [4]: [hash_partition_key#X, s_name#X, s_nationkey#X, l_orderkey#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [4]: [hash_partition_key#X, s_name#X, s_nationkey#X, l_orderkey#X] @@ -349,9 +349,9 @@ Input [2]: [o_orderkey#X, o_orderstatus#X] Input [2]: [hash_partition_key#X, o_orderkey#X] Arguments: false -(57) VeloxAppendBatches +(57) VeloxResizeBatches Input [2]: [hash_partition_key#X, o_orderkey#X] -Arguments: X +Arguments: X, X (58) ColumnarExchange Input [2]: [hash_partition_key#X, o_orderkey#X] @@ -380,9 +380,9 @@ Input [4]: [s_name#X, s_nationkey#X, l_orderkey#X, o_orderkey#X] Input [3]: [hash_partition_key#X, s_name#X, s_nationkey#X] Arguments: false -(65) VeloxAppendBatches +(65) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_name#X, s_nationkey#X] -Arguments: X +Arguments: X, X (66) ColumnarExchange Input [3]: [hash_partition_key#X, s_name#X, s_nationkey#X] @@ -417,9 +417,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: false -(74) VeloxAppendBatches +(74) VeloxResizeBatches Input [2]: [hash_partition_key#X, n_nationkey#X] -Arguments: X +Arguments: X, X (75) ColumnarExchange Input [2]: [hash_partition_key#X, n_nationkey#X] @@ -459,9 +459,9 @@ Input [2]: [s_name#X, count#X] Input [3]: [hash_partition_key#X, s_name#X, count#X] Arguments: false -(84) VeloxAppendBatches +(84) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_name#X, count#X] -Arguments: X +Arguments: X, X (85) ColumnarExchange Input [3]: [hash_partition_key#X, s_name#X, count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/22.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/22.txt index fcf712a9d5fd..9e899a1b5e15 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/22.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/22.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (52) +- ^ InputIteratorTransformer (34) +- ShuffleQueryStage (32) +- ColumnarExchange (31) - +- VeloxAppendBatches (30) + +- VeloxResizeBatches (30) +- ^ RegularHashAggregateExecTransformer (28) +- ^ InputIteratorTransformer (27) +- ShuffleQueryStage (25) +- ColumnarExchange (24) - +- VeloxAppendBatches (23) + +- VeloxResizeBatches (23) +- ^ ProjectExecTransformer (21) +- ^ FlushableHashAggregateExecTransformer (20) +- ^ ProjectExecTransformer (19) @@ -19,14 +19,14 @@ AdaptiveSparkPlan (52) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (17) +- ShuffleQueryStage (15) +- ColumnarExchange (14) - +- VeloxAppendBatches (13) + +- VeloxResizeBatches (13) +- ^ ProjectExecTransformer (11) +- ^ Scan parquet (10) +- == Initial Plan == @@ -65,9 +65,9 @@ Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X] Input [4]: [hash_partition_key#X, c_custkey#X, c_phone#X, c_acctbal#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [4]: [hash_partition_key#X, c_custkey#X, c_phone#X, c_acctbal#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [4]: [hash_partition_key#X, c_custkey#X, c_phone#X, c_acctbal#X] @@ -97,9 +97,9 @@ Input [1]: [o_custkey#X] Input [2]: [hash_partition_key#X, o_custkey#X] Arguments: false -(13) VeloxAppendBatches +(13) VeloxResizeBatches Input [2]: [hash_partition_key#X, o_custkey#X] -Arguments: X +Arguments: X, X (14) ColumnarExchange Input [2]: [hash_partition_key#X, o_custkey#X] @@ -139,9 +139,9 @@ Input [4]: [cntrycode#X, count#X, sum#X, isEmpty#X] Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] Arguments: false -(23) VeloxAppendBatches +(23) VeloxResizeBatches Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (24) ColumnarExchange Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] @@ -168,9 +168,9 @@ Results [3]: [cntrycode#X, count(1)#X AS numcust#X, sum(c_acctbal#X)#X AS totacc Input [3]: [cntrycode#X, numcust#X, totacctbal#X] Arguments: false -(30) VeloxAppendBatches +(30) VeloxResizeBatches Input [3]: [cntrycode#X, numcust#X, totacctbal#X] -Arguments: X +Arguments: X, X (31) ColumnarExchange Input [3]: [cntrycode#X, numcust#X, totacctbal#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/3.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/3.txt index 607d6444f432..a3fb4062c83d 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/3.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/3.txt @@ -11,27 +11,27 @@ AdaptiveSparkPlan (67) :- ^ InputIteratorTransformer (26) : +- ShuffleQueryStage (24) : +- ColumnarExchange (23) - : +- VeloxAppendBatches (22) + : +- VeloxResizeBatches (22) : +- ^ ProjectExecTransformer (20) : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7) : : +- ColumnarExchange (6) - : : +- VeloxAppendBatches (5) + : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16) : +- ColumnarExchange (15) - : +- VeloxAppendBatches (14) + : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ FilterExecTransformer (11) : +- ^ Scan parquet (10) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33) +- ColumnarExchange (32) - +- VeloxAppendBatches (31) + +- VeloxResizeBatches (31) +- ^ ProjectExecTransformer (29) +- ^ FilterExecTransformer (28) +- ^ Scan parquet (27) @@ -80,9 +80,9 @@ Input [2]: [c_custkey#X, c_mktsegment#X] Input [2]: [hash_partition_key#X, c_custkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [2]: [hash_partition_key#X, c_custkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [2]: [hash_partition_key#X, c_custkey#X] @@ -117,9 +117,9 @@ Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] @@ -148,9 +148,9 @@ Input [5]: [c_custkey#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriorit Input [4]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X, o_shippriority#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [4]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X, o_shippriority#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [4]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X, o_shippriority#X] @@ -185,9 +185,9 @@ Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/4.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/4.txt index cc6b8f351600..eae6f66bbb18 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/4.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/4.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (56) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33) +- ColumnarExchange (32) - +- VeloxAppendBatches (31) + +- VeloxResizeBatches (31) +- ^ RegularHashAggregateExecTransformer (29) +- ^ InputIteratorTransformer (28) +- ShuffleQueryStage (26) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ ProjectExecTransformer (22) +- ^ FlushableHashAggregateExecTransformer (21) +- ^ ProjectExecTransformer (20) @@ -19,14 +19,14 @@ AdaptiveSparkPlan (56) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) +- ^ Scan parquet (10) @@ -69,9 +69,9 @@ Input [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] @@ -106,9 +106,9 @@ Input [3]: [l_orderkey#X, l_commitdate#X, l_receiptdate#X] Input [2]: [hash_partition_key#X, l_orderkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [2]: [hash_partition_key#X, l_orderkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [2]: [hash_partition_key#X, l_orderkey#X] @@ -148,9 +148,9 @@ Input [2]: [o_orderpriority#X, count#X] Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] @@ -177,9 +177,9 @@ Results [2]: [o_orderpriority#X, count(1)#X AS order_count#X] Input [2]: [o_orderpriority#X, order_count#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [2]: [o_orderpriority#X, order_count#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [2]: [o_orderpriority#X, order_count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/5.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/5.txt index a1f95887aae3..662616ac077a 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/5.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/5.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (156) +- ^ InputIteratorTransformer (103) +- ShuffleQueryStage (101) +- ColumnarExchange (100) - +- VeloxAppendBatches (99) + +- VeloxResizeBatches (99) +- ^ RegularHashAggregateExecTransformer (97) +- ^ InputIteratorTransformer (96) +- ShuffleQueryStage (94) +- ColumnarExchange (93) - +- VeloxAppendBatches (92) + +- VeloxResizeBatches (92) +- ^ ProjectExecTransformer (90) +- ^ FlushableHashAggregateExecTransformer (89) +- ^ ProjectExecTransformer (88) @@ -19,66 +19,66 @@ AdaptiveSparkPlan (156) :- ^ InputIteratorTransformer (77) : +- ShuffleQueryStage (75) : +- ColumnarExchange (74) - : +- VeloxAppendBatches (73) + : +- VeloxResizeBatches (73) : +- ^ ProjectExecTransformer (71) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (70) : :- ^ InputIteratorTransformer (60) : : +- ShuffleQueryStage (58) : : +- ColumnarExchange (57) - : : +- VeloxAppendBatches (56) + : : +- VeloxResizeBatches (56) : : +- ^ ProjectExecTransformer (54) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (53) : : :- ^ InputIteratorTransformer (43) : : : +- ShuffleQueryStage (41) : : : +- ColumnarExchange (40) - : : : +- VeloxAppendBatches (39) + : : : +- VeloxResizeBatches (39) : : : +- ^ ProjectExecTransformer (37) : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (36) : : : :- ^ InputIteratorTransformer (26) : : : : +- ShuffleQueryStage (24) : : : : +- ColumnarExchange (23) - : : : : +- VeloxAppendBatches (22) + : : : : +- VeloxResizeBatches (22) : : : : +- ^ ProjectExecTransformer (20) : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) : : : : :- ^ InputIteratorTransformer (9) : : : : : +- ShuffleQueryStage (7) : : : : : +- ColumnarExchange (6) - : : : : : +- VeloxAppendBatches (5) + : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ FilterExecTransformer (2) : : : : : +- ^ Scan parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16) : : : : +- ColumnarExchange (15) - : : : : +- VeloxAppendBatches (14) + : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) : : : : +- ^ Scan parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33) : : : +- ColumnarExchange (32) - : : : +- VeloxAppendBatches (31) + : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ FilterExecTransformer (28) : : : +- ^ Scan parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50) : : +- ColumnarExchange (49) - : : +- VeloxAppendBatches (48) + : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ FilterExecTransformer (45) : : +- ^ Scan parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67) : +- ColumnarExchange (66) - : +- VeloxAppendBatches (65) + : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ FilterExecTransformer (62) : +- ^ Scan parquet (61) +- ^ InputIteratorTransformer (86) +- ShuffleQueryStage (84) +- ColumnarExchange (83) - +- VeloxAppendBatches (82) + +- VeloxResizeBatches (82) +- ^ ProjectExecTransformer (80) +- ^ FilterExecTransformer (79) +- ^ Scan parquet (78) @@ -153,9 +153,9 @@ Input [2]: [c_custkey#X, c_nationkey#X] Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] @@ -190,9 +190,9 @@ Input [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] @@ -221,9 +221,9 @@ Input [4]: [c_custkey#X, c_nationkey#X, o_orderkey#X, o_custkey#X] Input [3]: [hash_partition_key#X, c_nationkey#X, o_orderkey#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_nationkey#X, o_orderkey#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [3]: [hash_partition_key#X, c_nationkey#X, o_orderkey#X] @@ -258,9 +258,9 @@ Input [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] @@ -289,9 +289,9 @@ Input [6]: [c_nationkey#X, o_orderkey#X, l_orderkey#X, l_suppkey#X, l_extendedpr Input [5]: [hash_partition_key#X, c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(39) VeloxAppendBatches +(39) VeloxResizeBatches Input [5]: [hash_partition_key#X, c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (40) ColumnarExchange Input [5]: [hash_partition_key#X, c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] @@ -326,9 +326,9 @@ Input [2]: [s_suppkey#X, s_nationkey#X] Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -357,9 +357,9 @@ Input [6]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, s_suppk Input [4]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X] Arguments: false -(56) VeloxAppendBatches +(56) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X] -Arguments: X +Arguments: X, X (57) ColumnarExchange Input [4]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X] @@ -394,9 +394,9 @@ Input [3]: [n_nationkey#X, n_name#X, n_regionkey#X] Input [4]: [hash_partition_key#X, n_nationkey#X, n_name#X, n_regionkey#X] Arguments: false -(65) VeloxAppendBatches +(65) VeloxResizeBatches Input [4]: [hash_partition_key#X, n_nationkey#X, n_name#X, n_regionkey#X] -Arguments: X +Arguments: X, X (66) ColumnarExchange Input [4]: [hash_partition_key#X, n_nationkey#X, n_name#X, n_regionkey#X] @@ -425,9 +425,9 @@ Input [6]: [l_extendedprice#X, l_discount#X, s_nationkey#X, n_nationkey#X, n_nam Input [5]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] Arguments: false -(73) VeloxAppendBatches +(73) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] -Arguments: X +Arguments: X, X (74) ColumnarExchange Input [5]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] @@ -462,9 +462,9 @@ Input [2]: [r_regionkey#X, r_name#X] Input [2]: [hash_partition_key#X, r_regionkey#X] Arguments: false -(82) VeloxAppendBatches +(82) VeloxResizeBatches Input [2]: [hash_partition_key#X, r_regionkey#X] -Arguments: X +Arguments: X, X (83) ColumnarExchange Input [2]: [hash_partition_key#X, r_regionkey#X] @@ -504,9 +504,9 @@ Input [3]: [n_name#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] Arguments: false -(92) VeloxAppendBatches +(92) VeloxResizeBatches Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (93) ColumnarExchange Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] @@ -533,9 +533,9 @@ Results [2]: [n_name#X, sum(CheckOverflow((promote_precision(cast(l_extendedpric Input [2]: [n_name#X, revenue#X] Arguments: false -(99) VeloxAppendBatches +(99) VeloxResizeBatches Input [2]: [n_name#X, revenue#X] -Arguments: X +Arguments: X, X (100) ColumnarExchange Input [2]: [n_name#X, revenue#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/6.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/6.txt index 786a89fe715a..ee8c494f56a3 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/6.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/6.txt @@ -6,7 +6,7 @@ AdaptiveSparkPlan (20) +- ^ InputIteratorTransformer (10) +- ShuffleQueryStage (8) +- ColumnarExchange (7) - +- VeloxAppendBatches (6) + +- VeloxResizeBatches (6) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ FilterExecTransformer (2) @@ -46,9 +46,9 @@ Results [2]: [sum#X, isEmpty#X] Input [2]: [sum#X, isEmpty#X] Arguments: false -(6) VeloxAppendBatches +(6) VeloxResizeBatches Input [2]: [sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (7) ColumnarExchange Input [2]: [sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/7.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/7.txt index 64d51413a084..ef168ad7bb8f 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/7.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/7.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (149) +- ^ InputIteratorTransformer (98) +- ShuffleQueryStage (96) +- ColumnarExchange (95) - +- VeloxAppendBatches (94) + +- VeloxResizeBatches (94) +- ^ RegularHashAggregateExecTransformer (92) +- ^ InputIteratorTransformer (91) +- ShuffleQueryStage (89) +- ColumnarExchange (88) - +- VeloxAppendBatches (87) + +- VeloxResizeBatches (87) +- ^ ProjectExecTransformer (85) +- ^ FlushableHashAggregateExecTransformer (84) +- ^ ProjectExecTransformer (83) @@ -19,59 +19,59 @@ AdaptiveSparkPlan (149) :- ^ InputIteratorTransformer (77) : +- ShuffleQueryStage (75) : +- ColumnarExchange (74) - : +- VeloxAppendBatches (73) + : +- VeloxResizeBatches (73) : +- ^ ProjectExecTransformer (71) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (70) : :- ^ InputIteratorTransformer (60) : : +- ShuffleQueryStage (58) : : +- ColumnarExchange (57) - : : +- VeloxAppendBatches (56) + : : +- VeloxResizeBatches (56) : : +- ^ ProjectExecTransformer (54) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (53) : : :- ^ InputIteratorTransformer (43) : : : +- ShuffleQueryStage (41) : : : +- ColumnarExchange (40) - : : : +- VeloxAppendBatches (39) + : : : +- VeloxResizeBatches (39) : : : +- ^ ProjectExecTransformer (37) : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (36) : : : :- ^ InputIteratorTransformer (26) : : : : +- ShuffleQueryStage (24) : : : : +- ColumnarExchange (23) - : : : : +- VeloxAppendBatches (22) + : : : : +- VeloxResizeBatches (22) : : : : +- ^ ProjectExecTransformer (20) : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) : : : : :- ^ InputIteratorTransformer (9) : : : : : +- ShuffleQueryStage (7) : : : : : +- ColumnarExchange (6) - : : : : : +- VeloxAppendBatches (5) + : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ FilterExecTransformer (2) : : : : : +- ^ Scan parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16) : : : : +- ColumnarExchange (15) - : : : : +- VeloxAppendBatches (14) + : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) : : : : +- ^ Scan parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33) : : : +- ColumnarExchange (32) - : : : +- VeloxAppendBatches (31) + : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ FilterExecTransformer (28) : : : +- ^ Scan parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50) : : +- ColumnarExchange (49) - : : +- VeloxAppendBatches (48) + : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ FilterExecTransformer (45) : : +- ^ Scan parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67) : +- ColumnarExchange (66) - : +- VeloxAppendBatches (65) + : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ FilterExecTransformer (62) : +- ^ Scan parquet (61) @@ -147,9 +147,9 @@ Input [2]: [s_suppkey#X, s_nationkey#X] Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -184,9 +184,9 @@ Input [5]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipda Input [6]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [6]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] @@ -215,9 +215,9 @@ Input [7]: [s_suppkey#X, s_nationkey#X, l_orderkey#X, l_suppkey#X, l_extendedpri Input [6]: [hash_partition_key#X, s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [6]: [hash_partition_key#X, s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [6]: [hash_partition_key#X, s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] @@ -252,9 +252,9 @@ Input [2]: [o_orderkey#X, o_custkey#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] @@ -283,9 +283,9 @@ Input [7]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_ship Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X] Arguments: false -(39) VeloxAppendBatches +(39) VeloxResizeBatches Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X] -Arguments: X +Arguments: X, X (40) ColumnarExchange Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X] @@ -320,9 +320,9 @@ Input [2]: [c_custkey#X, c_nationkey#X] Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] @@ -351,9 +351,9 @@ Input [7]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_cust Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X] Arguments: false -(56) VeloxAppendBatches +(56) VeloxResizeBatches Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X] -Arguments: X +Arguments: X, X (57) ColumnarExchange Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X] @@ -388,9 +388,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] Arguments: false -(65) VeloxAppendBatches +(65) VeloxResizeBatches Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] -Arguments: X +Arguments: X, X (66) ColumnarExchange Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] @@ -419,9 +419,9 @@ Input [7]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nati Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X, n_name#X] Arguments: false -(73) VeloxAppendBatches +(73) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X, n_name#X] -Arguments: X +Arguments: X, X (74) ColumnarExchange Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X, n_name#X] @@ -474,9 +474,9 @@ Input [5]: [supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] Arguments: false -(87) VeloxAppendBatches +(87) VeloxResizeBatches Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (88) ColumnarExchange Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] @@ -503,9 +503,9 @@ Results [4]: [supp_nation#X, cust_nation#X, l_year#X, sum(volume#X)#X AS revenue Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] Arguments: false -(94) VeloxAppendBatches +(94) VeloxResizeBatches Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] -Arguments: X +Arguments: X, X (95) ColumnarExchange Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/8.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/8.txt index 8934d1a2e7a3..55e4f19764d1 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/8.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/8.txt @@ -6,13 +6,13 @@ AdaptiveSparkPlan (207) +- ^ InputIteratorTransformer (138) +- ShuffleQueryStage (136) +- ColumnarExchange (135) - +- VeloxAppendBatches (134) + +- VeloxResizeBatches (134) +- ^ ProjectExecTransformer (132) +- ^ RegularHashAggregateExecTransformer (131) +- ^ InputIteratorTransformer (130) +- ShuffleQueryStage (128) +- ColumnarExchange (127) - +- VeloxAppendBatches (126) + +- VeloxResizeBatches (126) +- ^ ProjectExecTransformer (124) +- ^ FlushableHashAggregateExecTransformer (123) +- ^ ProjectExecTransformer (122) @@ -20,92 +20,92 @@ AdaptiveSparkPlan (207) :- ^ InputIteratorTransformer (111) : +- ShuffleQueryStage (109) : +- ColumnarExchange (108) - : +- VeloxAppendBatches (107) + : +- VeloxResizeBatches (107) : +- ^ ProjectExecTransformer (105) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (104) : :- ^ InputIteratorTransformer (94) : : +- ShuffleQueryStage (92) : : +- ColumnarExchange (91) - : : +- VeloxAppendBatches (90) + : : +- VeloxResizeBatches (90) : : +- ^ ProjectExecTransformer (88) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (87) : : :- ^ InputIteratorTransformer (77) : : : +- ShuffleQueryStage (75) : : : +- ColumnarExchange (74) - : : : +- VeloxAppendBatches (73) + : : : +- VeloxResizeBatches (73) : : : +- ^ ProjectExecTransformer (71) : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (70) : : : :- ^ InputIteratorTransformer (60) : : : : +- ShuffleQueryStage (58) : : : : +- ColumnarExchange (57) - : : : : +- VeloxAppendBatches (56) + : : : : +- VeloxResizeBatches (56) : : : : +- ^ ProjectExecTransformer (54) : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (53) : : : : :- ^ InputIteratorTransformer (43) : : : : : +- ShuffleQueryStage (41) : : : : : +- ColumnarExchange (40) - : : : : : +- VeloxAppendBatches (39) + : : : : : +- VeloxResizeBatches (39) : : : : : +- ^ ProjectExecTransformer (37) : : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (36) : : : : : :- ^ InputIteratorTransformer (26) : : : : : : +- ShuffleQueryStage (24) : : : : : : +- ColumnarExchange (23) - : : : : : : +- VeloxAppendBatches (22) + : : : : : : +- VeloxResizeBatches (22) : : : : : : +- ^ ProjectExecTransformer (20) : : : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) : : : : : : :- ^ InputIteratorTransformer (9) : : : : : : : +- ShuffleQueryStage (7) : : : : : : : +- ColumnarExchange (6) - : : : : : : : +- VeloxAppendBatches (5) + : : : : : : : +- VeloxResizeBatches (5) : : : : : : : +- ^ ProjectExecTransformer (3) : : : : : : : +- ^ FilterExecTransformer (2) : : : : : : : +- ^ Scan parquet (1) : : : : : : +- ^ InputIteratorTransformer (18) : : : : : : +- ShuffleQueryStage (16) : : : : : : +- ColumnarExchange (15) - : : : : : : +- VeloxAppendBatches (14) + : : : : : : +- VeloxResizeBatches (14) : : : : : : +- ^ ProjectExecTransformer (12) : : : : : : +- ^ FilterExecTransformer (11) : : : : : : +- ^ Scan parquet (10) : : : : : +- ^ InputIteratorTransformer (35) : : : : : +- ShuffleQueryStage (33) : : : : : +- ColumnarExchange (32) - : : : : : +- VeloxAppendBatches (31) + : : : : : +- VeloxResizeBatches (31) : : : : : +- ^ ProjectExecTransformer (29) : : : : : +- ^ FilterExecTransformer (28) : : : : : +- ^ Scan parquet (27) : : : : +- ^ InputIteratorTransformer (52) : : : : +- ShuffleQueryStage (50) : : : : +- ColumnarExchange (49) - : : : : +- VeloxAppendBatches (48) + : : : : +- VeloxResizeBatches (48) : : : : +- ^ ProjectExecTransformer (46) : : : : +- ^ FilterExecTransformer (45) : : : : +- ^ Scan parquet (44) : : : +- ^ InputIteratorTransformer (69) : : : +- ShuffleQueryStage (67) : : : +- ColumnarExchange (66) - : : : +- VeloxAppendBatches (65) + : : : +- VeloxResizeBatches (65) : : : +- ^ ProjectExecTransformer (63) : : : +- ^ FilterExecTransformer (62) : : : +- ^ Scan parquet (61) : : +- ^ InputIteratorTransformer (86) : : +- ShuffleQueryStage (84) : : +- ColumnarExchange (83) - : : +- VeloxAppendBatches (82) + : : +- VeloxResizeBatches (82) : : +- ^ ProjectExecTransformer (80) : : +- ^ FilterExecTransformer (79) : : +- ^ Scan parquet (78) : +- ^ InputIteratorTransformer (103) : +- ShuffleQueryStage (101) : +- ColumnarExchange (100) - : +- VeloxAppendBatches (99) + : +- VeloxResizeBatches (99) : +- ^ ProjectExecTransformer (97) : +- ^ FilterExecTransformer (96) : +- ^ Scan parquet (95) +- ^ InputIteratorTransformer (120) +- ShuffleQueryStage (118) +- ColumnarExchange (117) - +- VeloxAppendBatches (116) + +- VeloxResizeBatches (116) +- ^ ProjectExecTransformer (114) +- ^ FilterExecTransformer (113) +- ^ Scan parquet (112) @@ -196,9 +196,9 @@ Input [2]: [p_partkey#X, p_type#X] Input [2]: [hash_partition_key#X, p_partkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [2]: [hash_partition_key#X, p_partkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [2]: [hash_partition_key#X, p_partkey#X] @@ -233,9 +233,9 @@ Input [5]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discoun Input [6]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [6]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] @@ -264,9 +264,9 @@ Input [6]: [p_partkey#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] @@ -301,9 +301,9 @@ Input [2]: [s_suppkey#X, s_nationkey#X] Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -332,9 +332,9 @@ Input [6]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, s_suppke Input [5]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] Arguments: false -(39) VeloxAppendBatches +(39) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] -Arguments: X +Arguments: X, X (40) ColumnarExchange Input [5]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] @@ -369,9 +369,9 @@ Input [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Input [4]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [4]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [4]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X] @@ -400,9 +400,9 @@ Input [7]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orde Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X] Arguments: false -(56) VeloxAppendBatches +(56) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X] -Arguments: X +Arguments: X, X (57) ColumnarExchange Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X] @@ -437,9 +437,9 @@ Input [2]: [c_custkey#X, c_nationkey#X] Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] Arguments: false -(65) VeloxAppendBatches +(65) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] -Arguments: X +Arguments: X, X (66) ColumnarExchange Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] @@ -468,9 +468,9 @@ Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_order Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X] Arguments: false -(73) VeloxAppendBatches +(73) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X] -Arguments: X +Arguments: X, X (74) ColumnarExchange Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X] @@ -505,9 +505,9 @@ Input [2]: [n_nationkey#X, n_regionkey#X] Input [3]: [hash_partition_key#X, n_nationkey#X, n_regionkey#X] Arguments: false -(82) VeloxAppendBatches +(82) VeloxResizeBatches Input [3]: [hash_partition_key#X, n_nationkey#X, n_regionkey#X] -Arguments: X +Arguments: X, X (83) ColumnarExchange Input [3]: [hash_partition_key#X, n_nationkey#X, n_regionkey#X] @@ -536,9 +536,9 @@ Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nat Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X] Arguments: false -(90) VeloxAppendBatches +(90) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X] -Arguments: X +Arguments: X, X (91) ColumnarExchange Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X] @@ -573,9 +573,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] Arguments: false -(99) VeloxAppendBatches +(99) VeloxResizeBatches Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] -Arguments: X +Arguments: X, X (100) ColumnarExchange Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] @@ -604,9 +604,9 @@ Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_reg Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X] Arguments: false -(107) VeloxAppendBatches +(107) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X] -Arguments: X +Arguments: X, X (108) ColumnarExchange Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X] @@ -641,9 +641,9 @@ Input [2]: [r_regionkey#X, r_name#X] Input [2]: [hash_partition_key#X, r_regionkey#X] Arguments: false -(116) VeloxAppendBatches +(116) VeloxResizeBatches Input [2]: [hash_partition_key#X, r_regionkey#X] -Arguments: X +Arguments: X, X (117) ColumnarExchange Input [2]: [hash_partition_key#X, r_regionkey#X] @@ -683,9 +683,9 @@ Input [5]: [o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] Arguments: false -(126) VeloxAppendBatches +(126) VeloxResizeBatches Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (127) ColumnarExchange Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] @@ -716,9 +716,9 @@ Input [3]: [o_year#X, sum(CASE WHEN (nation#X = BRAZIL) THEN volume#X ELSE 0.000 Input [2]: [o_year#X, mkt_share#X] Arguments: false -(134) VeloxAppendBatches +(134) VeloxResizeBatches Input [2]: [o_year#X, mkt_share#X] -Arguments: X +Arguments: X, X (135) ColumnarExchange Input [2]: [o_year#X, mkt_share#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/9.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/9.txt index cb207c0800c3..f07b5ce81c14 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/9.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/9.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (155) +- ^ InputIteratorTransformer (103) +- ShuffleQueryStage (101) +- ColumnarExchange (100) - +- VeloxAppendBatches (99) + +- VeloxResizeBatches (99) +- ^ RegularHashAggregateExecTransformer (97) +- ^ InputIteratorTransformer (96) +- ShuffleQueryStage (94) +- ColumnarExchange (93) - +- VeloxAppendBatches (92) + +- VeloxResizeBatches (92) +- ^ ProjectExecTransformer (90) +- ^ FlushableHashAggregateExecTransformer (89) +- ^ ProjectExecTransformer (88) @@ -19,66 +19,66 @@ AdaptiveSparkPlan (155) :- ^ InputIteratorTransformer (77) : +- ShuffleQueryStage (75) : +- ColumnarExchange (74) - : +- VeloxAppendBatches (73) + : +- VeloxResizeBatches (73) : +- ^ ProjectExecTransformer (71) : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (70) : :- ^ InputIteratorTransformer (60) : : +- ShuffleQueryStage (58) : : +- ColumnarExchange (57) - : : +- VeloxAppendBatches (56) + : : +- VeloxResizeBatches (56) : : +- ^ ProjectExecTransformer (54) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (53) : : :- ^ InputIteratorTransformer (43) : : : +- ShuffleQueryStage (41) : : : +- ColumnarExchange (40) - : : : +- VeloxAppendBatches (39) + : : : +- VeloxResizeBatches (39) : : : +- ^ ProjectExecTransformer (37) : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (36) : : : :- ^ InputIteratorTransformer (26) : : : : +- ShuffleQueryStage (24) : : : : +- ColumnarExchange (23) - : : : : +- VeloxAppendBatches (22) + : : : : +- VeloxResizeBatches (22) : : : : +- ^ ProjectExecTransformer (20) : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) : : : : :- ^ InputIteratorTransformer (9) : : : : : +- ShuffleQueryStage (7) : : : : : +- ColumnarExchange (6) - : : : : : +- VeloxAppendBatches (5) + : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ FilterExecTransformer (2) : : : : : +- ^ Scan parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16) : : : : +- ColumnarExchange (15) - : : : : +- VeloxAppendBatches (14) + : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) : : : : +- ^ Scan parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33) : : : +- ColumnarExchange (32) - : : : +- VeloxAppendBatches (31) + : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ FilterExecTransformer (28) : : : +- ^ Scan parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50) : : +- ColumnarExchange (49) - : : +- VeloxAppendBatches (48) + : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ FilterExecTransformer (45) : : +- ^ Scan parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67) : +- ColumnarExchange (66) - : +- VeloxAppendBatches (65) + : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ FilterExecTransformer (62) : +- ^ Scan parquet (61) +- ^ InputIteratorTransformer (86) +- ShuffleQueryStage (84) +- ColumnarExchange (83) - +- VeloxAppendBatches (82) + +- VeloxResizeBatches (82) +- ^ ProjectExecTransformer (80) +- ^ FilterExecTransformer (79) +- ^ Scan parquet (78) @@ -152,9 +152,9 @@ Input [2]: [p_partkey#X, p_name#X] Input [2]: [hash_partition_key#X, p_partkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [2]: [hash_partition_key#X, p_partkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [2]: [hash_partition_key#X, p_partkey#X] @@ -189,9 +189,9 @@ Input [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpric Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] @@ -220,9 +220,9 @@ Input [7]: [p_partkey#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] @@ -257,9 +257,9 @@ Input [2]: [s_suppkey#X, s_nationkey#X] Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -288,9 +288,9 @@ Input [8]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpric Input [8]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X] Arguments: false -(39) VeloxAppendBatches +(39) VeloxResizeBatches Input [8]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X] -Arguments: X +Arguments: X, X (40) ColumnarExchange Input [8]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X] @@ -325,9 +325,9 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_supplycost#X] Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_supplycost#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_supplycost#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_supplycost#X] @@ -356,9 +356,9 @@ Input [10]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpri Input [7]: [hash_partition_key#X, l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X] Arguments: false -(56) VeloxAppendBatches +(56) VeloxResizeBatches Input [7]: [hash_partition_key#X, l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X] -Arguments: X +Arguments: X, X (57) ColumnarExchange Input [7]: [hash_partition_key#X, l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X] @@ -393,9 +393,9 @@ Input [2]: [o_orderkey#X, o_orderdate#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X] Arguments: false -(65) VeloxAppendBatches +(65) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X] -Arguments: X +Arguments: X, X (66) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X] @@ -424,9 +424,9 @@ Input [8]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_natio Input [7]: [hash_partition_key#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X] Arguments: false -(73) VeloxAppendBatches +(73) VeloxResizeBatches Input [7]: [hash_partition_key#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X] -Arguments: X +Arguments: X, X (74) ColumnarExchange Input [7]: [hash_partition_key#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X] @@ -461,9 +461,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] Arguments: false -(82) VeloxAppendBatches +(82) VeloxResizeBatches Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] -Arguments: X +Arguments: X, X (83) ColumnarExchange Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] @@ -503,9 +503,9 @@ Input [4]: [nation#X, o_year#X, sum#X, isEmpty#X] Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] Arguments: false -(92) VeloxAppendBatches +(92) VeloxResizeBatches Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (93) ColumnarExchange Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] @@ -532,9 +532,9 @@ Results [3]: [nation#X, o_year#X, sum(amount#X)#X AS sum_profit#X] Input [3]: [nation#X, o_year#X, sum_profit#X] Arguments: false -(99) VeloxAppendBatches +(99) VeloxResizeBatches Input [3]: [nation#X, o_year#X, sum_profit#X] -Arguments: X +Arguments: X, X (100) ColumnarExchange Input [3]: [nation#X, o_year#X, sum_profit#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/1.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/1.txt index 22dd5100c4fb..12c19c45e38d 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/1.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/1.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (30) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ RegularHashAggregateExecTransformer (12) +- ^ InputIteratorTransformer (11) +- ShuffleQueryStage (9), Statistics(X) +- ColumnarExchange (8) - +- VeloxAppendBatches (7) + +- VeloxResizeBatches (7) +- ^ ProjectExecTransformer (5) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) @@ -58,9 +58,9 @@ Input [17]: [l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] Arguments: false -(7) VeloxAppendBatches +(7) VeloxResizeBatches Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] -Arguments: X +Arguments: X, X (8) ColumnarExchange Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] @@ -87,9 +87,9 @@ Results [10]: [l_returnflag#X, l_linestatus#X, sum(l_quantity#X)#X AS sum_qty#X, Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/10.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/10.txt index c295515b8a6c..d26861c8bcb3 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/10.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/10.txt @@ -8,7 +8,7 @@ AdaptiveSparkPlan (100) +- ^ InputIteratorTransformer (62) +- ShuffleQueryStage (60), Statistics(X) +- ColumnarExchange (59) - +- VeloxAppendBatches (58) + +- VeloxResizeBatches (58) +- ^ ProjectExecTransformer (56) +- ^ FlushableHashAggregateExecTransformer (55) +- ^ ProjectExecTransformer (54) @@ -16,40 +16,40 @@ AdaptiveSparkPlan (100) :- ^ InputIteratorTransformer (43) : +- ShuffleQueryStage (41), Statistics(X) : +- ColumnarExchange (40) - : +- VeloxAppendBatches (39) + : +- VeloxResizeBatches (39) : +- ^ ProjectExecTransformer (37) : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (36) : :- ^ InputIteratorTransformer (26) : : +- ShuffleQueryStage (24), Statistics(X) : : +- ColumnarExchange (23) - : : +- VeloxAppendBatches (22) + : : +- VeloxResizeBatches (22) : : +- ^ ProjectExecTransformer (20) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) : : :- ^ InputIteratorTransformer (9) : : : +- ShuffleQueryStage (7), Statistics(X) : : : +- ColumnarExchange (6) - : : : +- VeloxAppendBatches (5) + : : : +- VeloxResizeBatches (5) : : : +- ^ ProjectExecTransformer (3) : : : +- ^ FilterExecTransformer (2) : : : +- ^ Scan parquet (1) : : +- ^ InputIteratorTransformer (18) : : +- ShuffleQueryStage (16), Statistics(X) : : +- ColumnarExchange (15) - : : +- VeloxAppendBatches (14) + : : +- VeloxResizeBatches (14) : : +- ^ ProjectExecTransformer (12) : : +- ^ FilterExecTransformer (11) : : +- ^ Scan parquet (10) : +- ^ InputIteratorTransformer (35) : +- ShuffleQueryStage (33), Statistics(X) : +- ColumnarExchange (32) - : +- VeloxAppendBatches (31) + : +- VeloxResizeBatches (31) : +- ^ ProjectExecTransformer (29) : +- ^ FilterExecTransformer (28) : +- ^ Scan parquet (27) +- ^ InputIteratorTransformer (52) +- ShuffleQueryStage (50), Statistics(X) +- ColumnarExchange (49) - +- VeloxAppendBatches (48) + +- VeloxResizeBatches (48) +- ^ ProjectExecTransformer (46) +- ^ FilterExecTransformer (45) +- ^ Scan parquet (44) @@ -107,9 +107,9 @@ Input [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct Input [8]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [8]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [8]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] @@ -144,9 +144,9 @@ Input [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] @@ -175,9 +175,9 @@ Input [9]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct Input [9]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, o_orderkey#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [9]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, o_orderkey#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [9]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, o_orderkey#X] @@ -212,9 +212,9 @@ Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_returnflag#X] Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] @@ -243,9 +243,9 @@ Input [11]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acc Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, l_extendedprice#X, l_discount#X] Arguments: false -(39) VeloxAppendBatches +(39) VeloxResizeBatches Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (40) ColumnarExchange Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, l_extendedprice#X, l_discount#X] @@ -280,9 +280,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] @@ -322,9 +322,9 @@ Input [9]: [c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] Arguments: false -(58) VeloxAppendBatches +(58) VeloxResizeBatches Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (59) ColumnarExchange Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/11.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/11.txt index 20bb486f3841..5dee0fa9091b 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/11.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/11.txt @@ -6,13 +6,13 @@ AdaptiveSparkPlan (82) +- ^ InputIteratorTransformer (53) +- ShuffleQueryStage (51), Statistics(X) +- ColumnarExchange (50) - +- VeloxAppendBatches (49) + +- VeloxResizeBatches (49) +- ^ FilterExecTransformer (47) +- ^ RegularHashAggregateExecTransformer (46) +- ^ InputIteratorTransformer (45) +- ShuffleQueryStage (43), Statistics(X) +- ColumnarExchange (42) - +- VeloxAppendBatches (41) + +- VeloxResizeBatches (41) +- ^ ProjectExecTransformer (39) +- ^ FlushableHashAggregateExecTransformer (38) +- ^ ProjectExecTransformer (37) @@ -20,27 +20,27 @@ AdaptiveSparkPlan (82) :- ^ InputIteratorTransformer (26) : +- ShuffleQueryStage (24), Statistics(X) : +- ColumnarExchange (23) - : +- VeloxAppendBatches (22) + : +- VeloxResizeBatches (22) : +- ^ ProjectExecTransformer (20) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7), Statistics(X) : : +- ColumnarExchange (6) - : : +- VeloxAppendBatches (5) + : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16), Statistics(X) : +- ColumnarExchange (15) - : +- VeloxAppendBatches (14) + : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ FilterExecTransformer (11) : +- ^ Scan parquet (10) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33), Statistics(X) +- ColumnarExchange (32) - +- VeloxAppendBatches (31) + +- VeloxResizeBatches (31) +- ^ ProjectExecTransformer (29) +- ^ FilterExecTransformer (28) +- ^ Scan parquet (27) @@ -91,9 +91,9 @@ Input [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Input [5]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [5]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [5]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] @@ -128,9 +128,9 @@ Input [2]: [s_suppkey#X, s_nationkey#X] Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -159,9 +159,9 @@ Input [6]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X, s_suppke Input [5]: [hash_partition_key#X, ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [5]: [hash_partition_key#X, ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [5]: [hash_partition_key#X, ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] @@ -196,9 +196,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [2]: [hash_partition_key#X, n_nationkey#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [2]: [hash_partition_key#X, n_nationkey#X] @@ -238,9 +238,9 @@ Input [3]: [ps_partkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] Arguments: false -(41) VeloxAppendBatches +(41) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (42) ColumnarExchange Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] @@ -271,9 +271,9 @@ Arguments: (isnotnull(value#X) AND (cast(value#X as decimal(38,6)) > Subquery su Input [2]: [ps_partkey#X, value#X] Arguments: false -(49) VeloxAppendBatches +(49) VeloxResizeBatches Input [2]: [ps_partkey#X, value#X] -Arguments: X +Arguments: X, X (50) ColumnarExchange Input [2]: [ps_partkey#X, value#X] @@ -435,13 +435,13 @@ AdaptiveSparkPlan (136) :- ^ InputIteratorTransformer (103) : +- ShuffleQueryStage (101), Statistics(X) : +- ColumnarExchange (100) - : +- VeloxAppendBatches (99) + : +- VeloxResizeBatches (99) : +- ^ ProjectExecTransformer (97) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (96) : :- ^ InputIteratorTransformer (91) : : +- ShuffleQueryStage (89), Statistics(X) : : +- ColumnarExchange (88) - : : +- VeloxAppendBatches (87) + : : +- VeloxResizeBatches (87) : : +- ^ ProjectExecTransformer (85) : : +- ^ FilterExecTransformer (84) : : +- ^ Scan parquet (83) @@ -494,9 +494,9 @@ Input [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Input [4]: [hash_partition_key#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Arguments: false -(87) VeloxAppendBatches +(87) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] -Arguments: X +Arguments: X, X (88) ColumnarExchange Input [4]: [hash_partition_key#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] @@ -538,9 +538,9 @@ Input [5]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X, s_suppkey#X, s_nationk Input [4]: [hash_partition_key#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] Arguments: false -(99) VeloxAppendBatches +(99) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] -Arguments: X +Arguments: X, X (100) ColumnarExchange Input [4]: [hash_partition_key#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/12.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/12.txt index 1b36d274aab4..4545acbc282a 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/12.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/12.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (55) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33), Statistics(X) +- ColumnarExchange (32) - +- VeloxAppendBatches (31) + +- VeloxResizeBatches (31) +- ^ RegularHashAggregateExecTransformer (29) +- ^ InputIteratorTransformer (28) +- ShuffleQueryStage (26), Statistics(X) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ ProjectExecTransformer (22) +- ^ FlushableHashAggregateExecTransformer (21) +- ^ ProjectExecTransformer (20) @@ -19,14 +19,14 @@ AdaptiveSparkPlan (55) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7), Statistics(X) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) +- ^ Scan parquet (10) @@ -68,9 +68,9 @@ Input [2]: [o_orderkey#X, o_orderpriority#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] @@ -105,9 +105,9 @@ Input [5]: [l_orderkey#X, l_shipdate#X, l_commitdate#X, l_receiptdate#X, l_shipm Input [3]: [hash_partition_key#X, l_orderkey#X, l_shipmode#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_shipmode#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_shipmode#X] @@ -147,9 +147,9 @@ Input [3]: [l_shipmode#X, sum#X, sum#X] Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] @@ -176,9 +176,9 @@ Results [3]: [l_shipmode#X, sum(CASE WHEN ((o_orderpriority#X = 1-URGENT) OR (o_ Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/13.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/13.txt index 83ec9aeda98a..758ece2d8fce 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/13.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/13.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (58) +- ^ InputIteratorTransformer (37) +- ShuffleQueryStage (35), Statistics(X) +- ColumnarExchange (34) - +- VeloxAppendBatches (33) + +- VeloxResizeBatches (33) +- ^ RegularHashAggregateExecTransformer (31) +- ^ InputIteratorTransformer (30) +- ShuffleQueryStage (28), Statistics(X) +- ColumnarExchange (27) - +- VeloxAppendBatches (26) + +- VeloxResizeBatches (26) +- ^ ProjectExecTransformer (24) +- ^ FlushableHashAggregateExecTransformer (23) +- ^ ProjectExecTransformer (22) @@ -22,13 +22,13 @@ AdaptiveSparkPlan (58) :- ^ InputIteratorTransformer (8) : +- ShuffleQueryStage (6), Statistics(X) : +- ColumnarExchange (5) - : +- VeloxAppendBatches (4) + : +- VeloxResizeBatches (4) : +- ^ ProjectExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (17) +- ShuffleQueryStage (15), Statistics(X) +- ColumnarExchange (14) - +- VeloxAppendBatches (13) + +- VeloxResizeBatches (13) +- ^ ProjectExecTransformer (11) +- ^ FilterExecTransformer (10) +- ^ Scan parquet (9) @@ -66,9 +66,9 @@ Input [1]: [c_custkey#X] Input [2]: [hash_partition_key#X, c_custkey#X] Arguments: false -(4) VeloxAppendBatches +(4) VeloxResizeBatches Input [2]: [hash_partition_key#X, c_custkey#X] -Arguments: X +Arguments: X, X (5) ColumnarExchange Input [2]: [hash_partition_key#X, c_custkey#X] @@ -103,9 +103,9 @@ Input [3]: [o_orderkey#X, o_custkey#X, o_comment#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] Arguments: false -(13) VeloxAppendBatches +(13) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] -Arguments: X +Arguments: X, X (14) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] @@ -163,9 +163,9 @@ Input [2]: [c_count#X, count#X] Input [3]: [hash_partition_key#X, c_count#X, count#X] Arguments: false -(26) VeloxAppendBatches +(26) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_count#X, count#X] -Arguments: X +Arguments: X, X (27) ColumnarExchange Input [3]: [hash_partition_key#X, c_count#X, count#X] @@ -192,9 +192,9 @@ Results [2]: [c_count#X, count(1)#X AS custdist#X] Input [2]: [c_count#X, custdist#X] Arguments: false -(33) VeloxAppendBatches +(33) VeloxResizeBatches Input [2]: [c_count#X, custdist#X] -Arguments: X +Arguments: X, X (34) ColumnarExchange Input [2]: [c_count#X, custdist#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/14.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/14.txt index a1108606b5bb..ce943bd043e0 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/14.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/14.txt @@ -10,14 +10,14 @@ AdaptiveSparkPlan (39) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7), Statistics(X) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) +- ^ Scan parquet (10) @@ -56,9 +56,9 @@ Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Input [4]: [hash_partition_key#X, l_partkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_partkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [4]: [hash_partition_key#X, l_partkey#X, l_extendedprice#X, l_discount#X] @@ -93,9 +93,9 @@ Input [2]: [p_partkey#X, p_type#X] Input [3]: [hash_partition_key#X, p_partkey#X, p_type#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, p_partkey#X, p_type#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, p_partkey#X, p_type#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/15.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/15.txt index 88730deb3c32..62b0047bcc06 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/15.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/15.txt @@ -5,13 +5,13 @@ AdaptiveSparkPlan (47) +- AQEShuffleRead (29) +- ShuffleQueryStage (28), Statistics(X) +- ColumnarExchange (27) - +- VeloxAppendBatches (26) + +- VeloxResizeBatches (26) +- ^ ProjectExecTransformer (24) +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (23) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7), Statistics(X) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) @@ -20,7 +20,7 @@ AdaptiveSparkPlan (47) +- ^ InputIteratorTransformer (20) +- ShuffleQueryStage (18), Statistics(X) +- ColumnarExchange (17) - +- VeloxAppendBatches (16) + +- VeloxResizeBatches (16) +- ^ ProjectExecTransformer (14) +- ^ FlushableHashAggregateExecTransformer (13) +- ^ ProjectExecTransformer (12) @@ -64,9 +64,9 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_phone#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_phone#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_phone#X] @@ -112,9 +112,9 @@ Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] Arguments: false -(16) VeloxAppendBatches +(16) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (17) ColumnarExchange Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] @@ -154,9 +154,9 @@ Input [6]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, supplier_no#X, total_ Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] Arguments: false -(26) VeloxAppendBatches +(26) VeloxResizeBatches Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] -Arguments: X +Arguments: X, X (27) ColumnarExchange Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] @@ -267,7 +267,7 @@ AdaptiveSparkPlan (73) +- ^ InputIteratorTransformer (58) +- ShuffleQueryStage (56), Statistics(X) +- ColumnarExchange (55) - +- VeloxAppendBatches (54) + +- VeloxResizeBatches (54) +- ^ ProjectExecTransformer (52) +- ^ FlushableHashAggregateExecTransformer (51) +- ^ ProjectExecTransformer (50) @@ -314,9 +314,9 @@ Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] Arguments: false -(54) VeloxAppendBatches +(54) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (55) ColumnarExchange Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/16.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/16.txt index 535b6940301d..573dfb3514f5 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/16.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/16.txt @@ -6,19 +6,19 @@ AdaptiveSparkPlan (71) +- ^ InputIteratorTransformer (44) +- ShuffleQueryStage (42), Statistics(X) +- ColumnarExchange (41) - +- VeloxAppendBatches (40) + +- VeloxResizeBatches (40) +- ^ RegularHashAggregateExecTransformer (38) +- ^ InputIteratorTransformer (37) +- ShuffleQueryStage (35), Statistics(X) +- ColumnarExchange (34) - +- VeloxAppendBatches (33) + +- VeloxResizeBatches (33) +- ^ ProjectExecTransformer (31) +- ^ FlushableHashAggregateExecTransformer (30) +- ^ RegularHashAggregateExecTransformer (29) +- ^ InputIteratorTransformer (28) +- ShuffleQueryStage (26), Statistics(X) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ ProjectExecTransformer (22) +- ^ FlushableHashAggregateExecTransformer (21) +- ^ ProjectExecTransformer (20) @@ -26,14 +26,14 @@ AdaptiveSparkPlan (71) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7), Statistics(X) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) +- ^ Scan parquet (10) @@ -82,9 +82,9 @@ Input [2]: [ps_partkey#X, ps_suppkey#X] Input [3]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X] @@ -119,9 +119,9 @@ Input [4]: [p_partkey#X, p_brand#X, p_type#X, p_size#X] Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_type#X, p_size#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_type#X, p_size#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_type#X, p_size#X] @@ -161,9 +161,9 @@ Input [4]: [p_brand#X, p_type#X, p_size#X, ps_suppkey#X] Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] @@ -201,9 +201,9 @@ Input [4]: [p_brand#X, p_type#X, p_size#X, count#X] Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] Arguments: false -(33) VeloxAppendBatches +(33) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] -Arguments: X +Arguments: X, X (34) ColumnarExchange Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] @@ -230,9 +230,9 @@ Results [4]: [p_brand#X, p_type#X, p_size#X, count(ps_suppkey#X)#X AS supplier_c Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] Arguments: false -(40) VeloxAppendBatches +(40) VeloxResizeBatches Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] -Arguments: X +Arguments: X, X (41) ColumnarExchange Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/17.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/17.txt index d360b6c948e3..7e1d41ca4251 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/17.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/17.txt @@ -12,14 +12,14 @@ AdaptiveSparkPlan (63) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7), Statistics(X) : : +- ColumnarExchange (6) - : : +- VeloxAppendBatches (5) + : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16), Statistics(X) : +- ColumnarExchange (15) - : +- VeloxAppendBatches (14) + : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ FilterExecTransformer (11) : +- ^ Scan parquet (10) @@ -29,7 +29,7 @@ AdaptiveSparkPlan (63) +- ^ InputIteratorTransformer (30) +- ShuffleQueryStage (28), Statistics(X) +- ColumnarExchange (27) - +- VeloxAppendBatches (26) + +- VeloxResizeBatches (26) +- ^ ProjectExecTransformer (24) +- ^ FlushableHashAggregateExecTransformer (23) +- ^ FilterExecTransformer (22) @@ -78,9 +78,9 @@ Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Input [4]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [4]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X] @@ -115,9 +115,9 @@ Input [3]: [p_partkey#X, p_brand#X, p_container#X] Input [2]: [hash_partition_key#X, p_partkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [2]: [hash_partition_key#X, p_partkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [2]: [hash_partition_key#X, p_partkey#X] @@ -168,9 +168,9 @@ Input [3]: [l_partkey#X, sum#X, count#X] Input [4]: [hash_partition_key#X, l_partkey#X, sum#X, count#X] Arguments: false -(26) VeloxAppendBatches +(26) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_partkey#X, sum#X, count#X] -Arguments: X +Arguments: X, X (27) ColumnarExchange Input [4]: [hash_partition_key#X, l_partkey#X, sum#X, count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/18.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/18.txt index a664adfd3175..42b2e6f76291 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/18.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/18.txt @@ -10,26 +10,26 @@ AdaptiveSparkPlan (110) :- ^ InputIteratorTransformer (46) : +- ShuffleQueryStage (44), Statistics(X) : +- ColumnarExchange (43) - : +- VeloxAppendBatches (42) + : +- VeloxResizeBatches (42) : +- ^ ProjectExecTransformer (40) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (39) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7), Statistics(X) : : +- ColumnarExchange (6) - : : +- VeloxAppendBatches (5) + : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (38) : +- ShuffleQueryStage (36), Statistics(X) : +- ColumnarExchange (35) - : +- VeloxAppendBatches (34) + : +- VeloxResizeBatches (34) : +- ^ ProjectExecTransformer (32) : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (31) : :- ^ InputIteratorTransformer (18) : : +- ShuffleQueryStage (16), Statistics(X) : : +- ColumnarExchange (15) - : : +- VeloxAppendBatches (14) + : : +- VeloxResizeBatches (14) : : +- ^ ProjectExecTransformer (12) : : +- ^ FilterExecTransformer (11) : : +- ^ Scan parquet (10) @@ -39,7 +39,7 @@ AdaptiveSparkPlan (110) : +- ^ InputIteratorTransformer (27) : +- ShuffleQueryStage (25), Statistics(X) : +- ColumnarExchange (24) - : +- VeloxAppendBatches (23) + : +- VeloxResizeBatches (23) : +- ^ ProjectExecTransformer (21) : +- ^ FlushableHashAggregateExecTransformer (20) : +- ^ Scan parquet (19) @@ -47,7 +47,7 @@ AdaptiveSparkPlan (110) :- ^ InputIteratorTransformer (55) : +- ShuffleQueryStage (53), Statistics(X) : +- ColumnarExchange (52) - : +- VeloxAppendBatches (51) + : +- VeloxResizeBatches (51) : +- ^ ProjectExecTransformer (49) : +- ^ FilterExecTransformer (48) : +- ^ Scan parquet (47) @@ -118,9 +118,9 @@ Input [2]: [c_custkey#X, c_name#X] Input [3]: [hash_partition_key#X, c_custkey#X, c_name#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_custkey#X, c_name#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, c_custkey#X, c_name#X] @@ -155,9 +155,9 @@ Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] @@ -194,9 +194,9 @@ Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] Arguments: false -(23) VeloxAppendBatches +(23) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (24) ColumnarExchange Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] @@ -240,9 +240,9 @@ Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: false -(34) VeloxAppendBatches +(34) VeloxResizeBatches Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] -Arguments: X +Arguments: X, X (35) ColumnarExchange Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] @@ -271,9 +271,9 @@ Input [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_ Input [6]: [hash_partition_key#X, c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] Arguments: false -(42) VeloxAppendBatches +(42) VeloxResizeBatches Input [6]: [hash_partition_key#X, c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] -Arguments: X +Arguments: X, X (43) ColumnarExchange Input [6]: [hash_partition_key#X, c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] @@ -308,9 +308,9 @@ Input [2]: [l_orderkey#X, l_quantity#X] Input [3]: [hash_partition_key#X, l_orderkey#X, l_quantity#X] Arguments: false -(51) VeloxAppendBatches +(51) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_quantity#X] -Arguments: X +Arguments: X, X (52) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_quantity#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/19.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/19.txt index 58e80362020f..d05ba10a15a6 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/19.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/19.txt @@ -9,14 +9,14 @@ AdaptiveSparkPlan (38) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7), Statistics(X) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) +- ^ Scan parquet (10) @@ -55,9 +55,9 @@ Input [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipin Input [5]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [5]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] @@ -92,9 +92,9 @@ Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_size#X, p_container#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_size#X, p_container#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_size#X, p_container#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/20.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/20.txt index c22b822e6f7d..d74e9ecde999 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/20.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/20.txt @@ -5,52 +5,52 @@ AdaptiveSparkPlan (143) +- AQEShuffleRead (92) +- ShuffleQueryStage (91), Statistics(X) +- ColumnarExchange (90) - +- VeloxAppendBatches (89) + +- VeloxResizeBatches (89) +- ^ ProjectExecTransformer (87) +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (86) :- ^ InputIteratorTransformer (76) : +- ShuffleQueryStage (74), Statistics(X) : +- ColumnarExchange (73) - : +- VeloxAppendBatches (72) + : +- VeloxResizeBatches (72) : +- ^ ProjectExecTransformer (70) : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (69) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7), Statistics(X) : : +- ColumnarExchange (6) - : : +- VeloxAppendBatches (5) + : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (68) : +- ShuffleQueryStage (66), Statistics(X) : +- ColumnarExchange (65) - : +- VeloxAppendBatches (64) + : +- VeloxResizeBatches (64) : +- ^ ProjectExecTransformer (62) : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (61) : :- ^ InputIteratorTransformer (35) : : +- ShuffleQueryStage (33), Statistics(X) : : +- ColumnarExchange (32) - : : +- VeloxAppendBatches (31) + : : +- VeloxResizeBatches (31) : : +- ^ ProjectExecTransformer (29) : : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (28) : : :- ^ InputIteratorTransformer (18) : : : +- ShuffleQueryStage (16), Statistics(X) : : : +- ColumnarExchange (15) - : : : +- VeloxAppendBatches (14) + : : : +- VeloxResizeBatches (14) : : : +- ^ ProjectExecTransformer (12) : : : +- ^ FilterExecTransformer (11) : : : +- ^ Scan parquet (10) : : +- ^ InputIteratorTransformer (27) : : +- ShuffleQueryStage (25), Statistics(X) : : +- ColumnarExchange (24) - : : +- VeloxAppendBatches (23) + : : +- VeloxResizeBatches (23) : : +- ^ ProjectExecTransformer (21) : : +- ^ FilterExecTransformer (20) : : +- ^ Scan parquet (19) : +- ^ InputIteratorTransformer (60) : +- ShuffleQueryStage (58), Statistics(X) : +- ColumnarExchange (57) - : +- VeloxAppendBatches (56) + : +- VeloxResizeBatches (56) : +- ^ ProjectExecTransformer (54) : +- ^ FilterExecTransformer (53) : +- ^ ProjectExecTransformer (52) @@ -60,7 +60,7 @@ AdaptiveSparkPlan (143) : :- ^ InputIteratorTransformer (44) : : +- ShuffleQueryStage (42), Statistics(X) : : +- ColumnarExchange (41) - : : +- VeloxAppendBatches (40) + : : +- VeloxResizeBatches (40) : : +- ^ ProjectExecTransformer (38) : : +- ^ FilterExecTransformer (37) : : +- ^ Scan parquet (36) @@ -70,7 +70,7 @@ AdaptiveSparkPlan (143) +- ^ InputIteratorTransformer (85) +- ShuffleQueryStage (83), Statistics(X) +- ColumnarExchange (82) - +- VeloxAppendBatches (81) + +- VeloxResizeBatches (81) +- ^ ProjectExecTransformer (79) +- ^ FilterExecTransformer (78) +- ^ Scan parquet (77) @@ -145,9 +145,9 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] @@ -182,9 +182,9 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] @@ -219,9 +219,9 @@ Input [2]: [p_partkey#X, p_name#X] Input [2]: [hash_partition_key#X, p_partkey#X] Arguments: false -(23) VeloxAppendBatches +(23) VeloxResizeBatches Input [2]: [hash_partition_key#X, p_partkey#X] -Arguments: X +Arguments: X, X (24) ColumnarExchange Input [2]: [hash_partition_key#X, p_partkey#X] @@ -250,9 +250,9 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] @@ -287,9 +287,9 @@ Input [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] Input [4]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, l_quantity#X] Arguments: false -(40) VeloxAppendBatches +(40) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, l_quantity#X] -Arguments: X +Arguments: X, X (41) ColumnarExchange Input [4]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, l_quantity#X] @@ -353,9 +353,9 @@ Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: false -(56) VeloxAppendBatches +(56) VeloxResizeBatches Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -Arguments: X +Arguments: X, X (57) ColumnarExchange Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] @@ -384,9 +384,9 @@ Input [6]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, (0.5 * sum(l_quantity))#X Input [2]: [hash_partition_key#X, ps_suppkey#X] Arguments: false -(64) VeloxAppendBatches +(64) VeloxResizeBatches Input [2]: [hash_partition_key#X, ps_suppkey#X] -Arguments: X +Arguments: X, X (65) ColumnarExchange Input [2]: [hash_partition_key#X, ps_suppkey#X] @@ -415,9 +415,9 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] Arguments: false -(72) VeloxAppendBatches +(72) VeloxResizeBatches Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] -Arguments: X +Arguments: X, X (73) ColumnarExchange Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] @@ -452,9 +452,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: false -(81) VeloxAppendBatches +(81) VeloxResizeBatches Input [2]: [hash_partition_key#X, n_nationkey#X] -Arguments: X +Arguments: X, X (82) ColumnarExchange Input [2]: [hash_partition_key#X, n_nationkey#X] @@ -483,9 +483,9 @@ Input [4]: [s_name#X, s_address#X, s_nationkey#X, n_nationkey#X] Input [2]: [s_name#X, s_address#X] Arguments: false -(89) VeloxAppendBatches +(89) VeloxResizeBatches Input [2]: [s_name#X, s_address#X] -Arguments: X +Arguments: X, X (90) ColumnarExchange Input [2]: [s_name#X, s_address#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/21.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/21.txt index 8413e2f8f232..36ecd282aa4f 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/21.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/21.txt @@ -6,7 +6,7 @@ AdaptiveSparkPlan (137) +- ^ InputIteratorTransformer (88) +- ShuffleQueryStage (86), Statistics(X) +- ColumnarExchange (85) - +- VeloxAppendBatches (84) + +- VeloxResizeBatches (84) +- ^ ProjectExecTransformer (82) +- ^ FlushableHashAggregateExecTransformer (81) +- ^ ProjectExecTransformer (80) @@ -14,60 +14,60 @@ AdaptiveSparkPlan (137) :- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67), Statistics(X) : +- ColumnarExchange (66) - : +- VeloxAppendBatches (65) + : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (62) : :- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50), Statistics(X) : : +- ColumnarExchange (49) - : : +- VeloxAppendBatches (48) + : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (45) : : :- ^ InputIteratorTransformer (9) : : : +- ShuffleQueryStage (7), Statistics(X) : : : +- ColumnarExchange (6) - : : : +- VeloxAppendBatches (5) + : : : +- VeloxResizeBatches (5) : : : +- ^ ProjectExecTransformer (3) : : : +- ^ FilterExecTransformer (2) : : : +- ^ Scan parquet (1) : : +- ^ InputIteratorTransformer (44) : : +- ShuffleQueryStage (42), Statistics(X) : : +- ColumnarExchange (41) - : : +- VeloxAppendBatches (40) + : : +- VeloxResizeBatches (40) : : +- ^ ProjectExecTransformer (38) : : +- ^ ShuffledHashJoinExecTransformer LeftAnti BuildRight (37) : : :- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (27) : : : :- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : +- ColumnarExchange (15) - : : : : +- VeloxAppendBatches (14) + : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) : : : : +- ^ Scan parquet (10) : : : +- ^ InputIteratorTransformer (26) : : : +- ShuffleQueryStage (24), Statistics(X) : : : +- ColumnarExchange (23) - : : : +- VeloxAppendBatches (22) + : : : +- VeloxResizeBatches (22) : : : +- ^ ProjectExecTransformer (20) : : : +- ^ Scan parquet (19) : : +- ^ InputIteratorTransformer (36) : : +- ShuffleQueryStage (34), Statistics(X) : : +- ColumnarExchange (33) - : : +- VeloxAppendBatches (32) + : : +- VeloxResizeBatches (32) : : +- ^ ProjectExecTransformer (30) : : +- ^ FilterExecTransformer (29) : : +- ^ Scan parquet (28) : +- ^ InputIteratorTransformer (61) : +- ShuffleQueryStage (59), Statistics(X) : +- ColumnarExchange (58) - : +- VeloxAppendBatches (57) + : +- VeloxResizeBatches (57) : +- ^ ProjectExecTransformer (55) : +- ^ FilterExecTransformer (54) : +- ^ Scan parquet (53) +- ^ InputIteratorTransformer (78) +- ShuffleQueryStage (76), Statistics(X) +- ColumnarExchange (75) - +- VeloxAppendBatches (74) + +- VeloxResizeBatches (74) +- ^ ProjectExecTransformer (72) +- ^ FilterExecTransformer (71) +- ^ Scan parquet (70) @@ -138,9 +138,9 @@ Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X] Input [4]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_nationkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [4]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_nationkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [4]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_nationkey#X] @@ -175,9 +175,9 @@ Input [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] @@ -207,9 +207,9 @@ Input [2]: [l_orderkey#X, l_suppkey#X] Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] @@ -249,9 +249,9 @@ Input [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] Arguments: false -(32) VeloxAppendBatches +(32) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] -Arguments: X +Arguments: X, X (33) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] @@ -280,9 +280,9 @@ Input [2]: [l_orderkey#X, l_suppkey#X] Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] Arguments: false -(40) VeloxAppendBatches +(40) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] -Arguments: X +Arguments: X, X (41) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] @@ -311,9 +311,9 @@ Input [5]: [s_suppkey#X, s_name#X, s_nationkey#X, l_orderkey#X, l_suppkey#X] Input [4]: [hash_partition_key#X, s_name#X, s_nationkey#X, l_orderkey#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [4]: [hash_partition_key#X, s_name#X, s_nationkey#X, l_orderkey#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [4]: [hash_partition_key#X, s_name#X, s_nationkey#X, l_orderkey#X] @@ -348,9 +348,9 @@ Input [2]: [o_orderkey#X, o_orderstatus#X] Input [2]: [hash_partition_key#X, o_orderkey#X] Arguments: false -(57) VeloxAppendBatches +(57) VeloxResizeBatches Input [2]: [hash_partition_key#X, o_orderkey#X] -Arguments: X +Arguments: X, X (58) ColumnarExchange Input [2]: [hash_partition_key#X, o_orderkey#X] @@ -379,9 +379,9 @@ Input [4]: [s_name#X, s_nationkey#X, l_orderkey#X, o_orderkey#X] Input [3]: [hash_partition_key#X, s_name#X, s_nationkey#X] Arguments: false -(65) VeloxAppendBatches +(65) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_name#X, s_nationkey#X] -Arguments: X +Arguments: X, X (66) ColumnarExchange Input [3]: [hash_partition_key#X, s_name#X, s_nationkey#X] @@ -416,9 +416,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: false -(74) VeloxAppendBatches +(74) VeloxResizeBatches Input [2]: [hash_partition_key#X, n_nationkey#X] -Arguments: X +Arguments: X, X (75) ColumnarExchange Input [2]: [hash_partition_key#X, n_nationkey#X] @@ -458,9 +458,9 @@ Input [2]: [s_name#X, count#X] Input [3]: [hash_partition_key#X, s_name#X, count#X] Arguments: false -(84) VeloxAppendBatches +(84) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_name#X, count#X] -Arguments: X +Arguments: X, X (85) ColumnarExchange Input [3]: [hash_partition_key#X, s_name#X, count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/22.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/22.txt index 214b34066a8f..50a049ea3ce9 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/22.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/22.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (52) +- ^ InputIteratorTransformer (34) +- ShuffleQueryStage (32), Statistics(X) +- ColumnarExchange (31) - +- VeloxAppendBatches (30) + +- VeloxResizeBatches (30) +- ^ RegularHashAggregateExecTransformer (28) +- ^ InputIteratorTransformer (27) +- ShuffleQueryStage (25), Statistics(X) +- ColumnarExchange (24) - +- VeloxAppendBatches (23) + +- VeloxResizeBatches (23) +- ^ ProjectExecTransformer (21) +- ^ FlushableHashAggregateExecTransformer (20) +- ^ ProjectExecTransformer (19) @@ -19,14 +19,14 @@ AdaptiveSparkPlan (52) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7), Statistics(X) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (17) +- ShuffleQueryStage (15), Statistics(X) +- ColumnarExchange (14) - +- VeloxAppendBatches (13) + +- VeloxResizeBatches (13) +- ^ ProjectExecTransformer (11) +- ^ Scan parquet (10) +- == Initial Plan == @@ -65,9 +65,9 @@ Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X] Input [4]: [hash_partition_key#X, c_custkey#X, c_phone#X, c_acctbal#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [4]: [hash_partition_key#X, c_custkey#X, c_phone#X, c_acctbal#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [4]: [hash_partition_key#X, c_custkey#X, c_phone#X, c_acctbal#X] @@ -97,9 +97,9 @@ Input [1]: [o_custkey#X] Input [2]: [hash_partition_key#X, o_custkey#X] Arguments: false -(13) VeloxAppendBatches +(13) VeloxResizeBatches Input [2]: [hash_partition_key#X, o_custkey#X] -Arguments: X +Arguments: X, X (14) ColumnarExchange Input [2]: [hash_partition_key#X, o_custkey#X] @@ -139,9 +139,9 @@ Input [4]: [cntrycode#X, count#X, sum#X, isEmpty#X] Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] Arguments: false -(23) VeloxAppendBatches +(23) VeloxResizeBatches Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (24) ColumnarExchange Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] @@ -168,9 +168,9 @@ Results [3]: [cntrycode#X, count(1)#X AS numcust#X, sum(c_acctbal#X)#X AS totacc Input [3]: [cntrycode#X, numcust#X, totacctbal#X] Arguments: false -(30) VeloxAppendBatches +(30) VeloxResizeBatches Input [3]: [cntrycode#X, numcust#X, totacctbal#X] -Arguments: X +Arguments: X, X (31) ColumnarExchange Input [3]: [cntrycode#X, numcust#X, totacctbal#X] @@ -279,7 +279,7 @@ AdaptiveSparkPlan (72) +- ^ InputIteratorTransformer (62) +- ShuffleQueryStage (60), Statistics(X) +- ColumnarExchange (59) - +- VeloxAppendBatches (58) + +- VeloxResizeBatches (58) +- ^ FlushableHashAggregateExecTransformer (56) +- ^ ProjectExecTransformer (55) +- ^ FilterExecTransformer (54) @@ -319,9 +319,9 @@ Results [2]: [sum#X, count#X] Input [2]: [sum#X, count#X] Arguments: false -(58) VeloxAppendBatches +(58) VeloxResizeBatches Input [2]: [sum#X, count#X] -Arguments: X +Arguments: X, X (59) ColumnarExchange Input [2]: [sum#X, count#X] @@ -396,7 +396,7 @@ AdaptiveSparkPlan (72) +- ^ InputIteratorTransformer (62) +- ShuffleQueryStage (60), Statistics(X) +- ColumnarExchange (59) - +- VeloxAppendBatches (58) + +- VeloxResizeBatches (58) +- ^ FlushableHashAggregateExecTransformer (56) +- ^ ProjectExecTransformer (55) +- ^ FilterExecTransformer (54) diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/3.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/3.txt index df17819cafe9..4d701845a38a 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/3.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/3.txt @@ -11,27 +11,27 @@ AdaptiveSparkPlan (67) :- ^ InputIteratorTransformer (26) : +- ShuffleQueryStage (24), Statistics(X) : +- ColumnarExchange (23) - : +- VeloxAppendBatches (22) + : +- VeloxResizeBatches (22) : +- ^ ProjectExecTransformer (20) : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7), Statistics(X) : : +- ColumnarExchange (6) - : : +- VeloxAppendBatches (5) + : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16), Statistics(X) : +- ColumnarExchange (15) - : +- VeloxAppendBatches (14) + : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ FilterExecTransformer (11) : +- ^ Scan parquet (10) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33), Statistics(X) +- ColumnarExchange (32) - +- VeloxAppendBatches (31) + +- VeloxResizeBatches (31) +- ^ ProjectExecTransformer (29) +- ^ FilterExecTransformer (28) +- ^ Scan parquet (27) @@ -80,9 +80,9 @@ Input [2]: [c_custkey#X, c_mktsegment#X] Input [2]: [hash_partition_key#X, c_custkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [2]: [hash_partition_key#X, c_custkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [2]: [hash_partition_key#X, c_custkey#X] @@ -117,9 +117,9 @@ Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] @@ -148,9 +148,9 @@ Input [5]: [c_custkey#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriorit Input [4]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X, o_shippriority#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [4]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X, o_shippriority#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [4]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X, o_shippriority#X] @@ -185,9 +185,9 @@ Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/4.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/4.txt index 85d303df874f..163bb15a43fa 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/4.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/4.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (56) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33), Statistics(X) +- ColumnarExchange (32) - +- VeloxAppendBatches (31) + +- VeloxResizeBatches (31) +- ^ RegularHashAggregateExecTransformer (29) +- ^ InputIteratorTransformer (28) +- ShuffleQueryStage (26), Statistics(X) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ ProjectExecTransformer (22) +- ^ FlushableHashAggregateExecTransformer (21) +- ^ ProjectExecTransformer (20) @@ -19,14 +19,14 @@ AdaptiveSparkPlan (56) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7), Statistics(X) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) +- ^ Scan parquet (10) @@ -69,9 +69,9 @@ Input [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] @@ -106,9 +106,9 @@ Input [3]: [l_orderkey#X, l_commitdate#X, l_receiptdate#X] Input [2]: [hash_partition_key#X, l_orderkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [2]: [hash_partition_key#X, l_orderkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [2]: [hash_partition_key#X, l_orderkey#X] @@ -148,9 +148,9 @@ Input [2]: [o_orderpriority#X, count#X] Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] @@ -177,9 +177,9 @@ Results [2]: [o_orderpriority#X, count(1)#X AS order_count#X] Input [2]: [o_orderpriority#X, order_count#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [2]: [o_orderpriority#X, order_count#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [2]: [o_orderpriority#X, order_count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/5.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/5.txt index 8978f9563c68..6aadfc753a9a 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/5.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/5.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (156) +- ^ InputIteratorTransformer (103) +- ShuffleQueryStage (101), Statistics(X) +- ColumnarExchange (100) - +- VeloxAppendBatches (99) + +- VeloxResizeBatches (99) +- ^ RegularHashAggregateExecTransformer (97) +- ^ InputIteratorTransformer (96) +- ShuffleQueryStage (94), Statistics(X) +- ColumnarExchange (93) - +- VeloxAppendBatches (92) + +- VeloxResizeBatches (92) +- ^ ProjectExecTransformer (90) +- ^ FlushableHashAggregateExecTransformer (89) +- ^ ProjectExecTransformer (88) @@ -19,66 +19,66 @@ AdaptiveSparkPlan (156) :- ^ InputIteratorTransformer (77) : +- ShuffleQueryStage (75), Statistics(X) : +- ColumnarExchange (74) - : +- VeloxAppendBatches (73) + : +- VeloxResizeBatches (73) : +- ^ ProjectExecTransformer (71) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (70) : :- ^ InputIteratorTransformer (60) : : +- ShuffleQueryStage (58), Statistics(X) : : +- ColumnarExchange (57) - : : +- VeloxAppendBatches (56) + : : +- VeloxResizeBatches (56) : : +- ^ ProjectExecTransformer (54) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (53) : : :- ^ InputIteratorTransformer (43) : : : +- ShuffleQueryStage (41), Statistics(X) : : : +- ColumnarExchange (40) - : : : +- VeloxAppendBatches (39) + : : : +- VeloxResizeBatches (39) : : : +- ^ ProjectExecTransformer (37) : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (36) : : : :- ^ InputIteratorTransformer (26) : : : : +- ShuffleQueryStage (24), Statistics(X) : : : : +- ColumnarExchange (23) - : : : : +- VeloxAppendBatches (22) + : : : : +- VeloxResizeBatches (22) : : : : +- ^ ProjectExecTransformer (20) : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) : : : : :- ^ InputIteratorTransformer (9) : : : : : +- ShuffleQueryStage (7), Statistics(X) : : : : : +- ColumnarExchange (6) - : : : : : +- VeloxAppendBatches (5) + : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ FilterExecTransformer (2) : : : : : +- ^ Scan parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : +- ColumnarExchange (15) - : : : : +- VeloxAppendBatches (14) + : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) : : : : +- ^ Scan parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33), Statistics(X) : : : +- ColumnarExchange (32) - : : : +- VeloxAppendBatches (31) + : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ FilterExecTransformer (28) : : : +- ^ Scan parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50), Statistics(X) : : +- ColumnarExchange (49) - : : +- VeloxAppendBatches (48) + : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ FilterExecTransformer (45) : : +- ^ Scan parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67), Statistics(X) : +- ColumnarExchange (66) - : +- VeloxAppendBatches (65) + : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ FilterExecTransformer (62) : +- ^ Scan parquet (61) +- ^ InputIteratorTransformer (86) +- ShuffleQueryStage (84), Statistics(X) +- ColumnarExchange (83) - +- VeloxAppendBatches (82) + +- VeloxResizeBatches (82) +- ^ ProjectExecTransformer (80) +- ^ FilterExecTransformer (79) +- ^ Scan parquet (78) @@ -153,9 +153,9 @@ Input [2]: [c_custkey#X, c_nationkey#X] Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] @@ -190,9 +190,9 @@ Input [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] @@ -221,9 +221,9 @@ Input [4]: [c_custkey#X, c_nationkey#X, o_orderkey#X, o_custkey#X] Input [3]: [hash_partition_key#X, c_nationkey#X, o_orderkey#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_nationkey#X, o_orderkey#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [3]: [hash_partition_key#X, c_nationkey#X, o_orderkey#X] @@ -258,9 +258,9 @@ Input [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] @@ -289,9 +289,9 @@ Input [6]: [c_nationkey#X, o_orderkey#X, l_orderkey#X, l_suppkey#X, l_extendedpr Input [5]: [hash_partition_key#X, c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(39) VeloxAppendBatches +(39) VeloxResizeBatches Input [5]: [hash_partition_key#X, c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (40) ColumnarExchange Input [5]: [hash_partition_key#X, c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] @@ -326,9 +326,9 @@ Input [2]: [s_suppkey#X, s_nationkey#X] Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -357,9 +357,9 @@ Input [6]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, s_suppk Input [4]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X] Arguments: false -(56) VeloxAppendBatches +(56) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X] -Arguments: X +Arguments: X, X (57) ColumnarExchange Input [4]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X] @@ -394,9 +394,9 @@ Input [3]: [n_nationkey#X, n_name#X, n_regionkey#X] Input [4]: [hash_partition_key#X, n_nationkey#X, n_name#X, n_regionkey#X] Arguments: false -(65) VeloxAppendBatches +(65) VeloxResizeBatches Input [4]: [hash_partition_key#X, n_nationkey#X, n_name#X, n_regionkey#X] -Arguments: X +Arguments: X, X (66) ColumnarExchange Input [4]: [hash_partition_key#X, n_nationkey#X, n_name#X, n_regionkey#X] @@ -425,9 +425,9 @@ Input [6]: [l_extendedprice#X, l_discount#X, s_nationkey#X, n_nationkey#X, n_nam Input [5]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] Arguments: false -(73) VeloxAppendBatches +(73) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] -Arguments: X +Arguments: X, X (74) ColumnarExchange Input [5]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] @@ -462,9 +462,9 @@ Input [2]: [r_regionkey#X, r_name#X] Input [2]: [hash_partition_key#X, r_regionkey#X] Arguments: false -(82) VeloxAppendBatches +(82) VeloxResizeBatches Input [2]: [hash_partition_key#X, r_regionkey#X] -Arguments: X +Arguments: X, X (83) ColumnarExchange Input [2]: [hash_partition_key#X, r_regionkey#X] @@ -504,9 +504,9 @@ Input [3]: [n_name#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] Arguments: false -(92) VeloxAppendBatches +(92) VeloxResizeBatches Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (93) ColumnarExchange Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] @@ -533,9 +533,9 @@ Results [2]: [n_name#X, sum(CheckOverflow((promote_precision(cast(l_extendedpric Input [2]: [n_name#X, revenue#X] Arguments: false -(99) VeloxAppendBatches +(99) VeloxResizeBatches Input [2]: [n_name#X, revenue#X] -Arguments: X +Arguments: X, X (100) ColumnarExchange Input [2]: [n_name#X, revenue#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/6.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/6.txt index 3432579a0de0..864cdf3832e1 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/6.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/6.txt @@ -6,7 +6,7 @@ AdaptiveSparkPlan (20) +- ^ InputIteratorTransformer (10) +- ShuffleQueryStage (8), Statistics(X) +- ColumnarExchange (7) - +- VeloxAppendBatches (6) + +- VeloxResizeBatches (6) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ FilterExecTransformer (2) @@ -46,9 +46,9 @@ Results [2]: [sum#X, isEmpty#X] Input [2]: [sum#X, isEmpty#X] Arguments: false -(6) VeloxAppendBatches +(6) VeloxResizeBatches Input [2]: [sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (7) ColumnarExchange Input [2]: [sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/7.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/7.txt index 244f650f3a72..85a7dbf545a4 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/7.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/7.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (149) +- ^ InputIteratorTransformer (98) +- ShuffleQueryStage (96), Statistics(X) +- ColumnarExchange (95) - +- VeloxAppendBatches (94) + +- VeloxResizeBatches (94) +- ^ RegularHashAggregateExecTransformer (92) +- ^ InputIteratorTransformer (91) +- ShuffleQueryStage (89), Statistics(X) +- ColumnarExchange (88) - +- VeloxAppendBatches (87) + +- VeloxResizeBatches (87) +- ^ ProjectExecTransformer (85) +- ^ FlushableHashAggregateExecTransformer (84) +- ^ ProjectExecTransformer (83) @@ -19,59 +19,59 @@ AdaptiveSparkPlan (149) :- ^ InputIteratorTransformer (77) : +- ShuffleQueryStage (75), Statistics(X) : +- ColumnarExchange (74) - : +- VeloxAppendBatches (73) + : +- VeloxResizeBatches (73) : +- ^ ProjectExecTransformer (71) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (70) : :- ^ InputIteratorTransformer (60) : : +- ShuffleQueryStage (58), Statistics(X) : : +- ColumnarExchange (57) - : : +- VeloxAppendBatches (56) + : : +- VeloxResizeBatches (56) : : +- ^ ProjectExecTransformer (54) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (53) : : :- ^ InputIteratorTransformer (43) : : : +- ShuffleQueryStage (41), Statistics(X) : : : +- ColumnarExchange (40) - : : : +- VeloxAppendBatches (39) + : : : +- VeloxResizeBatches (39) : : : +- ^ ProjectExecTransformer (37) : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (36) : : : :- ^ InputIteratorTransformer (26) : : : : +- ShuffleQueryStage (24), Statistics(X) : : : : +- ColumnarExchange (23) - : : : : +- VeloxAppendBatches (22) + : : : : +- VeloxResizeBatches (22) : : : : +- ^ ProjectExecTransformer (20) : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) : : : : :- ^ InputIteratorTransformer (9) : : : : : +- ShuffleQueryStage (7), Statistics(X) : : : : : +- ColumnarExchange (6) - : : : : : +- VeloxAppendBatches (5) + : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ FilterExecTransformer (2) : : : : : +- ^ Scan parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : +- ColumnarExchange (15) - : : : : +- VeloxAppendBatches (14) + : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) : : : : +- ^ Scan parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33), Statistics(X) : : : +- ColumnarExchange (32) - : : : +- VeloxAppendBatches (31) + : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ FilterExecTransformer (28) : : : +- ^ Scan parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50), Statistics(X) : : +- ColumnarExchange (49) - : : +- VeloxAppendBatches (48) + : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ FilterExecTransformer (45) : : +- ^ Scan parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67), Statistics(X) : +- ColumnarExchange (66) - : +- VeloxAppendBatches (65) + : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ FilterExecTransformer (62) : +- ^ Scan parquet (61) @@ -147,9 +147,9 @@ Input [2]: [s_suppkey#X, s_nationkey#X] Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -184,9 +184,9 @@ Input [5]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipda Input [6]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [6]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] @@ -215,9 +215,9 @@ Input [7]: [s_suppkey#X, s_nationkey#X, l_orderkey#X, l_suppkey#X, l_extendedpri Input [6]: [hash_partition_key#X, s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [6]: [hash_partition_key#X, s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [6]: [hash_partition_key#X, s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] @@ -252,9 +252,9 @@ Input [2]: [o_orderkey#X, o_custkey#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] @@ -283,9 +283,9 @@ Input [7]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_ship Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X] Arguments: false -(39) VeloxAppendBatches +(39) VeloxResizeBatches Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X] -Arguments: X +Arguments: X, X (40) ColumnarExchange Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X] @@ -320,9 +320,9 @@ Input [2]: [c_custkey#X, c_nationkey#X] Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] @@ -351,9 +351,9 @@ Input [7]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_cust Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X] Arguments: false -(56) VeloxAppendBatches +(56) VeloxResizeBatches Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X] -Arguments: X +Arguments: X, X (57) ColumnarExchange Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X] @@ -388,9 +388,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] Arguments: false -(65) VeloxAppendBatches +(65) VeloxResizeBatches Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] -Arguments: X +Arguments: X, X (66) ColumnarExchange Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] @@ -419,9 +419,9 @@ Input [7]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nati Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X, n_name#X] Arguments: false -(73) VeloxAppendBatches +(73) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X, n_name#X] -Arguments: X +Arguments: X, X (74) ColumnarExchange Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X, n_name#X] @@ -474,9 +474,9 @@ Input [5]: [supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] Arguments: false -(87) VeloxAppendBatches +(87) VeloxResizeBatches Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (88) ColumnarExchange Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] @@ -503,9 +503,9 @@ Results [4]: [supp_nation#X, cust_nation#X, l_year#X, sum(volume#X)#X AS revenue Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] Arguments: false -(94) VeloxAppendBatches +(94) VeloxResizeBatches Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] -Arguments: X +Arguments: X, X (95) ColumnarExchange Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/8.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/8.txt index 282790ba6507..88597b61d0ee 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/8.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/8.txt @@ -6,13 +6,13 @@ AdaptiveSparkPlan (207) +- ^ InputIteratorTransformer (138) +- ShuffleQueryStage (136), Statistics(X) +- ColumnarExchange (135) - +- VeloxAppendBatches (134) + +- VeloxResizeBatches (134) +- ^ ProjectExecTransformer (132) +- ^ RegularHashAggregateExecTransformer (131) +- ^ InputIteratorTransformer (130) +- ShuffleQueryStage (128), Statistics(X) +- ColumnarExchange (127) - +- VeloxAppendBatches (126) + +- VeloxResizeBatches (126) +- ^ ProjectExecTransformer (124) +- ^ FlushableHashAggregateExecTransformer (123) +- ^ ProjectExecTransformer (122) @@ -20,92 +20,92 @@ AdaptiveSparkPlan (207) :- ^ InputIteratorTransformer (111) : +- ShuffleQueryStage (109), Statistics(X) : +- ColumnarExchange (108) - : +- VeloxAppendBatches (107) + : +- VeloxResizeBatches (107) : +- ^ ProjectExecTransformer (105) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (104) : :- ^ InputIteratorTransformer (94) : : +- ShuffleQueryStage (92), Statistics(X) : : +- ColumnarExchange (91) - : : +- VeloxAppendBatches (90) + : : +- VeloxResizeBatches (90) : : +- ^ ProjectExecTransformer (88) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (87) : : :- ^ InputIteratorTransformer (77) : : : +- ShuffleQueryStage (75), Statistics(X) : : : +- ColumnarExchange (74) - : : : +- VeloxAppendBatches (73) + : : : +- VeloxResizeBatches (73) : : : +- ^ ProjectExecTransformer (71) : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (70) : : : :- ^ InputIteratorTransformer (60) : : : : +- ShuffleQueryStage (58), Statistics(X) : : : : +- ColumnarExchange (57) - : : : : +- VeloxAppendBatches (56) + : : : : +- VeloxResizeBatches (56) : : : : +- ^ ProjectExecTransformer (54) : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (53) : : : : :- ^ InputIteratorTransformer (43) : : : : : +- ShuffleQueryStage (41), Statistics(X) : : : : : +- ColumnarExchange (40) - : : : : : +- VeloxAppendBatches (39) + : : : : : +- VeloxResizeBatches (39) : : : : : +- ^ ProjectExecTransformer (37) : : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (36) : : : : : :- ^ InputIteratorTransformer (26) : : : : : : +- ShuffleQueryStage (24), Statistics(X) : : : : : : +- ColumnarExchange (23) - : : : : : : +- VeloxAppendBatches (22) + : : : : : : +- VeloxResizeBatches (22) : : : : : : +- ^ ProjectExecTransformer (20) : : : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) : : : : : : :- ^ InputIteratorTransformer (9) : : : : : : : +- ShuffleQueryStage (7), Statistics(X) : : : : : : : +- ColumnarExchange (6) - : : : : : : : +- VeloxAppendBatches (5) + : : : : : : : +- VeloxResizeBatches (5) : : : : : : : +- ^ ProjectExecTransformer (3) : : : : : : : +- ^ FilterExecTransformer (2) : : : : : : : +- ^ Scan parquet (1) : : : : : : +- ^ InputIteratorTransformer (18) : : : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : : : +- ColumnarExchange (15) - : : : : : : +- VeloxAppendBatches (14) + : : : : : : +- VeloxResizeBatches (14) : : : : : : +- ^ ProjectExecTransformer (12) : : : : : : +- ^ FilterExecTransformer (11) : : : : : : +- ^ Scan parquet (10) : : : : : +- ^ InputIteratorTransformer (35) : : : : : +- ShuffleQueryStage (33), Statistics(X) : : : : : +- ColumnarExchange (32) - : : : : : +- VeloxAppendBatches (31) + : : : : : +- VeloxResizeBatches (31) : : : : : +- ^ ProjectExecTransformer (29) : : : : : +- ^ FilterExecTransformer (28) : : : : : +- ^ Scan parquet (27) : : : : +- ^ InputIteratorTransformer (52) : : : : +- ShuffleQueryStage (50), Statistics(X) : : : : +- ColumnarExchange (49) - : : : : +- VeloxAppendBatches (48) + : : : : +- VeloxResizeBatches (48) : : : : +- ^ ProjectExecTransformer (46) : : : : +- ^ FilterExecTransformer (45) : : : : +- ^ Scan parquet (44) : : : +- ^ InputIteratorTransformer (69) : : : +- ShuffleQueryStage (67), Statistics(X) : : : +- ColumnarExchange (66) - : : : +- VeloxAppendBatches (65) + : : : +- VeloxResizeBatches (65) : : : +- ^ ProjectExecTransformer (63) : : : +- ^ FilterExecTransformer (62) : : : +- ^ Scan parquet (61) : : +- ^ InputIteratorTransformer (86) : : +- ShuffleQueryStage (84), Statistics(X) : : +- ColumnarExchange (83) - : : +- VeloxAppendBatches (82) + : : +- VeloxResizeBatches (82) : : +- ^ ProjectExecTransformer (80) : : +- ^ FilterExecTransformer (79) : : +- ^ Scan parquet (78) : +- ^ InputIteratorTransformer (103) : +- ShuffleQueryStage (101), Statistics(X) : +- ColumnarExchange (100) - : +- VeloxAppendBatches (99) + : +- VeloxResizeBatches (99) : +- ^ ProjectExecTransformer (97) : +- ^ FilterExecTransformer (96) : +- ^ Scan parquet (95) +- ^ InputIteratorTransformer (120) +- ShuffleQueryStage (118), Statistics(X) +- ColumnarExchange (117) - +- VeloxAppendBatches (116) + +- VeloxResizeBatches (116) +- ^ ProjectExecTransformer (114) +- ^ FilterExecTransformer (113) +- ^ Scan parquet (112) @@ -196,9 +196,9 @@ Input [2]: [p_partkey#X, p_type#X] Input [2]: [hash_partition_key#X, p_partkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [2]: [hash_partition_key#X, p_partkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [2]: [hash_partition_key#X, p_partkey#X] @@ -233,9 +233,9 @@ Input [5]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discoun Input [6]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [6]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] @@ -264,9 +264,9 @@ Input [6]: [p_partkey#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] @@ -301,9 +301,9 @@ Input [2]: [s_suppkey#X, s_nationkey#X] Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -332,9 +332,9 @@ Input [6]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, s_suppke Input [5]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] Arguments: false -(39) VeloxAppendBatches +(39) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] -Arguments: X +Arguments: X, X (40) ColumnarExchange Input [5]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] @@ -369,9 +369,9 @@ Input [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Input [4]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [4]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [4]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X] @@ -400,9 +400,9 @@ Input [7]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orde Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X] Arguments: false -(56) VeloxAppendBatches +(56) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X] -Arguments: X +Arguments: X, X (57) ColumnarExchange Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X] @@ -437,9 +437,9 @@ Input [2]: [c_custkey#X, c_nationkey#X] Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] Arguments: false -(65) VeloxAppendBatches +(65) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] -Arguments: X +Arguments: X, X (66) ColumnarExchange Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] @@ -468,9 +468,9 @@ Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_order Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X] Arguments: false -(73) VeloxAppendBatches +(73) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X] -Arguments: X +Arguments: X, X (74) ColumnarExchange Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X] @@ -505,9 +505,9 @@ Input [2]: [n_nationkey#X, n_regionkey#X] Input [3]: [hash_partition_key#X, n_nationkey#X, n_regionkey#X] Arguments: false -(82) VeloxAppendBatches +(82) VeloxResizeBatches Input [3]: [hash_partition_key#X, n_nationkey#X, n_regionkey#X] -Arguments: X +Arguments: X, X (83) ColumnarExchange Input [3]: [hash_partition_key#X, n_nationkey#X, n_regionkey#X] @@ -536,9 +536,9 @@ Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nat Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X] Arguments: false -(90) VeloxAppendBatches +(90) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X] -Arguments: X +Arguments: X, X (91) ColumnarExchange Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X] @@ -573,9 +573,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] Arguments: false -(99) VeloxAppendBatches +(99) VeloxResizeBatches Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] -Arguments: X +Arguments: X, X (100) ColumnarExchange Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] @@ -604,9 +604,9 @@ Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_reg Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X] Arguments: false -(107) VeloxAppendBatches +(107) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X] -Arguments: X +Arguments: X, X (108) ColumnarExchange Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X] @@ -641,9 +641,9 @@ Input [2]: [r_regionkey#X, r_name#X] Input [2]: [hash_partition_key#X, r_regionkey#X] Arguments: false -(116) VeloxAppendBatches +(116) VeloxResizeBatches Input [2]: [hash_partition_key#X, r_regionkey#X] -Arguments: X +Arguments: X, X (117) ColumnarExchange Input [2]: [hash_partition_key#X, r_regionkey#X] @@ -683,9 +683,9 @@ Input [5]: [o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] Arguments: false -(126) VeloxAppendBatches +(126) VeloxResizeBatches Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (127) ColumnarExchange Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] @@ -716,9 +716,9 @@ Input [3]: [o_year#X, sum(CASE WHEN (nation#X = BRAZIL) THEN volume#X ELSE 0.000 Input [2]: [o_year#X, mkt_share#X] Arguments: false -(134) VeloxAppendBatches +(134) VeloxResizeBatches Input [2]: [o_year#X, mkt_share#X] -Arguments: X +Arguments: X, X (135) ColumnarExchange Input [2]: [o_year#X, mkt_share#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/9.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/9.txt index 15fbf97a77f3..884d1d3563b7 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/9.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/9.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (155) +- ^ InputIteratorTransformer (103) +- ShuffleQueryStage (101), Statistics(X) +- ColumnarExchange (100) - +- VeloxAppendBatches (99) + +- VeloxResizeBatches (99) +- ^ RegularHashAggregateExecTransformer (97) +- ^ InputIteratorTransformer (96) +- ShuffleQueryStage (94), Statistics(X) +- ColumnarExchange (93) - +- VeloxAppendBatches (92) + +- VeloxResizeBatches (92) +- ^ ProjectExecTransformer (90) +- ^ FlushableHashAggregateExecTransformer (89) +- ^ ProjectExecTransformer (88) @@ -19,66 +19,66 @@ AdaptiveSparkPlan (155) :- ^ InputIteratorTransformer (77) : +- ShuffleQueryStage (75), Statistics(X) : +- ColumnarExchange (74) - : +- VeloxAppendBatches (73) + : +- VeloxResizeBatches (73) : +- ^ ProjectExecTransformer (71) : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (70) : :- ^ InputIteratorTransformer (60) : : +- ShuffleQueryStage (58), Statistics(X) : : +- ColumnarExchange (57) - : : +- VeloxAppendBatches (56) + : : +- VeloxResizeBatches (56) : : +- ^ ProjectExecTransformer (54) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (53) : : :- ^ InputIteratorTransformer (43) : : : +- ShuffleQueryStage (41), Statistics(X) : : : +- ColumnarExchange (40) - : : : +- VeloxAppendBatches (39) + : : : +- VeloxResizeBatches (39) : : : +- ^ ProjectExecTransformer (37) : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (36) : : : :- ^ InputIteratorTransformer (26) : : : : +- ShuffleQueryStage (24), Statistics(X) : : : : +- ColumnarExchange (23) - : : : : +- VeloxAppendBatches (22) + : : : : +- VeloxResizeBatches (22) : : : : +- ^ ProjectExecTransformer (20) : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) : : : : :- ^ InputIteratorTransformer (9) : : : : : +- ShuffleQueryStage (7), Statistics(X) : : : : : +- ColumnarExchange (6) - : : : : : +- VeloxAppendBatches (5) + : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ FilterExecTransformer (2) : : : : : +- ^ Scan parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : +- ColumnarExchange (15) - : : : : +- VeloxAppendBatches (14) + : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) : : : : +- ^ Scan parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33), Statistics(X) : : : +- ColumnarExchange (32) - : : : +- VeloxAppendBatches (31) + : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ FilterExecTransformer (28) : : : +- ^ Scan parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50), Statistics(X) : : +- ColumnarExchange (49) - : : +- VeloxAppendBatches (48) + : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ FilterExecTransformer (45) : : +- ^ Scan parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67), Statistics(X) : +- ColumnarExchange (66) - : +- VeloxAppendBatches (65) + : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ FilterExecTransformer (62) : +- ^ Scan parquet (61) +- ^ InputIteratorTransformer (86) +- ShuffleQueryStage (84), Statistics(X) +- ColumnarExchange (83) - +- VeloxAppendBatches (82) + +- VeloxResizeBatches (82) +- ^ ProjectExecTransformer (80) +- ^ FilterExecTransformer (79) +- ^ Scan parquet (78) @@ -152,9 +152,9 @@ Input [2]: [p_partkey#X, p_name#X] Input [2]: [hash_partition_key#X, p_partkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [2]: [hash_partition_key#X, p_partkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [2]: [hash_partition_key#X, p_partkey#X] @@ -189,9 +189,9 @@ Input [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpric Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] @@ -220,9 +220,9 @@ Input [7]: [p_partkey#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] @@ -257,9 +257,9 @@ Input [2]: [s_suppkey#X, s_nationkey#X] Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -288,9 +288,9 @@ Input [8]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpric Input [8]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X] Arguments: false -(39) VeloxAppendBatches +(39) VeloxResizeBatches Input [8]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X] -Arguments: X +Arguments: X, X (40) ColumnarExchange Input [8]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X] @@ -325,9 +325,9 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_supplycost#X] Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_supplycost#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_supplycost#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_supplycost#X] @@ -356,9 +356,9 @@ Input [10]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpri Input [7]: [hash_partition_key#X, l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X] Arguments: false -(56) VeloxAppendBatches +(56) VeloxResizeBatches Input [7]: [hash_partition_key#X, l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X] -Arguments: X +Arguments: X, X (57) ColumnarExchange Input [7]: [hash_partition_key#X, l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X] @@ -393,9 +393,9 @@ Input [2]: [o_orderkey#X, o_orderdate#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X] Arguments: false -(65) VeloxAppendBatches +(65) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X] -Arguments: X +Arguments: X, X (66) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X] @@ -424,9 +424,9 @@ Input [8]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_natio Input [7]: [hash_partition_key#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X] Arguments: false -(73) VeloxAppendBatches +(73) VeloxResizeBatches Input [7]: [hash_partition_key#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X] -Arguments: X +Arguments: X, X (74) ColumnarExchange Input [7]: [hash_partition_key#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X] @@ -461,9 +461,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] Arguments: false -(82) VeloxAppendBatches +(82) VeloxResizeBatches Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] -Arguments: X +Arguments: X, X (83) ColumnarExchange Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] @@ -503,9 +503,9 @@ Input [4]: [nation#X, o_year#X, sum#X, isEmpty#X] Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] Arguments: false -(92) VeloxAppendBatches +(92) VeloxResizeBatches Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (93) ColumnarExchange Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] @@ -532,9 +532,9 @@ Results [3]: [nation#X, o_year#X, sum(amount#X)#X AS sum_profit#X] Input [3]: [nation#X, o_year#X, sum_profit#X] Arguments: false -(99) VeloxAppendBatches +(99) VeloxResizeBatches Input [3]: [nation#X, o_year#X, sum_profit#X] -Arguments: X +Arguments: X, X (100) ColumnarExchange Input [3]: [nation#X, o_year#X, sum_profit#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/1.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/1.txt index 1e53cd90e1b3..0c773785eafc 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/1.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/1.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (30) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ RegularHashAggregateExecTransformer (12) +- ^ InputIteratorTransformer (11) +- ShuffleQueryStage (9), Statistics(X) +- ColumnarExchange (8) - +- VeloxAppendBatches (7) + +- VeloxResizeBatches (7) +- ^ ProjectExecTransformer (5) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) @@ -58,9 +58,9 @@ Input [17]: [l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] Arguments: false -(7) VeloxAppendBatches +(7) VeloxResizeBatches Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] -Arguments: X +Arguments: X, X (8) ColumnarExchange Input [18]: [hash_partition_key#X, l_returnflag#X, l_linestatus#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, isEmpty#X, sum#X, count#X, sum#X, count#X, sum#X, count#X, count#X] @@ -87,9 +87,9 @@ Results [10]: [l_returnflag#X, l_linestatus#X, sum(l_quantity#X)#X AS sum_qty#X, Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [10]: [l_returnflag#X, l_linestatus#X, sum_qty#X, sum_base_price#X, sum_disc_price#X, sum_charge#X, avg_qty#X, avg_price#X, avg_disc#X, count_order#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/10.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/10.txt index d7376c740f93..3d4d5db424cd 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/10.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/10.txt @@ -8,7 +8,7 @@ AdaptiveSparkPlan (100) +- ^ InputIteratorTransformer (62) +- ShuffleQueryStage (60), Statistics(X) +- ColumnarExchange (59) - +- VeloxAppendBatches (58) + +- VeloxResizeBatches (58) +- ^ ProjectExecTransformer (56) +- ^ FlushableHashAggregateExecTransformer (55) +- ^ ProjectExecTransformer (54) @@ -16,40 +16,40 @@ AdaptiveSparkPlan (100) :- ^ InputIteratorTransformer (43) : +- ShuffleQueryStage (41), Statistics(X) : +- ColumnarExchange (40) - : +- VeloxAppendBatches (39) + : +- VeloxResizeBatches (39) : +- ^ ProjectExecTransformer (37) : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (36) : :- ^ InputIteratorTransformer (26) : : +- ShuffleQueryStage (24), Statistics(X) : : +- ColumnarExchange (23) - : : +- VeloxAppendBatches (22) + : : +- VeloxResizeBatches (22) : : +- ^ ProjectExecTransformer (20) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) : : :- ^ InputIteratorTransformer (9) : : : +- ShuffleQueryStage (7), Statistics(X) : : : +- ColumnarExchange (6) - : : : +- VeloxAppendBatches (5) + : : : +- VeloxResizeBatches (5) : : : +- ^ ProjectExecTransformer (3) : : : +- ^ FilterExecTransformer (2) : : : +- ^ Scan parquet (1) : : +- ^ InputIteratorTransformer (18) : : +- ShuffleQueryStage (16), Statistics(X) : : +- ColumnarExchange (15) - : : +- VeloxAppendBatches (14) + : : +- VeloxResizeBatches (14) : : +- ^ ProjectExecTransformer (12) : : +- ^ FilterExecTransformer (11) : : +- ^ Scan parquet (10) : +- ^ InputIteratorTransformer (35) : +- ShuffleQueryStage (33), Statistics(X) : +- ColumnarExchange (32) - : +- VeloxAppendBatches (31) + : +- VeloxResizeBatches (31) : +- ^ ProjectExecTransformer (29) : +- ^ FilterExecTransformer (28) : +- ^ Scan parquet (27) +- ^ InputIteratorTransformer (52) +- ShuffleQueryStage (50), Statistics(X) +- ColumnarExchange (49) - +- VeloxAppendBatches (48) + +- VeloxResizeBatches (48) +- ^ ProjectExecTransformer (46) +- ^ FilterExecTransformer (45) +- ^ Scan parquet (44) @@ -107,9 +107,9 @@ Input [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct Input [8]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [8]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [8]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] @@ -144,9 +144,9 @@ Input [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] @@ -176,9 +176,9 @@ Input [9]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct Input [9]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, o_orderkey#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [9]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, o_orderkey#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [9]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, o_orderkey#X] @@ -213,9 +213,9 @@ Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_returnflag#X] Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] @@ -245,9 +245,9 @@ Input [11]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acc Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, l_extendedprice#X, l_discount#X] Arguments: false -(39) VeloxAppendBatches +(39) VeloxResizeBatches Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (40) ColumnarExchange Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, l_extendedprice#X, l_discount#X] @@ -282,9 +282,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] @@ -325,9 +325,9 @@ Input [9]: [c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] Arguments: false -(58) VeloxAppendBatches +(58) VeloxResizeBatches Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (59) ColumnarExchange Input [10]: [hash_partition_key#X, c_custkey#X, c_name#X, c_acctbal#X, c_phone#X, n_name#X, c_address#X, c_comment#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/11.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/11.txt index c9371ffbf2c3..8dce41dc2898 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/11.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/11.txt @@ -6,13 +6,13 @@ AdaptiveSparkPlan (82) +- ^ InputIteratorTransformer (53) +- ShuffleQueryStage (51), Statistics(X) +- ColumnarExchange (50) - +- VeloxAppendBatches (49) + +- VeloxResizeBatches (49) +- ^ FilterExecTransformer (47) +- ^ RegularHashAggregateExecTransformer (46) +- ^ InputIteratorTransformer (45) +- ShuffleQueryStage (43), Statistics(X) +- ColumnarExchange (42) - +- VeloxAppendBatches (41) + +- VeloxResizeBatches (41) +- ^ ProjectExecTransformer (39) +- ^ FlushableHashAggregateExecTransformer (38) +- ^ ProjectExecTransformer (37) @@ -20,27 +20,27 @@ AdaptiveSparkPlan (82) :- ^ InputIteratorTransformer (26) : +- ShuffleQueryStage (24), Statistics(X) : +- ColumnarExchange (23) - : +- VeloxAppendBatches (22) + : +- VeloxResizeBatches (22) : +- ^ ProjectExecTransformer (20) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7), Statistics(X) : : +- ColumnarExchange (6) - : : +- VeloxAppendBatches (5) + : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16), Statistics(X) : +- ColumnarExchange (15) - : +- VeloxAppendBatches (14) + : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ FilterExecTransformer (11) : +- ^ Scan parquet (10) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33), Statistics(X) +- ColumnarExchange (32) - +- VeloxAppendBatches (31) + +- VeloxResizeBatches (31) +- ^ ProjectExecTransformer (29) +- ^ FilterExecTransformer (28) +- ^ Scan parquet (27) @@ -91,9 +91,9 @@ Input [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Input [5]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [5]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [5]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] @@ -128,9 +128,9 @@ Input [2]: [s_suppkey#X, s_nationkey#X] Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -160,9 +160,9 @@ Input [6]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X, s_suppke Input [5]: [hash_partition_key#X, ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [5]: [hash_partition_key#X, ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [5]: [hash_partition_key#X, ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] @@ -197,9 +197,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [2]: [hash_partition_key#X, n_nationkey#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [2]: [hash_partition_key#X, n_nationkey#X] @@ -240,9 +240,9 @@ Input [3]: [ps_partkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] Arguments: false -(41) VeloxAppendBatches +(41) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (42) ColumnarExchange Input [4]: [hash_partition_key#X, ps_partkey#X, sum#X, isEmpty#X] @@ -273,9 +273,9 @@ Arguments: (isnotnull(value#X) AND (cast(value#X as decimal(38,6)) > Subquery su Input [2]: [ps_partkey#X, value#X] Arguments: false -(49) VeloxAppendBatches +(49) VeloxResizeBatches Input [2]: [ps_partkey#X, value#X] -Arguments: X +Arguments: X, X (50) ColumnarExchange Input [2]: [ps_partkey#X, value#X] @@ -439,13 +439,13 @@ AdaptiveSparkPlan (136) :- ^ InputIteratorTransformer (103) : +- ShuffleQueryStage (101), Statistics(X) : +- ColumnarExchange (100) - : +- VeloxAppendBatches (99) + : +- VeloxResizeBatches (99) : +- ^ ProjectExecTransformer (97) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (96) : :- ^ InputIteratorTransformer (91) : : +- ShuffleQueryStage (89), Statistics(X) : : +- ColumnarExchange (88) - : : +- VeloxAppendBatches (87) + : : +- VeloxResizeBatches (87) : : +- ^ ProjectExecTransformer (85) : : +- ^ FilterExecTransformer (84) : : +- ^ Scan parquet (83) @@ -498,9 +498,9 @@ Input [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Input [4]: [hash_partition_key#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Arguments: false -(87) VeloxAppendBatches +(87) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] -Arguments: X +Arguments: X, X (88) ColumnarExchange Input [4]: [hash_partition_key#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] @@ -543,9 +543,9 @@ Input [5]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X, s_suppkey#X, s_nationk Input [4]: [hash_partition_key#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] Arguments: false -(99) VeloxAppendBatches +(99) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] -Arguments: X +Arguments: X, X (100) ColumnarExchange Input [4]: [hash_partition_key#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/12.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/12.txt index ce033f5468d1..ffa8c4b4ac02 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/12.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/12.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (55) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33), Statistics(X) +- ColumnarExchange (32) - +- VeloxAppendBatches (31) + +- VeloxResizeBatches (31) +- ^ RegularHashAggregateExecTransformer (29) +- ^ InputIteratorTransformer (28) +- ShuffleQueryStage (26), Statistics(X) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ ProjectExecTransformer (22) +- ^ FlushableHashAggregateExecTransformer (21) +- ^ ProjectExecTransformer (20) @@ -19,14 +19,14 @@ AdaptiveSparkPlan (55) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7), Statistics(X) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) +- ^ Scan parquet (10) @@ -68,9 +68,9 @@ Input [2]: [o_orderkey#X, o_orderpriority#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] @@ -105,9 +105,9 @@ Input [5]: [l_orderkey#X, l_shipdate#X, l_commitdate#X, l_receiptdate#X, l_shipm Input [3]: [hash_partition_key#X, l_orderkey#X, l_shipmode#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_shipmode#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_shipmode#X] @@ -148,9 +148,9 @@ Input [3]: [l_shipmode#X, sum#X, sum#X] Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [4]: [hash_partition_key#X, l_shipmode#X, sum#X, sum#X] @@ -177,9 +177,9 @@ Results [3]: [l_shipmode#X, sum(CASE WHEN ((o_orderpriority#X = 1-URGENT) OR (o_ Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [3]: [l_shipmode#X, high_line_count#X, low_line_count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/13.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/13.txt index c71d03b93e12..a2da9210750e 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/13.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/13.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (58) +- ^ InputIteratorTransformer (37) +- ShuffleQueryStage (35), Statistics(X) +- ColumnarExchange (34) - +- VeloxAppendBatches (33) + +- VeloxResizeBatches (33) +- ^ RegularHashAggregateExecTransformer (31) +- ^ InputIteratorTransformer (30) +- ShuffleQueryStage (28), Statistics(X) +- ColumnarExchange (27) - +- VeloxAppendBatches (26) + +- VeloxResizeBatches (26) +- ^ ProjectExecTransformer (24) +- ^ FlushableHashAggregateExecTransformer (23) +- ^ ProjectExecTransformer (22) @@ -22,13 +22,13 @@ AdaptiveSparkPlan (58) :- ^ InputIteratorTransformer (8) : +- ShuffleQueryStage (6), Statistics(X) : +- ColumnarExchange (5) - : +- VeloxAppendBatches (4) + : +- VeloxResizeBatches (4) : +- ^ ProjectExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (17) +- ShuffleQueryStage (15), Statistics(X) +- ColumnarExchange (14) - +- VeloxAppendBatches (13) + +- VeloxResizeBatches (13) +- ^ ProjectExecTransformer (11) +- ^ FilterExecTransformer (10) +- ^ Scan parquet (9) @@ -66,9 +66,9 @@ Input [1]: [c_custkey#X] Input [2]: [hash_partition_key#X, c_custkey#X] Arguments: false -(4) VeloxAppendBatches +(4) VeloxResizeBatches Input [2]: [hash_partition_key#X, c_custkey#X] -Arguments: X +Arguments: X, X (5) ColumnarExchange Input [2]: [hash_partition_key#X, c_custkey#X] @@ -103,9 +103,9 @@ Input [3]: [o_orderkey#X, o_custkey#X, o_comment#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] Arguments: false -(13) VeloxAppendBatches +(13) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] -Arguments: X +Arguments: X, X (14) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] @@ -164,9 +164,9 @@ Input [2]: [c_count#X, count#X] Input [3]: [hash_partition_key#X, c_count#X, count#X] Arguments: false -(26) VeloxAppendBatches +(26) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_count#X, count#X] -Arguments: X +Arguments: X, X (27) ColumnarExchange Input [3]: [hash_partition_key#X, c_count#X, count#X] @@ -193,9 +193,9 @@ Results [2]: [c_count#X, count(1)#X AS custdist#X] Input [2]: [c_count#X, custdist#X] Arguments: false -(33) VeloxAppendBatches +(33) VeloxResizeBatches Input [2]: [c_count#X, custdist#X] -Arguments: X +Arguments: X, X (34) ColumnarExchange Input [2]: [c_count#X, custdist#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/14.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/14.txt index 492d3f8b9d07..9994b8328915 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/14.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/14.txt @@ -10,14 +10,14 @@ AdaptiveSparkPlan (39) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7), Statistics(X) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) +- ^ Scan parquet (10) @@ -56,9 +56,9 @@ Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Input [4]: [hash_partition_key#X, l_partkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_partkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [4]: [hash_partition_key#X, l_partkey#X, l_extendedprice#X, l_discount#X] @@ -93,9 +93,9 @@ Input [2]: [p_partkey#X, p_type#X] Input [3]: [hash_partition_key#X, p_partkey#X, p_type#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, p_partkey#X, p_type#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, p_partkey#X, p_type#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/15.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/15.txt index 129e4ad927e9..2c9849a2b1cf 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/15.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/15.txt @@ -5,13 +5,13 @@ AdaptiveSparkPlan (47) +- AQEShuffleRead (29) +- ShuffleQueryStage (28), Statistics(X) +- ColumnarExchange (27) - +- VeloxAppendBatches (26) + +- VeloxResizeBatches (26) +- ^ ProjectExecTransformer (24) +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (23) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7), Statistics(X) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) @@ -20,7 +20,7 @@ AdaptiveSparkPlan (47) +- ^ InputIteratorTransformer (20) +- ShuffleQueryStage (18), Statistics(X) +- ColumnarExchange (17) - +- VeloxAppendBatches (16) + +- VeloxResizeBatches (16) +- ^ ProjectExecTransformer (14) +- ^ FlushableHashAggregateExecTransformer (13) +- ^ ProjectExecTransformer (12) @@ -64,9 +64,9 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_phone#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_phone#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_phone#X] @@ -112,9 +112,9 @@ Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] Arguments: false -(16) VeloxAppendBatches +(16) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (17) ColumnarExchange Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] @@ -155,9 +155,9 @@ Input [6]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, supplier_no#X, total_ Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] Arguments: false -(26) VeloxAppendBatches +(26) VeloxResizeBatches Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] -Arguments: X +Arguments: X, X (27) ColumnarExchange Input [5]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X, total_revenue#X] @@ -269,7 +269,7 @@ AdaptiveSparkPlan (73) +- ^ InputIteratorTransformer (58) +- ShuffleQueryStage (56), Statistics(X) +- ColumnarExchange (55) - +- VeloxAppendBatches (54) + +- VeloxResizeBatches (54) +- ^ ProjectExecTransformer (52) +- ^ FlushableHashAggregateExecTransformer (51) +- ^ ProjectExecTransformer (50) @@ -316,9 +316,9 @@ Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] Arguments: false -(54) VeloxAppendBatches +(54) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (55) ColumnarExchange Input [4]: [hash_partition_key#X, l_suppkey#X, sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/16.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/16.txt index 45b6041f8b4b..4d7f8090a3b1 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/16.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/16.txt @@ -6,19 +6,19 @@ AdaptiveSparkPlan (71) +- ^ InputIteratorTransformer (44) +- ShuffleQueryStage (42), Statistics(X) +- ColumnarExchange (41) - +- VeloxAppendBatches (40) + +- VeloxResizeBatches (40) +- ^ RegularHashAggregateExecTransformer (38) +- ^ InputIteratorTransformer (37) +- ShuffleQueryStage (35), Statistics(X) +- ColumnarExchange (34) - +- VeloxAppendBatches (33) + +- VeloxResizeBatches (33) +- ^ ProjectExecTransformer (31) +- ^ FlushableHashAggregateExecTransformer (30) +- ^ RegularHashAggregateExecTransformer (29) +- ^ InputIteratorTransformer (28) +- ShuffleQueryStage (26), Statistics(X) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ ProjectExecTransformer (22) +- ^ FlushableHashAggregateExecTransformer (21) +- ^ ProjectExecTransformer (20) @@ -26,14 +26,14 @@ AdaptiveSparkPlan (71) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7), Statistics(X) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) +- ^ Scan parquet (10) @@ -82,9 +82,9 @@ Input [2]: [ps_partkey#X, ps_suppkey#X] Input [3]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X] @@ -119,9 +119,9 @@ Input [4]: [p_partkey#X, p_brand#X, p_type#X, p_size#X] Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_type#X, p_size#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_type#X, p_size#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_type#X, p_size#X] @@ -162,9 +162,9 @@ Input [4]: [p_brand#X, p_type#X, p_size#X, ps_suppkey#X] Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, ps_suppkey#X] @@ -202,9 +202,9 @@ Input [4]: [p_brand#X, p_type#X, p_size#X, count#X] Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] Arguments: false -(33) VeloxAppendBatches +(33) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] -Arguments: X +Arguments: X, X (34) ColumnarExchange Input [5]: [hash_partition_key#X, p_brand#X, p_type#X, p_size#X, count#X] @@ -231,9 +231,9 @@ Results [4]: [p_brand#X, p_type#X, p_size#X, count(ps_suppkey#X)#X AS supplier_c Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] Arguments: false -(40) VeloxAppendBatches +(40) VeloxResizeBatches Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] -Arguments: X +Arguments: X, X (41) ColumnarExchange Input [4]: [p_brand#X, p_type#X, p_size#X, supplier_cnt#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/17.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/17.txt index b46b3e3f2724..aba53c7c1f32 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/17.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/17.txt @@ -12,14 +12,14 @@ AdaptiveSparkPlan (63) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7), Statistics(X) : : +- ColumnarExchange (6) - : : +- VeloxAppendBatches (5) + : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16), Statistics(X) : +- ColumnarExchange (15) - : +- VeloxAppendBatches (14) + : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ FilterExecTransformer (11) : +- ^ Scan parquet (10) @@ -29,7 +29,7 @@ AdaptiveSparkPlan (63) +- ^ InputIteratorTransformer (30) +- ShuffleQueryStage (28), Statistics(X) +- ColumnarExchange (27) - +- VeloxAppendBatches (26) + +- VeloxResizeBatches (26) +- ^ ProjectExecTransformer (24) +- ^ FlushableHashAggregateExecTransformer (23) +- ^ FilterExecTransformer (22) @@ -78,9 +78,9 @@ Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Input [4]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [4]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X] @@ -115,9 +115,9 @@ Input [3]: [p_partkey#X, p_brand#X, p_container#X] Input [2]: [hash_partition_key#X, p_partkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [2]: [hash_partition_key#X, p_partkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [2]: [hash_partition_key#X, p_partkey#X] @@ -169,9 +169,9 @@ Input [3]: [l_partkey#X, sum#X, count#X] Input [4]: [hash_partition_key#X, l_partkey#X, sum#X, count#X] Arguments: false -(26) VeloxAppendBatches +(26) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_partkey#X, sum#X, count#X] -Arguments: X +Arguments: X, X (27) ColumnarExchange Input [4]: [hash_partition_key#X, l_partkey#X, sum#X, count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/18.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/18.txt index febb48962446..740918c259fc 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/18.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/18.txt @@ -10,26 +10,26 @@ AdaptiveSparkPlan (110) :- ^ InputIteratorTransformer (46) : +- ShuffleQueryStage (44), Statistics(X) : +- ColumnarExchange (43) - : +- VeloxAppendBatches (42) + : +- VeloxResizeBatches (42) : +- ^ ProjectExecTransformer (40) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (39) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7), Statistics(X) : : +- ColumnarExchange (6) - : : +- VeloxAppendBatches (5) + : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (38) : +- ShuffleQueryStage (36), Statistics(X) : +- ColumnarExchange (35) - : +- VeloxAppendBatches (34) + : +- VeloxResizeBatches (34) : +- ^ ProjectExecTransformer (32) : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (31) : :- ^ InputIteratorTransformer (18) : : +- ShuffleQueryStage (16), Statistics(X) : : +- ColumnarExchange (15) - : : +- VeloxAppendBatches (14) + : : +- VeloxResizeBatches (14) : : +- ^ ProjectExecTransformer (12) : : +- ^ FilterExecTransformer (11) : : +- ^ Scan parquet (10) @@ -39,7 +39,7 @@ AdaptiveSparkPlan (110) : +- ^ InputIteratorTransformer (27) : +- ShuffleQueryStage (25), Statistics(X) : +- ColumnarExchange (24) - : +- VeloxAppendBatches (23) + : +- VeloxResizeBatches (23) : +- ^ ProjectExecTransformer (21) : +- ^ FlushableHashAggregateExecTransformer (20) : +- ^ Scan parquet (19) @@ -47,7 +47,7 @@ AdaptiveSparkPlan (110) :- ^ InputIteratorTransformer (55) : +- ShuffleQueryStage (53), Statistics(X) : +- ColumnarExchange (52) - : +- VeloxAppendBatches (51) + : +- VeloxResizeBatches (51) : +- ^ ProjectExecTransformer (49) : +- ^ FilterExecTransformer (48) : +- ^ Scan parquet (47) @@ -118,9 +118,9 @@ Input [2]: [c_custkey#X, c_name#X] Input [3]: [hash_partition_key#X, c_custkey#X, c_name#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_custkey#X, c_name#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, c_custkey#X, c_name#X] @@ -155,9 +155,9 @@ Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] @@ -194,9 +194,9 @@ Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] Arguments: false -(23) VeloxAppendBatches +(23) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (24) ColumnarExchange Input [4]: [hash_partition_key#X, l_orderkey#X, sum#X, isEmpty#X] @@ -241,9 +241,9 @@ Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: false -(34) VeloxAppendBatches +(34) VeloxResizeBatches Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] -Arguments: X +Arguments: X, X (35) ColumnarExchange Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] @@ -273,9 +273,9 @@ Input [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_ Input [6]: [hash_partition_key#X, c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] Arguments: false -(42) VeloxAppendBatches +(42) VeloxResizeBatches Input [6]: [hash_partition_key#X, c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] -Arguments: X +Arguments: X, X (43) ColumnarExchange Input [6]: [hash_partition_key#X, c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] @@ -310,9 +310,9 @@ Input [2]: [l_orderkey#X, l_quantity#X] Input [3]: [hash_partition_key#X, l_orderkey#X, l_quantity#X] Arguments: false -(51) VeloxAppendBatches +(51) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_quantity#X] -Arguments: X +Arguments: X, X (52) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_quantity#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/19.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/19.txt index fa78645313e4..1acbbcee646f 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/19.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/19.txt @@ -9,14 +9,14 @@ AdaptiveSparkPlan (38) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7), Statistics(X) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) +- ^ Scan parquet (10) @@ -55,9 +55,9 @@ Input [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipin Input [5]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [5]: [hash_partition_key#X, l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] @@ -92,9 +92,9 @@ Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_size#X, p_container#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_size#X, p_container#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [5]: [hash_partition_key#X, p_partkey#X, p_brand#X, p_size#X, p_container#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/20.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/20.txt index bb9987fc32c1..54b461c23e0f 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/20.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/20.txt @@ -5,52 +5,52 @@ AdaptiveSparkPlan (143) +- AQEShuffleRead (92) +- ShuffleQueryStage (91), Statistics(X) +- ColumnarExchange (90) - +- VeloxAppendBatches (89) + +- VeloxResizeBatches (89) +- ^ ProjectExecTransformer (87) +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (86) :- ^ InputIteratorTransformer (76) : +- ShuffleQueryStage (74), Statistics(X) : +- ColumnarExchange (73) - : +- VeloxAppendBatches (72) + : +- VeloxResizeBatches (72) : +- ^ ProjectExecTransformer (70) : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (69) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7), Statistics(X) : : +- ColumnarExchange (6) - : : +- VeloxAppendBatches (5) + : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (68) : +- ShuffleQueryStage (66), Statistics(X) : +- ColumnarExchange (65) - : +- VeloxAppendBatches (64) + : +- VeloxResizeBatches (64) : +- ^ ProjectExecTransformer (62) : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (61) : :- ^ InputIteratorTransformer (35) : : +- ShuffleQueryStage (33), Statistics(X) : : +- ColumnarExchange (32) - : : +- VeloxAppendBatches (31) + : : +- VeloxResizeBatches (31) : : +- ^ ProjectExecTransformer (29) : : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (28) : : :- ^ InputIteratorTransformer (18) : : : +- ShuffleQueryStage (16), Statistics(X) : : : +- ColumnarExchange (15) - : : : +- VeloxAppendBatches (14) + : : : +- VeloxResizeBatches (14) : : : +- ^ ProjectExecTransformer (12) : : : +- ^ FilterExecTransformer (11) : : : +- ^ Scan parquet (10) : : +- ^ InputIteratorTransformer (27) : : +- ShuffleQueryStage (25), Statistics(X) : : +- ColumnarExchange (24) - : : +- VeloxAppendBatches (23) + : : +- VeloxResizeBatches (23) : : +- ^ ProjectExecTransformer (21) : : +- ^ FilterExecTransformer (20) : : +- ^ Scan parquet (19) : +- ^ InputIteratorTransformer (60) : +- ShuffleQueryStage (58), Statistics(X) : +- ColumnarExchange (57) - : +- VeloxAppendBatches (56) + : +- VeloxResizeBatches (56) : +- ^ ProjectExecTransformer (54) : +- ^ FilterExecTransformer (53) : +- ^ ProjectExecTransformer (52) @@ -60,7 +60,7 @@ AdaptiveSparkPlan (143) : :- ^ InputIteratorTransformer (44) : : +- ShuffleQueryStage (42), Statistics(X) : : +- ColumnarExchange (41) - : : +- VeloxAppendBatches (40) + : : +- VeloxResizeBatches (40) : : +- ^ ProjectExecTransformer (38) : : +- ^ FilterExecTransformer (37) : : +- ^ Scan parquet (36) @@ -70,7 +70,7 @@ AdaptiveSparkPlan (143) +- ^ InputIteratorTransformer (85) +- ShuffleQueryStage (83), Statistics(X) +- ColumnarExchange (82) - +- VeloxAppendBatches (81) + +- VeloxResizeBatches (81) +- ^ ProjectExecTransformer (79) +- ^ FilterExecTransformer (78) +- ^ Scan parquet (77) @@ -145,9 +145,9 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [5]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] @@ -182,9 +182,9 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] @@ -219,9 +219,9 @@ Input [2]: [p_partkey#X, p_name#X] Input [2]: [hash_partition_key#X, p_partkey#X] Arguments: false -(23) VeloxAppendBatches +(23) VeloxResizeBatches Input [2]: [hash_partition_key#X, p_partkey#X] -Arguments: X +Arguments: X, X (24) ColumnarExchange Input [2]: [hash_partition_key#X, p_partkey#X] @@ -251,9 +251,9 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_availqty#X] @@ -288,9 +288,9 @@ Input [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] Input [4]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, l_quantity#X] Arguments: false -(40) VeloxAppendBatches +(40) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, l_quantity#X] -Arguments: X +Arguments: X, X (41) ColumnarExchange Input [4]: [hash_partition_key#X, l_partkey#X, l_suppkey#X, l_quantity#X] @@ -355,9 +355,9 @@ Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: false -(56) VeloxAppendBatches +(56) VeloxResizeBatches Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -Arguments: X +Arguments: X, X (57) ColumnarExchange Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] @@ -387,9 +387,9 @@ Input [6]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, (0.5 * sum(l_quantity))#X Input [2]: [hash_partition_key#X, ps_suppkey#X] Arguments: false -(64) VeloxAppendBatches +(64) VeloxResizeBatches Input [2]: [hash_partition_key#X, ps_suppkey#X] -Arguments: X +Arguments: X, X (65) ColumnarExchange Input [2]: [hash_partition_key#X, ps_suppkey#X] @@ -419,9 +419,9 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] Arguments: false -(72) VeloxAppendBatches +(72) VeloxResizeBatches Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] -Arguments: X +Arguments: X, X (73) ColumnarExchange Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] @@ -456,9 +456,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: false -(81) VeloxAppendBatches +(81) VeloxResizeBatches Input [2]: [hash_partition_key#X, n_nationkey#X] -Arguments: X +Arguments: X, X (82) ColumnarExchange Input [2]: [hash_partition_key#X, n_nationkey#X] @@ -488,9 +488,9 @@ Input [4]: [s_name#X, s_address#X, s_nationkey#X, n_nationkey#X] Input [2]: [s_name#X, s_address#X] Arguments: false -(89) VeloxAppendBatches +(89) VeloxResizeBatches Input [2]: [s_name#X, s_address#X] -Arguments: X +Arguments: X, X (90) ColumnarExchange Input [2]: [s_name#X, s_address#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/21.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/21.txt index 5e8c9ad9f92a..e752cdadeec9 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/21.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/21.txt @@ -6,7 +6,7 @@ AdaptiveSparkPlan (137) +- ^ InputIteratorTransformer (88) +- ShuffleQueryStage (86), Statistics(X) +- ColumnarExchange (85) - +- VeloxAppendBatches (84) + +- VeloxResizeBatches (84) +- ^ ProjectExecTransformer (82) +- ^ FlushableHashAggregateExecTransformer (81) +- ^ ProjectExecTransformer (80) @@ -14,60 +14,60 @@ AdaptiveSparkPlan (137) :- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67), Statistics(X) : +- ColumnarExchange (66) - : +- VeloxAppendBatches (65) + : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (62) : :- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50), Statistics(X) : : +- ColumnarExchange (49) - : : +- VeloxAppendBatches (48) + : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (45) : : :- ^ InputIteratorTransformer (9) : : : +- ShuffleQueryStage (7), Statistics(X) : : : +- ColumnarExchange (6) - : : : +- VeloxAppendBatches (5) + : : : +- VeloxResizeBatches (5) : : : +- ^ ProjectExecTransformer (3) : : : +- ^ FilterExecTransformer (2) : : : +- ^ Scan parquet (1) : : +- ^ InputIteratorTransformer (44) : : +- ShuffleQueryStage (42), Statistics(X) : : +- ColumnarExchange (41) - : : +- VeloxAppendBatches (40) + : : +- VeloxResizeBatches (40) : : +- ^ ProjectExecTransformer (38) : : +- ^ ShuffledHashJoinExecTransformer LeftAnti BuildRight (37) : : :- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (27) : : : :- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : +- ColumnarExchange (15) - : : : : +- VeloxAppendBatches (14) + : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) : : : : +- ^ Scan parquet (10) : : : +- ^ InputIteratorTransformer (26) : : : +- ShuffleQueryStage (24), Statistics(X) : : : +- ColumnarExchange (23) - : : : +- VeloxAppendBatches (22) + : : : +- VeloxResizeBatches (22) : : : +- ^ ProjectExecTransformer (20) : : : +- ^ Scan parquet (19) : : +- ^ InputIteratorTransformer (36) : : +- ShuffleQueryStage (34), Statistics(X) : : +- ColumnarExchange (33) - : : +- VeloxAppendBatches (32) + : : +- VeloxResizeBatches (32) : : +- ^ ProjectExecTransformer (30) : : +- ^ FilterExecTransformer (29) : : +- ^ Scan parquet (28) : +- ^ InputIteratorTransformer (61) : +- ShuffleQueryStage (59), Statistics(X) : +- ColumnarExchange (58) - : +- VeloxAppendBatches (57) + : +- VeloxResizeBatches (57) : +- ^ ProjectExecTransformer (55) : +- ^ FilterExecTransformer (54) : +- ^ Scan parquet (53) +- ^ InputIteratorTransformer (78) +- ShuffleQueryStage (76), Statistics(X) +- ColumnarExchange (75) - +- VeloxAppendBatches (74) + +- VeloxResizeBatches (74) +- ^ ProjectExecTransformer (72) +- ^ FilterExecTransformer (71) +- ^ Scan parquet (70) @@ -138,9 +138,9 @@ Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X] Input [4]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_nationkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [4]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_nationkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [4]: [hash_partition_key#X, s_suppkey#X, s_name#X, s_nationkey#X] @@ -175,9 +175,9 @@ Input [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] @@ -207,9 +207,9 @@ Input [2]: [l_orderkey#X, l_suppkey#X] Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] @@ -250,9 +250,9 @@ Input [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] Arguments: false -(32) VeloxAppendBatches +(32) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] -Arguments: X +Arguments: X, X (33) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] @@ -282,9 +282,9 @@ Input [2]: [l_orderkey#X, l_suppkey#X] Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] Arguments: false -(40) VeloxAppendBatches +(40) VeloxResizeBatches Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] -Arguments: X +Arguments: X, X (41) ColumnarExchange Input [3]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X] @@ -314,9 +314,9 @@ Input [5]: [s_suppkey#X, s_name#X, s_nationkey#X, l_orderkey#X, l_suppkey#X] Input [4]: [hash_partition_key#X, s_name#X, s_nationkey#X, l_orderkey#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [4]: [hash_partition_key#X, s_name#X, s_nationkey#X, l_orderkey#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [4]: [hash_partition_key#X, s_name#X, s_nationkey#X, l_orderkey#X] @@ -351,9 +351,9 @@ Input [2]: [o_orderkey#X, o_orderstatus#X] Input [2]: [hash_partition_key#X, o_orderkey#X] Arguments: false -(57) VeloxAppendBatches +(57) VeloxResizeBatches Input [2]: [hash_partition_key#X, o_orderkey#X] -Arguments: X +Arguments: X, X (58) ColumnarExchange Input [2]: [hash_partition_key#X, o_orderkey#X] @@ -383,9 +383,9 @@ Input [4]: [s_name#X, s_nationkey#X, l_orderkey#X, o_orderkey#X] Input [3]: [hash_partition_key#X, s_name#X, s_nationkey#X] Arguments: false -(65) VeloxAppendBatches +(65) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_name#X, s_nationkey#X] -Arguments: X +Arguments: X, X (66) ColumnarExchange Input [3]: [hash_partition_key#X, s_name#X, s_nationkey#X] @@ -420,9 +420,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: false -(74) VeloxAppendBatches +(74) VeloxResizeBatches Input [2]: [hash_partition_key#X, n_nationkey#X] -Arguments: X +Arguments: X, X (75) ColumnarExchange Input [2]: [hash_partition_key#X, n_nationkey#X] @@ -463,9 +463,9 @@ Input [2]: [s_name#X, count#X] Input [3]: [hash_partition_key#X, s_name#X, count#X] Arguments: false -(84) VeloxAppendBatches +(84) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_name#X, count#X] -Arguments: X +Arguments: X, X (85) ColumnarExchange Input [3]: [hash_partition_key#X, s_name#X, count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/22.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/22.txt index 50f1c1bdef30..f4597ae85ae2 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/22.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/22.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (52) +- ^ InputIteratorTransformer (34) +- ShuffleQueryStage (32), Statistics(X) +- ColumnarExchange (31) - +- VeloxAppendBatches (30) + +- VeloxResizeBatches (30) +- ^ RegularHashAggregateExecTransformer (28) +- ^ InputIteratorTransformer (27) +- ShuffleQueryStage (25), Statistics(X) +- ColumnarExchange (24) - +- VeloxAppendBatches (23) + +- VeloxResizeBatches (23) +- ^ ProjectExecTransformer (21) +- ^ FlushableHashAggregateExecTransformer (20) +- ^ ProjectExecTransformer (19) @@ -19,14 +19,14 @@ AdaptiveSparkPlan (52) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7), Statistics(X) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (17) +- ShuffleQueryStage (15), Statistics(X) +- ColumnarExchange (14) - +- VeloxAppendBatches (13) + +- VeloxResizeBatches (13) +- ^ ProjectExecTransformer (11) +- ^ Scan parquet (10) +- == Initial Plan == @@ -65,9 +65,9 @@ Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X] Input [4]: [hash_partition_key#X, c_custkey#X, c_phone#X, c_acctbal#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [4]: [hash_partition_key#X, c_custkey#X, c_phone#X, c_acctbal#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [4]: [hash_partition_key#X, c_custkey#X, c_phone#X, c_acctbal#X] @@ -97,9 +97,9 @@ Input [1]: [o_custkey#X] Input [2]: [hash_partition_key#X, o_custkey#X] Arguments: false -(13) VeloxAppendBatches +(13) VeloxResizeBatches Input [2]: [hash_partition_key#X, o_custkey#X] -Arguments: X +Arguments: X, X (14) ColumnarExchange Input [2]: [hash_partition_key#X, o_custkey#X] @@ -140,9 +140,9 @@ Input [4]: [cntrycode#X, count#X, sum#X, isEmpty#X] Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] Arguments: false -(23) VeloxAppendBatches +(23) VeloxResizeBatches Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (24) ColumnarExchange Input [5]: [hash_partition_key#X, cntrycode#X, count#X, sum#X, isEmpty#X] @@ -169,9 +169,9 @@ Results [3]: [cntrycode#X, count(1)#X AS numcust#X, sum(c_acctbal#X)#X AS totacc Input [3]: [cntrycode#X, numcust#X, totacctbal#X] Arguments: false -(30) VeloxAppendBatches +(30) VeloxResizeBatches Input [3]: [cntrycode#X, numcust#X, totacctbal#X] -Arguments: X +Arguments: X, X (31) ColumnarExchange Input [3]: [cntrycode#X, numcust#X, totacctbal#X] @@ -281,7 +281,7 @@ AdaptiveSparkPlan (72) +- ^ InputIteratorTransformer (62) +- ShuffleQueryStage (60), Statistics(X) +- ColumnarExchange (59) - +- VeloxAppendBatches (58) + +- VeloxResizeBatches (58) +- ^ FlushableHashAggregateExecTransformer (56) +- ^ ProjectExecTransformer (55) +- ^ FilterExecTransformer (54) @@ -321,9 +321,9 @@ Results [2]: [sum#X, count#X] Input [2]: [sum#X, count#X] Arguments: false -(58) VeloxAppendBatches +(58) VeloxResizeBatches Input [2]: [sum#X, count#X] -Arguments: X +Arguments: X, X (59) ColumnarExchange Input [2]: [sum#X, count#X] @@ -398,7 +398,7 @@ AdaptiveSparkPlan (72) +- ^ InputIteratorTransformer (62) +- ShuffleQueryStage (60), Statistics(X) +- ColumnarExchange (59) - +- VeloxAppendBatches (58) + +- VeloxResizeBatches (58) +- ^ FlushableHashAggregateExecTransformer (56) +- ^ ProjectExecTransformer (55) +- ^ FilterExecTransformer (54) diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/3.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/3.txt index 50ad3b59c347..6ae748ed5b02 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/3.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/3.txt @@ -11,27 +11,27 @@ AdaptiveSparkPlan (67) :- ^ InputIteratorTransformer (26) : +- ShuffleQueryStage (24), Statistics(X) : +- ColumnarExchange (23) - : +- VeloxAppendBatches (22) + : +- VeloxResizeBatches (22) : +- ^ ProjectExecTransformer (20) : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7), Statistics(X) : : +- ColumnarExchange (6) - : : +- VeloxAppendBatches (5) + : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) : : +- ^ Scan parquet (1) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16), Statistics(X) : +- ColumnarExchange (15) - : +- VeloxAppendBatches (14) + : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ FilterExecTransformer (11) : +- ^ Scan parquet (10) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33), Statistics(X) +- ColumnarExchange (32) - +- VeloxAppendBatches (31) + +- VeloxResizeBatches (31) +- ^ ProjectExecTransformer (29) +- ^ FilterExecTransformer (28) +- ^ Scan parquet (27) @@ -80,9 +80,9 @@ Input [2]: [c_custkey#X, c_mktsegment#X] Input [2]: [hash_partition_key#X, c_custkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [2]: [hash_partition_key#X, c_custkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [2]: [hash_partition_key#X, c_custkey#X] @@ -117,9 +117,9 @@ Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [5]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] @@ -149,9 +149,9 @@ Input [5]: [c_custkey#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriorit Input [4]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X, o_shippriority#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [4]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X, o_shippriority#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [4]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X, o_shippriority#X] @@ -186,9 +186,9 @@ Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [4]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/4.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/4.txt index 2db46753e9fc..a51a10913f36 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/4.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/4.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (56) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33), Statistics(X) +- ColumnarExchange (32) - +- VeloxAppendBatches (31) + +- VeloxResizeBatches (31) +- ^ RegularHashAggregateExecTransformer (29) +- ^ InputIteratorTransformer (28) +- ShuffleQueryStage (26), Statistics(X) +- ColumnarExchange (25) - +- VeloxAppendBatches (24) + +- VeloxResizeBatches (24) +- ^ ProjectExecTransformer (22) +- ^ FlushableHashAggregateExecTransformer (21) +- ^ ProjectExecTransformer (20) @@ -19,14 +19,14 @@ AdaptiveSparkPlan (56) :- ^ InputIteratorTransformer (9) : +- ShuffleQueryStage (7), Statistics(X) : +- ColumnarExchange (6) - : +- VeloxAppendBatches (5) + : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) : +- ^ Scan parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) - +- VeloxAppendBatches (14) + +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) +- ^ Scan parquet (10) @@ -69,9 +69,9 @@ Input [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderpriority#X] @@ -106,9 +106,9 @@ Input [3]: [l_orderkey#X, l_commitdate#X, l_receiptdate#X] Input [2]: [hash_partition_key#X, l_orderkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [2]: [hash_partition_key#X, l_orderkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [2]: [hash_partition_key#X, l_orderkey#X] @@ -149,9 +149,9 @@ Input [2]: [o_orderpriority#X, count#X] Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] Arguments: false -(24) VeloxAppendBatches +(24) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] -Arguments: X +Arguments: X, X (25) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderpriority#X, count#X] @@ -178,9 +178,9 @@ Results [2]: [o_orderpriority#X, count(1)#X AS order_count#X] Input [2]: [o_orderpriority#X, order_count#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [2]: [o_orderpriority#X, order_count#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [2]: [o_orderpriority#X, order_count#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/5.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/5.txt index 07a5c86709f4..15425399ceb8 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/5.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/5.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (156) +- ^ InputIteratorTransformer (103) +- ShuffleQueryStage (101), Statistics(X) +- ColumnarExchange (100) - +- VeloxAppendBatches (99) + +- VeloxResizeBatches (99) +- ^ RegularHashAggregateExecTransformer (97) +- ^ InputIteratorTransformer (96) +- ShuffleQueryStage (94), Statistics(X) +- ColumnarExchange (93) - +- VeloxAppendBatches (92) + +- VeloxResizeBatches (92) +- ^ ProjectExecTransformer (90) +- ^ FlushableHashAggregateExecTransformer (89) +- ^ ProjectExecTransformer (88) @@ -19,66 +19,66 @@ AdaptiveSparkPlan (156) :- ^ InputIteratorTransformer (77) : +- ShuffleQueryStage (75), Statistics(X) : +- ColumnarExchange (74) - : +- VeloxAppendBatches (73) + : +- VeloxResizeBatches (73) : +- ^ ProjectExecTransformer (71) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (70) : :- ^ InputIteratorTransformer (60) : : +- ShuffleQueryStage (58), Statistics(X) : : +- ColumnarExchange (57) - : : +- VeloxAppendBatches (56) + : : +- VeloxResizeBatches (56) : : +- ^ ProjectExecTransformer (54) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (53) : : :- ^ InputIteratorTransformer (43) : : : +- ShuffleQueryStage (41), Statistics(X) : : : +- ColumnarExchange (40) - : : : +- VeloxAppendBatches (39) + : : : +- VeloxResizeBatches (39) : : : +- ^ ProjectExecTransformer (37) : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (36) : : : :- ^ InputIteratorTransformer (26) : : : : +- ShuffleQueryStage (24), Statistics(X) : : : : +- ColumnarExchange (23) - : : : : +- VeloxAppendBatches (22) + : : : : +- VeloxResizeBatches (22) : : : : +- ^ ProjectExecTransformer (20) : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) : : : : :- ^ InputIteratorTransformer (9) : : : : : +- ShuffleQueryStage (7), Statistics(X) : : : : : +- ColumnarExchange (6) - : : : : : +- VeloxAppendBatches (5) + : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ FilterExecTransformer (2) : : : : : +- ^ Scan parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : +- ColumnarExchange (15) - : : : : +- VeloxAppendBatches (14) + : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) : : : : +- ^ Scan parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33), Statistics(X) : : : +- ColumnarExchange (32) - : : : +- VeloxAppendBatches (31) + : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ FilterExecTransformer (28) : : : +- ^ Scan parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50), Statistics(X) : : +- ColumnarExchange (49) - : : +- VeloxAppendBatches (48) + : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ FilterExecTransformer (45) : : +- ^ Scan parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67), Statistics(X) : +- ColumnarExchange (66) - : +- VeloxAppendBatches (65) + : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ FilterExecTransformer (62) : +- ^ Scan parquet (61) +- ^ InputIteratorTransformer (86) +- ShuffleQueryStage (84), Statistics(X) +- ColumnarExchange (83) - +- VeloxAppendBatches (82) + +- VeloxResizeBatches (82) +- ^ ProjectExecTransformer (80) +- ^ FilterExecTransformer (79) +- ^ Scan parquet (78) @@ -153,9 +153,9 @@ Input [2]: [c_custkey#X, c_nationkey#X] Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] @@ -190,9 +190,9 @@ Input [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] @@ -222,9 +222,9 @@ Input [4]: [c_custkey#X, c_nationkey#X, o_orderkey#X, o_custkey#X] Input [3]: [hash_partition_key#X, c_nationkey#X, o_orderkey#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_nationkey#X, o_orderkey#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [3]: [hash_partition_key#X, c_nationkey#X, o_orderkey#X] @@ -259,9 +259,9 @@ Input [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] @@ -291,9 +291,9 @@ Input [6]: [c_nationkey#X, o_orderkey#X, l_orderkey#X, l_suppkey#X, l_extendedpr Input [5]: [hash_partition_key#X, c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(39) VeloxAppendBatches +(39) VeloxResizeBatches Input [5]: [hash_partition_key#X, c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (40) ColumnarExchange Input [5]: [hash_partition_key#X, c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] @@ -328,9 +328,9 @@ Input [2]: [s_suppkey#X, s_nationkey#X] Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -360,9 +360,9 @@ Input [6]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, s_suppk Input [4]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X] Arguments: false -(56) VeloxAppendBatches +(56) VeloxResizeBatches Input [4]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X] -Arguments: X +Arguments: X, X (57) ColumnarExchange Input [4]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X] @@ -397,9 +397,9 @@ Input [3]: [n_nationkey#X, n_name#X, n_regionkey#X] Input [4]: [hash_partition_key#X, n_nationkey#X, n_name#X, n_regionkey#X] Arguments: false -(65) VeloxAppendBatches +(65) VeloxResizeBatches Input [4]: [hash_partition_key#X, n_nationkey#X, n_name#X, n_regionkey#X] -Arguments: X +Arguments: X, X (66) ColumnarExchange Input [4]: [hash_partition_key#X, n_nationkey#X, n_name#X, n_regionkey#X] @@ -429,9 +429,9 @@ Input [6]: [l_extendedprice#X, l_discount#X, s_nationkey#X, n_nationkey#X, n_nam Input [5]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] Arguments: false -(73) VeloxAppendBatches +(73) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] -Arguments: X +Arguments: X, X (74) ColumnarExchange Input [5]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] @@ -466,9 +466,9 @@ Input [2]: [r_regionkey#X, r_name#X] Input [2]: [hash_partition_key#X, r_regionkey#X] Arguments: false -(82) VeloxAppendBatches +(82) VeloxResizeBatches Input [2]: [hash_partition_key#X, r_regionkey#X] -Arguments: X +Arguments: X, X (83) ColumnarExchange Input [2]: [hash_partition_key#X, r_regionkey#X] @@ -509,9 +509,9 @@ Input [3]: [n_name#X, sum#X, isEmpty#X] Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] Arguments: false -(92) VeloxAppendBatches +(92) VeloxResizeBatches Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (93) ColumnarExchange Input [4]: [hash_partition_key#X, n_name#X, sum#X, isEmpty#X] @@ -538,9 +538,9 @@ Results [2]: [n_name#X, sum((l_extendedprice#X * (1 - l_discount#X)))#X AS reven Input [2]: [n_name#X, revenue#X] Arguments: false -(99) VeloxAppendBatches +(99) VeloxResizeBatches Input [2]: [n_name#X, revenue#X] -Arguments: X +Arguments: X, X (100) ColumnarExchange Input [2]: [n_name#X, revenue#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/6.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/6.txt index ddc921e22d0f..9463fa1da9d9 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/6.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/6.txt @@ -6,7 +6,7 @@ AdaptiveSparkPlan (20) +- ^ InputIteratorTransformer (10) +- ShuffleQueryStage (8), Statistics(X) +- ColumnarExchange (7) - +- VeloxAppendBatches (6) + +- VeloxResizeBatches (6) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ FilterExecTransformer (2) @@ -46,9 +46,9 @@ Results [2]: [sum#X, isEmpty#X] Input [2]: [sum#X, isEmpty#X] Arguments: false -(6) VeloxAppendBatches +(6) VeloxResizeBatches Input [2]: [sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (7) ColumnarExchange Input [2]: [sum#X, isEmpty#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/7.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/7.txt index b27398e415d3..2960110d5b70 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/7.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/7.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (149) +- ^ InputIteratorTransformer (98) +- ShuffleQueryStage (96), Statistics(X) +- ColumnarExchange (95) - +- VeloxAppendBatches (94) + +- VeloxResizeBatches (94) +- ^ RegularHashAggregateExecTransformer (92) +- ^ InputIteratorTransformer (91) +- ShuffleQueryStage (89), Statistics(X) +- ColumnarExchange (88) - +- VeloxAppendBatches (87) + +- VeloxResizeBatches (87) +- ^ ProjectExecTransformer (85) +- ^ FlushableHashAggregateExecTransformer (84) +- ^ ProjectExecTransformer (83) @@ -19,59 +19,59 @@ AdaptiveSparkPlan (149) :- ^ InputIteratorTransformer (77) : +- ShuffleQueryStage (75), Statistics(X) : +- ColumnarExchange (74) - : +- VeloxAppendBatches (73) + : +- VeloxResizeBatches (73) : +- ^ ProjectExecTransformer (71) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (70) : :- ^ InputIteratorTransformer (60) : : +- ShuffleQueryStage (58), Statistics(X) : : +- ColumnarExchange (57) - : : +- VeloxAppendBatches (56) + : : +- VeloxResizeBatches (56) : : +- ^ ProjectExecTransformer (54) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (53) : : :- ^ InputIteratorTransformer (43) : : : +- ShuffleQueryStage (41), Statistics(X) : : : +- ColumnarExchange (40) - : : : +- VeloxAppendBatches (39) + : : : +- VeloxResizeBatches (39) : : : +- ^ ProjectExecTransformer (37) : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (36) : : : :- ^ InputIteratorTransformer (26) : : : : +- ShuffleQueryStage (24), Statistics(X) : : : : +- ColumnarExchange (23) - : : : : +- VeloxAppendBatches (22) + : : : : +- VeloxResizeBatches (22) : : : : +- ^ ProjectExecTransformer (20) : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) : : : : :- ^ InputIteratorTransformer (9) : : : : : +- ShuffleQueryStage (7), Statistics(X) : : : : : +- ColumnarExchange (6) - : : : : : +- VeloxAppendBatches (5) + : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ FilterExecTransformer (2) : : : : : +- ^ Scan parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : +- ColumnarExchange (15) - : : : : +- VeloxAppendBatches (14) + : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) : : : : +- ^ Scan parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33), Statistics(X) : : : +- ColumnarExchange (32) - : : : +- VeloxAppendBatches (31) + : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ FilterExecTransformer (28) : : : +- ^ Scan parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50), Statistics(X) : : +- ColumnarExchange (49) - : : +- VeloxAppendBatches (48) + : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ FilterExecTransformer (45) : : +- ^ Scan parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67), Statistics(X) : +- ColumnarExchange (66) - : +- VeloxAppendBatches (65) + : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ FilterExecTransformer (62) : +- ^ Scan parquet (61) @@ -147,9 +147,9 @@ Input [2]: [s_suppkey#X, s_nationkey#X] Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -184,9 +184,9 @@ Input [5]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipda Input [6]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [6]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] @@ -216,9 +216,9 @@ Input [7]: [s_suppkey#X, s_nationkey#X, l_orderkey#X, l_suppkey#X, l_extendedpri Input [6]: [hash_partition_key#X, s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [6]: [hash_partition_key#X, s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [6]: [hash_partition_key#X, s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] @@ -253,9 +253,9 @@ Input [2]: [o_orderkey#X, o_custkey#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_custkey#X] @@ -285,9 +285,9 @@ Input [7]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_ship Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X] Arguments: false -(39) VeloxAppendBatches +(39) VeloxResizeBatches Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X] -Arguments: X +Arguments: X, X (40) ColumnarExchange Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X] @@ -322,9 +322,9 @@ Input [2]: [c_custkey#X, c_nationkey#X] Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] @@ -354,9 +354,9 @@ Input [7]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_cust Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X] Arguments: false -(56) VeloxAppendBatches +(56) VeloxResizeBatches Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X] -Arguments: X +Arguments: X, X (57) ColumnarExchange Input [6]: [hash_partition_key#X, s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X] @@ -391,9 +391,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] Arguments: false -(65) VeloxAppendBatches +(65) VeloxResizeBatches Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] -Arguments: X +Arguments: X, X (66) ColumnarExchange Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] @@ -423,9 +423,9 @@ Input [7]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nati Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X, n_name#X] Arguments: false -(73) VeloxAppendBatches +(73) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X, n_name#X] -Arguments: X +Arguments: X, X (74) ColumnarExchange Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X, n_name#X] @@ -479,9 +479,9 @@ Input [5]: [supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] Arguments: false -(87) VeloxAppendBatches +(87) VeloxResizeBatches Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (88) ColumnarExchange Input [6]: [hash_partition_key#X, supp_nation#X, cust_nation#X, l_year#X, sum#X, isEmpty#X] @@ -508,9 +508,9 @@ Results [4]: [supp_nation#X, cust_nation#X, l_year#X, sum(volume#X)#X AS revenue Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] Arguments: false -(94) VeloxAppendBatches +(94) VeloxResizeBatches Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] -Arguments: X +Arguments: X, X (95) ColumnarExchange Input [4]: [supp_nation#X, cust_nation#X, l_year#X, revenue#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/8.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/8.txt index fa2a2789f4d3..3cacef9cdd5e 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/8.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/8.txt @@ -6,13 +6,13 @@ AdaptiveSparkPlan (207) +- ^ InputIteratorTransformer (138) +- ShuffleQueryStage (136), Statistics(X) +- ColumnarExchange (135) - +- VeloxAppendBatches (134) + +- VeloxResizeBatches (134) +- ^ ProjectExecTransformer (132) +- ^ RegularHashAggregateExecTransformer (131) +- ^ InputIteratorTransformer (130) +- ShuffleQueryStage (128), Statistics(X) +- ColumnarExchange (127) - +- VeloxAppendBatches (126) + +- VeloxResizeBatches (126) +- ^ ProjectExecTransformer (124) +- ^ FlushableHashAggregateExecTransformer (123) +- ^ ProjectExecTransformer (122) @@ -20,92 +20,92 @@ AdaptiveSparkPlan (207) :- ^ InputIteratorTransformer (111) : +- ShuffleQueryStage (109), Statistics(X) : +- ColumnarExchange (108) - : +- VeloxAppendBatches (107) + : +- VeloxResizeBatches (107) : +- ^ ProjectExecTransformer (105) : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (104) : :- ^ InputIteratorTransformer (94) : : +- ShuffleQueryStage (92), Statistics(X) : : +- ColumnarExchange (91) - : : +- VeloxAppendBatches (90) + : : +- VeloxResizeBatches (90) : : +- ^ ProjectExecTransformer (88) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (87) : : :- ^ InputIteratorTransformer (77) : : : +- ShuffleQueryStage (75), Statistics(X) : : : +- ColumnarExchange (74) - : : : +- VeloxAppendBatches (73) + : : : +- VeloxResizeBatches (73) : : : +- ^ ProjectExecTransformer (71) : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (70) : : : :- ^ InputIteratorTransformer (60) : : : : +- ShuffleQueryStage (58), Statistics(X) : : : : +- ColumnarExchange (57) - : : : : +- VeloxAppendBatches (56) + : : : : +- VeloxResizeBatches (56) : : : : +- ^ ProjectExecTransformer (54) : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (53) : : : : :- ^ InputIteratorTransformer (43) : : : : : +- ShuffleQueryStage (41), Statistics(X) : : : : : +- ColumnarExchange (40) - : : : : : +- VeloxAppendBatches (39) + : : : : : +- VeloxResizeBatches (39) : : : : : +- ^ ProjectExecTransformer (37) : : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (36) : : : : : :- ^ InputIteratorTransformer (26) : : : : : : +- ShuffleQueryStage (24), Statistics(X) : : : : : : +- ColumnarExchange (23) - : : : : : : +- VeloxAppendBatches (22) + : : : : : : +- VeloxResizeBatches (22) : : : : : : +- ^ ProjectExecTransformer (20) : : : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) : : : : : : :- ^ InputIteratorTransformer (9) : : : : : : : +- ShuffleQueryStage (7), Statistics(X) : : : : : : : +- ColumnarExchange (6) - : : : : : : : +- VeloxAppendBatches (5) + : : : : : : : +- VeloxResizeBatches (5) : : : : : : : +- ^ ProjectExecTransformer (3) : : : : : : : +- ^ FilterExecTransformer (2) : : : : : : : +- ^ Scan parquet (1) : : : : : : +- ^ InputIteratorTransformer (18) : : : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : : : +- ColumnarExchange (15) - : : : : : : +- VeloxAppendBatches (14) + : : : : : : +- VeloxResizeBatches (14) : : : : : : +- ^ ProjectExecTransformer (12) : : : : : : +- ^ FilterExecTransformer (11) : : : : : : +- ^ Scan parquet (10) : : : : : +- ^ InputIteratorTransformer (35) : : : : : +- ShuffleQueryStage (33), Statistics(X) : : : : : +- ColumnarExchange (32) - : : : : : +- VeloxAppendBatches (31) + : : : : : +- VeloxResizeBatches (31) : : : : : +- ^ ProjectExecTransformer (29) : : : : : +- ^ FilterExecTransformer (28) : : : : : +- ^ Scan parquet (27) : : : : +- ^ InputIteratorTransformer (52) : : : : +- ShuffleQueryStage (50), Statistics(X) : : : : +- ColumnarExchange (49) - : : : : +- VeloxAppendBatches (48) + : : : : +- VeloxResizeBatches (48) : : : : +- ^ ProjectExecTransformer (46) : : : : +- ^ FilterExecTransformer (45) : : : : +- ^ Scan parquet (44) : : : +- ^ InputIteratorTransformer (69) : : : +- ShuffleQueryStage (67), Statistics(X) : : : +- ColumnarExchange (66) - : : : +- VeloxAppendBatches (65) + : : : +- VeloxResizeBatches (65) : : : +- ^ ProjectExecTransformer (63) : : : +- ^ FilterExecTransformer (62) : : : +- ^ Scan parquet (61) : : +- ^ InputIteratorTransformer (86) : : +- ShuffleQueryStage (84), Statistics(X) : : +- ColumnarExchange (83) - : : +- VeloxAppendBatches (82) + : : +- VeloxResizeBatches (82) : : +- ^ ProjectExecTransformer (80) : : +- ^ FilterExecTransformer (79) : : +- ^ Scan parquet (78) : +- ^ InputIteratorTransformer (103) : +- ShuffleQueryStage (101), Statistics(X) : +- ColumnarExchange (100) - : +- VeloxAppendBatches (99) + : +- VeloxResizeBatches (99) : +- ^ ProjectExecTransformer (97) : +- ^ FilterExecTransformer (96) : +- ^ Scan parquet (95) +- ^ InputIteratorTransformer (120) +- ShuffleQueryStage (118), Statistics(X) +- ColumnarExchange (117) - +- VeloxAppendBatches (116) + +- VeloxResizeBatches (116) +- ^ ProjectExecTransformer (114) +- ^ FilterExecTransformer (113) +- ^ Scan parquet (112) @@ -196,9 +196,9 @@ Input [2]: [p_partkey#X, p_type#X] Input [2]: [hash_partition_key#X, p_partkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [2]: [hash_partition_key#X, p_partkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [2]: [hash_partition_key#X, p_partkey#X] @@ -233,9 +233,9 @@ Input [5]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discoun Input [6]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [6]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] @@ -265,9 +265,9 @@ Input [6]: [p_partkey#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [5]: [hash_partition_key#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] @@ -302,9 +302,9 @@ Input [2]: [s_suppkey#X, s_nationkey#X] Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -334,9 +334,9 @@ Input [6]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, s_suppke Input [5]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] Arguments: false -(39) VeloxAppendBatches +(39) VeloxResizeBatches Input [5]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] -Arguments: X +Arguments: X, X (40) ColumnarExchange Input [5]: [hash_partition_key#X, l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] @@ -371,9 +371,9 @@ Input [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Input [4]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [4]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [4]: [hash_partition_key#X, o_orderkey#X, o_custkey#X, o_orderdate#X] @@ -403,9 +403,9 @@ Input [7]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orde Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X] Arguments: false -(56) VeloxAppendBatches +(56) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X] -Arguments: X +Arguments: X, X (57) ColumnarExchange Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X] @@ -440,9 +440,9 @@ Input [2]: [c_custkey#X, c_nationkey#X] Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] Arguments: false -(65) VeloxAppendBatches +(65) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] -Arguments: X +Arguments: X, X (66) ColumnarExchange Input [3]: [hash_partition_key#X, c_custkey#X, c_nationkey#X] @@ -472,9 +472,9 @@ Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_order Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X] Arguments: false -(73) VeloxAppendBatches +(73) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X] -Arguments: X +Arguments: X, X (74) ColumnarExchange Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X] @@ -509,9 +509,9 @@ Input [2]: [n_nationkey#X, n_regionkey#X] Input [3]: [hash_partition_key#X, n_nationkey#X, n_regionkey#X] Arguments: false -(82) VeloxAppendBatches +(82) VeloxResizeBatches Input [3]: [hash_partition_key#X, n_nationkey#X, n_regionkey#X] -Arguments: X +Arguments: X, X (83) ColumnarExchange Input [3]: [hash_partition_key#X, n_nationkey#X, n_regionkey#X] @@ -541,9 +541,9 @@ Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nat Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X] Arguments: false -(90) VeloxAppendBatches +(90) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X] -Arguments: X +Arguments: X, X (91) ColumnarExchange Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X] @@ -578,9 +578,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] Arguments: false -(99) VeloxAppendBatches +(99) VeloxResizeBatches Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] -Arguments: X +Arguments: X, X (100) ColumnarExchange Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] @@ -610,9 +610,9 @@ Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_reg Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X] Arguments: false -(107) VeloxAppendBatches +(107) VeloxResizeBatches Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X] -Arguments: X +Arguments: X, X (108) ColumnarExchange Input [6]: [hash_partition_key#X, l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X] @@ -647,9 +647,9 @@ Input [2]: [r_regionkey#X, r_name#X] Input [2]: [hash_partition_key#X, r_regionkey#X] Arguments: false -(116) VeloxAppendBatches +(116) VeloxResizeBatches Input [2]: [hash_partition_key#X, r_regionkey#X] -Arguments: X +Arguments: X, X (117) ColumnarExchange Input [2]: [hash_partition_key#X, r_regionkey#X] @@ -690,9 +690,9 @@ Input [5]: [o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] Arguments: false -(126) VeloxAppendBatches +(126) VeloxResizeBatches Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (127) ColumnarExchange Input [6]: [hash_partition_key#X, o_year#X, sum#X, isEmpty#X, sum#X, isEmpty#X] @@ -723,9 +723,9 @@ Input [3]: [o_year#X, sum(CASE WHEN (nation#X = BRAZIL) THEN volume#X ELSE 0.000 Input [2]: [o_year#X, mkt_share#X] Arguments: false -(134) VeloxAppendBatches +(134) VeloxResizeBatches Input [2]: [o_year#X, mkt_share#X] -Arguments: X +Arguments: X, X (135) ColumnarExchange Input [2]: [o_year#X, mkt_share#X] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/9.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/9.txt index 3000cbae7a6d..61b6401046ab 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/9.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/9.txt @@ -6,12 +6,12 @@ AdaptiveSparkPlan (155) +- ^ InputIteratorTransformer (103) +- ShuffleQueryStage (101), Statistics(X) +- ColumnarExchange (100) - +- VeloxAppendBatches (99) + +- VeloxResizeBatches (99) +- ^ RegularHashAggregateExecTransformer (97) +- ^ InputIteratorTransformer (96) +- ShuffleQueryStage (94), Statistics(X) +- ColumnarExchange (93) - +- VeloxAppendBatches (92) + +- VeloxResizeBatches (92) +- ^ ProjectExecTransformer (90) +- ^ FlushableHashAggregateExecTransformer (89) +- ^ ProjectExecTransformer (88) @@ -19,66 +19,66 @@ AdaptiveSparkPlan (155) :- ^ InputIteratorTransformer (77) : +- ShuffleQueryStage (75), Statistics(X) : +- ColumnarExchange (74) - : +- VeloxAppendBatches (73) + : +- VeloxResizeBatches (73) : +- ^ ProjectExecTransformer (71) : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (70) : :- ^ InputIteratorTransformer (60) : : +- ShuffleQueryStage (58), Statistics(X) : : +- ColumnarExchange (57) - : : +- VeloxAppendBatches (56) + : : +- VeloxResizeBatches (56) : : +- ^ ProjectExecTransformer (54) : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (53) : : :- ^ InputIteratorTransformer (43) : : : +- ShuffleQueryStage (41), Statistics(X) : : : +- ColumnarExchange (40) - : : : +- VeloxAppendBatches (39) + : : : +- VeloxResizeBatches (39) : : : +- ^ ProjectExecTransformer (37) : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (36) : : : :- ^ InputIteratorTransformer (26) : : : : +- ShuffleQueryStage (24), Statistics(X) : : : : +- ColumnarExchange (23) - : : : : +- VeloxAppendBatches (22) + : : : : +- VeloxResizeBatches (22) : : : : +- ^ ProjectExecTransformer (20) : : : : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) : : : : :- ^ InputIteratorTransformer (9) : : : : : +- ShuffleQueryStage (7), Statistics(X) : : : : : +- ColumnarExchange (6) - : : : : : +- VeloxAppendBatches (5) + : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ FilterExecTransformer (2) : : : : : +- ^ Scan parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : +- ColumnarExchange (15) - : : : : +- VeloxAppendBatches (14) + : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) : : : : +- ^ Scan parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33), Statistics(X) : : : +- ColumnarExchange (32) - : : : +- VeloxAppendBatches (31) + : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ FilterExecTransformer (28) : : : +- ^ Scan parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50), Statistics(X) : : +- ColumnarExchange (49) - : : +- VeloxAppendBatches (48) + : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ FilterExecTransformer (45) : : +- ^ Scan parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67), Statistics(X) : +- ColumnarExchange (66) - : +- VeloxAppendBatches (65) + : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ FilterExecTransformer (62) : +- ^ Scan parquet (61) +- ^ InputIteratorTransformer (86) +- ShuffleQueryStage (84), Statistics(X) +- ColumnarExchange (83) - +- VeloxAppendBatches (82) + +- VeloxResizeBatches (82) +- ^ ProjectExecTransformer (80) +- ^ FilterExecTransformer (79) +- ^ Scan parquet (78) @@ -152,9 +152,9 @@ Input [2]: [p_partkey#X, p_name#X] Input [2]: [hash_partition_key#X, p_partkey#X] Arguments: false -(5) VeloxAppendBatches +(5) VeloxResizeBatches Input [2]: [hash_partition_key#X, p_partkey#X] -Arguments: X +Arguments: X, X (6) ColumnarExchange Input [2]: [hash_partition_key#X, p_partkey#X] @@ -189,9 +189,9 @@ Input [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpric Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Arguments: false -(14) VeloxAppendBatches +(14) VeloxResizeBatches Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (15) ColumnarExchange Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] @@ -221,9 +221,9 @@ Input [7]: [p_partkey#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Arguments: false -(22) VeloxAppendBatches +(22) VeloxResizeBatches Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -Arguments: X +Arguments: X, X (23) ColumnarExchange Input [7]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] @@ -258,9 +258,9 @@ Input [2]: [s_suppkey#X, s_nationkey#X] Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] Arguments: false -(31) VeloxAppendBatches +(31) VeloxResizeBatches Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] -Arguments: X +Arguments: X, X (32) ColumnarExchange Input [3]: [hash_partition_key#X, s_suppkey#X, s_nationkey#X] @@ -290,9 +290,9 @@ Input [8]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpric Input [8]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X] Arguments: false -(39) VeloxAppendBatches +(39) VeloxResizeBatches Input [8]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X] -Arguments: X +Arguments: X, X (40) ColumnarExchange Input [8]: [hash_partition_key#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X] @@ -327,9 +327,9 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_supplycost#X] Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_supplycost#X] Arguments: false -(48) VeloxAppendBatches +(48) VeloxResizeBatches Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_supplycost#X] -Arguments: X +Arguments: X, X (49) ColumnarExchange Input [4]: [hash_partition_key#X, ps_partkey#X, ps_suppkey#X, ps_supplycost#X] @@ -359,9 +359,9 @@ Input [10]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpri Input [7]: [hash_partition_key#X, l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X] Arguments: false -(56) VeloxAppendBatches +(56) VeloxResizeBatches Input [7]: [hash_partition_key#X, l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X] -Arguments: X +Arguments: X, X (57) ColumnarExchange Input [7]: [hash_partition_key#X, l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X] @@ -396,9 +396,9 @@ Input [2]: [o_orderkey#X, o_orderdate#X] Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X] Arguments: false -(65) VeloxAppendBatches +(65) VeloxResizeBatches Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X] -Arguments: X +Arguments: X, X (66) ColumnarExchange Input [3]: [hash_partition_key#X, o_orderkey#X, o_orderdate#X] @@ -428,9 +428,9 @@ Input [8]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_natio Input [7]: [hash_partition_key#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X] Arguments: false -(73) VeloxAppendBatches +(73) VeloxResizeBatches Input [7]: [hash_partition_key#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X] -Arguments: X +Arguments: X, X (74) ColumnarExchange Input [7]: [hash_partition_key#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X] @@ -465,9 +465,9 @@ Input [2]: [n_nationkey#X, n_name#X] Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] Arguments: false -(82) VeloxAppendBatches +(82) VeloxResizeBatches Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] -Arguments: X +Arguments: X, X (83) ColumnarExchange Input [3]: [hash_partition_key#X, n_nationkey#X, n_name#X] @@ -508,9 +508,9 @@ Input [4]: [nation#X, o_year#X, sum#X, isEmpty#X] Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] Arguments: false -(92) VeloxAppendBatches +(92) VeloxResizeBatches Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] -Arguments: X +Arguments: X, X (93) ColumnarExchange Input [5]: [hash_partition_key#X, nation#X, o_year#X, sum#X, isEmpty#X] @@ -537,9 +537,9 @@ Results [3]: [nation#X, o_year#X, sum(amount#X)#X AS sum_profit#X] Input [3]: [nation#X, o_year#X, sum_profit#X] Arguments: false -(99) VeloxAppendBatches +(99) VeloxResizeBatches Input [3]: [nation#X, o_year#X, sum_profit#X] -Arguments: X +Arguments: X, X (100) ColumnarExchange Input [3]: [nation#X, o_year#X, sum_profit#X] diff --git a/backends-velox/src/test/scala/org/apache/gluten/execution/TestOperator.scala b/backends-velox/src/test/scala/org/apache/gluten/execution/TestOperator.scala index dcae4920d01c..fe6ac23908e5 100644 --- a/backends-velox/src/test/scala/org/apache/gluten/execution/TestOperator.scala +++ b/backends-velox/src/test/scala/org/apache/gluten/execution/TestOperator.scala @@ -833,16 +833,18 @@ class TestOperator extends VeloxWholeStageTransformerSuite with AdaptiveSparkPla test("combine small batches before shuffle") { val minBatchSize = 15 + val maxBatchSize = 100 withSQLConf( - "spark.gluten.sql.columnar.backend.velox.coalesceBatchesBeforeShuffle" -> "true", + "spark.gluten.sql.columnar.backend.velox.resizeBatches.shuffleInput" -> "true", "spark.gluten.sql.columnar.maxBatchSize" -> "2", - "spark.gluten.sql.columnar.backend.velox.minBatchSizeForShuffle" -> s"$minBatchSize" + "spark.gluten.sql.columnar.backend.velox.resizeBatches.shuffleInput.range" -> + s"$minBatchSize~$maxBatchSize" ) { val df = runQueryAndCompare( "select l_orderkey, sum(l_partkey) as sum from lineitem " + "where l_orderkey < 100 group by l_orderkey") { _ => } checkLengthAndPlan(df, 27) - val ops = collect(df.queryExecution.executedPlan) { case p: VeloxAppendBatchesExec => p } + val ops = collect(df.queryExecution.executedPlan) { case p: VeloxResizeBatchesExec => p } assert(ops.size == 1) val op = ops.head assert(op.minOutputBatchSize == minBatchSize) @@ -854,6 +856,31 @@ class TestOperator extends VeloxWholeStageTransformerSuite with AdaptiveSparkPla } } + test("split small batches before shuffle") { + val minBatchSize = 1 + val maxBatchSize = 4 + withSQLConf( + "spark.gluten.sql.columnar.backend.velox.resizeBatches.shuffleInput" -> "true", + "spark.gluten.sql.columnar.maxBatchSize" -> "100", + "spark.gluten.sql.columnar.backend.velox.resizeBatches.shuffleInput.range" -> + s"$minBatchSize~$maxBatchSize" + ) { + val df = runQueryAndCompare( + "select l_orderkey, sum(l_partkey) as sum from lineitem " + + "where l_orderkey < 100 group by l_orderkey") { _ => } + checkLengthAndPlan(df, 27) + val ops = collect(df.queryExecution.executedPlan) { case p: VeloxResizeBatchesExec => p } + assert(ops.size == 1) + val op = ops.head + assert(op.minOutputBatchSize == minBatchSize) + val metrics = op.metrics + assert(metrics("numInputRows").value == 27) + assert(metrics("numInputBatches").value == 1) + assert(metrics("numOutputRows").value == 27) + assert(metrics("numOutputBatches").value == 7) + } + } + test("test OneRowRelation") { val df = sql("SELECT 1") checkAnswer(df, Row(1)) diff --git a/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxTPCHSuite.scala b/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxTPCHSuite.scala index 792ab407836c..798cea8cb94f 100644 --- a/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxTPCHSuite.scala +++ b/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxTPCHSuite.scala @@ -70,6 +70,7 @@ abstract class VeloxTPCHSuite extends VeloxTPCHTableSupport { // for unexpected blank .replaceAll("Scan parquet ", "Scan parquet") // Spark QueryStageExec will take it's id as argument, replace it with X + .replaceAll("Arguments: [0-9]+, [0-9]+", "Arguments: X, X") .replaceAll("Arguments: [0-9]+", "Arguments: X") // mask PullOutPostProject and PullOutPreProject id .replaceAll("_pre_[0-9]*", "_pre_X") diff --git a/cpp/velox/CMakeLists.txt b/cpp/velox/CMakeLists.txt index 4c37d1c28fbe..f1aa96277cb8 100644 --- a/cpp/velox/CMakeLists.txt +++ b/cpp/velox/CMakeLists.txt @@ -527,7 +527,7 @@ set(VELOX_SRCS utils/VeloxArrowUtils.cc utils/ConfigExtractor.cc utils/Common.cc - utils/VeloxBatchAppender.cc) + utils/VeloxBatchResizer.cc) if(ENABLE_HDFS) list(APPEND VELOX_SRCS utils/HdfsUtils.cc) diff --git a/cpp/velox/jni/VeloxJniWrapper.cc b/cpp/velox/jni/VeloxJniWrapper.cc index f9387188bc69..cb49abd7d466 100644 --- a/cpp/velox/jni/VeloxJniWrapper.cc +++ b/cpp/velox/jni/VeloxJniWrapper.cc @@ -31,7 +31,7 @@ #include "memory/VeloxMemoryManager.h" #include "substrait/SubstraitToVeloxPlanValidator.h" #include "utils/ObjectStore.h" -#include "utils/VeloxBatchAppender.h" +#include "utils/VeloxBatchResizer.h" #include "velox/common/base/BloomFilter.h" #include @@ -244,17 +244,18 @@ JNIEXPORT jbyteArray JNICALL Java_org_apache_gluten_utils_VeloxBloomFilterJniWra JNI_METHOD_END(nullptr) } -JNIEXPORT jlong JNICALL Java_org_apache_gluten_utils_VeloxBatchAppenderJniWrapper_create( // NOLINT +JNIEXPORT jlong JNICALL Java_org_apache_gluten_utils_VeloxBatchResizerJniWrapper_create( // NOLINT JNIEnv* env, jobject wrapper, jint minOutputBatchSize, + jint maxOutputBatchSize, jobject jIter) { JNI_METHOD_START auto ctx = gluten::getRuntime(env, wrapper); auto pool = dynamic_cast(ctx->memoryManager())->getLeafMemoryPool(); auto iter = gluten::makeJniColumnarBatchIterator(env, jIter, ctx, nullptr); auto appender = std::make_shared( - std::make_unique(pool.get(), minOutputBatchSize, std::move(iter))); + std::make_unique(pool.get(), minOutputBatchSize, maxOutputBatchSize, std::move(iter))); return ctx->saveObject(appender); JNI_METHOD_END(gluten::kInvalidObjectHandle) } diff --git a/cpp/velox/utils/VeloxBatchAppender.cc b/cpp/velox/utils/VeloxBatchAppender.cc deleted file mode 100644 index 8fa1ade217e0..000000000000 --- a/cpp/velox/utils/VeloxBatchAppender.cc +++ /dev/null @@ -1,59 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -#include "VeloxBatchAppender.h" - -namespace gluten { - -gluten::VeloxBatchAppender::VeloxBatchAppender( - facebook::velox::memory::MemoryPool* pool, - int32_t minOutputBatchSize, - std::unique_ptr in) - : pool_(pool), minOutputBatchSize_(minOutputBatchSize), in_(std::move(in)) {} - -std::shared_ptr VeloxBatchAppender::next() { - auto cb = in_->next(); - if (cb == nullptr) { - // Input iterator was drained. - return nullptr; - } - if (cb->numRows() >= minOutputBatchSize_) { - // Fast flush path. - return cb; - } - - auto vb = VeloxColumnarBatch::from(pool_, cb); - auto rv = vb->getRowVector(); - auto buffer = facebook::velox::RowVector::createEmpty(rv->type(), pool_); - buffer->append(rv.get()); - - for (auto nextCb = in_->next(); nextCb != nullptr; nextCb = in_->next()) { - auto nextVb = VeloxColumnarBatch::from(pool_, nextCb); - auto nextRv = nextVb->getRowVector(); - buffer->append(nextRv.get()); - if (buffer->size() >= minOutputBatchSize_) { - // Buffer is full. - break; - } - } - return std::make_shared(buffer); -} - -int64_t VeloxBatchAppender::spillFixedSize(int64_t size) { - return in_->spillFixedSize(size); -} -} // namespace gluten diff --git a/cpp/velox/utils/VeloxBatchResizer.cc b/cpp/velox/utils/VeloxBatchResizer.cc new file mode 100644 index 000000000000..7b51463068c9 --- /dev/null +++ b/cpp/velox/utils/VeloxBatchResizer.cc @@ -0,0 +1,111 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "VeloxBatchResizer.h" + +namespace gluten { + +namespace { +class SliceRowVector : public ColumnarBatchIterator { + public: + SliceRowVector(int32_t maxOutputBatchSize, facebook::velox::RowVectorPtr in) + : maxOutputBatchSize_(maxOutputBatchSize), in_(in) { + GLUTEN_CHECK(in->size() > maxOutputBatchSize, "Invalid state"); + } + + std::shared_ptr next() override { + int32_t remainingLength = in_->size() - cursor_; + GLUTEN_CHECK(remainingLength >= 0, "Invalid state"); + if (remainingLength == 0) { + return nullptr; + } + int32_t sliceLength = std::min(maxOutputBatchSize_, remainingLength); + auto out = std::dynamic_pointer_cast(in_->slice(cursor_, sliceLength)); + cursor_ += sliceLength; + GLUTEN_CHECK(out != nullptr, "Invalid state"); + return std::make_shared(out); + } + + private: + int32_t maxOutputBatchSize_; + facebook::velox::RowVectorPtr in_; + int32_t cursor_ = 0; +}; +} // namespace + +gluten::VeloxBatchResizer::VeloxBatchResizer( + facebook::velox::memory::MemoryPool* pool, + int32_t minOutputBatchSize, + int32_t maxOutputBatchSize, + std::unique_ptr in) + : pool_(pool), + minOutputBatchSize_(minOutputBatchSize), + maxOutputBatchSize_(maxOutputBatchSize), + in_(std::move(in)) {} + +std::shared_ptr VeloxBatchResizer::next() { + if (next_) { + auto next = next_->next(); + if (next != nullptr) { + return next; + } + // Cached output was drained. Continue reading data from input iterator. + next_ = nullptr; + } + + auto cb = in_->next(); + if (cb == nullptr) { + // Input iterator was drained. + return nullptr; + } + + if (cb->numRows() < minOutputBatchSize_) { + auto vb = VeloxColumnarBatch::from(pool_, cb); + auto rv = vb->getRowVector(); + auto buffer = facebook::velox::RowVector::createEmpty(rv->type(), pool_); + buffer->append(rv.get()); + + for (auto nextCb = in_->next(); nextCb != nullptr; nextCb = in_->next()) { + auto nextVb = VeloxColumnarBatch::from(pool_, nextCb); + auto nextRv = nextVb->getRowVector(); + buffer->append(nextRv.get()); + if (buffer->size() >= minOutputBatchSize_) { + // Buffer is full. + break; + } + } + return std::make_shared(buffer); + } + + if (cb->numRows() > maxOutputBatchSize_) { + auto vb = VeloxColumnarBatch::from(pool_, cb); + auto rv = vb->getRowVector(); + GLUTEN_CHECK(next_ == nullptr, "Invalid state"); + next_ = std::make_unique(maxOutputBatchSize_, rv); + auto next = next_->next(); + GLUTEN_CHECK(next != nullptr, "Invalid state"); + return next; + } + + // Fast flush path. + return cb; +} + +int64_t VeloxBatchResizer::spillFixedSize(int64_t size) { + return in_->spillFixedSize(size); +} +} // namespace gluten diff --git a/cpp/velox/utils/VeloxBatchAppender.h b/cpp/velox/utils/VeloxBatchResizer.h similarity index 87% rename from cpp/velox/utils/VeloxBatchAppender.h rename to cpp/velox/utils/VeloxBatchResizer.h index 3698381d0add..09cdf4d51170 100644 --- a/cpp/velox/utils/VeloxBatchAppender.h +++ b/cpp/velox/utils/VeloxBatchResizer.h @@ -22,11 +22,12 @@ #include "velox/vector/ComplexVector.h" namespace gluten { -class VeloxBatchAppender : public ColumnarBatchIterator { +class VeloxBatchResizer : public ColumnarBatchIterator { public: - VeloxBatchAppender( + VeloxBatchResizer( facebook::velox::memory::MemoryPool* pool, int32_t minOutputBatchSize, + int32_t maxOutputBatchSize, std::unique_ptr in); std::shared_ptr next() override; @@ -36,6 +37,9 @@ class VeloxBatchAppender : public ColumnarBatchIterator { private: facebook::velox::memory::MemoryPool* pool_; const int32_t minOutputBatchSize_; + const int32_t maxOutputBatchSize_; std::unique_ptr in_; + + std::unique_ptr next_ = nullptr; }; } // namespace gluten diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/errors/GlutenQueryExecutionErrorsSuite.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/errors/GlutenQueryExecutionErrorsSuite.scala index c25e65cf0b68..8c661f86023a 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/errors/GlutenQueryExecutionErrorsSuite.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/errors/GlutenQueryExecutionErrorsSuite.scala @@ -27,7 +27,7 @@ class GlutenQueryExecutionErrorsSuite // Disables VeloxAppendBatches in which GeneralOutIterator wraps vanilla Spark's exceptions // with GlutenException. super.sparkConf - .set("spark.gluten.sql.columnar.backend.velox.coalesceBatchesBeforeShuffle", "false") + .set("spark.gluten.sql.columnar.backend.velox.resizeBatches.shuffleInput", "false") } override protected def getResourceParquetFilePath(name: String): String = { diff --git a/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala b/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala index 5547feafe331..d4a4be15e521 100644 --- a/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala +++ b/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala @@ -305,14 +305,32 @@ class GlutenConfig(conf: SQLConf) extends Logging { def veloxBloomFilterMaxNumBits: Long = conf.getConf(COLUMNAR_VELOX_BLOOM_FILTER_MAX_NUM_BITS) - def veloxCoalesceBatchesBeforeShuffle: Boolean = - conf.getConf(COLUMNAR_VELOX_COALESCE_BATCHES_BEFORE_SHUFFLE) + case class ResizeRange(min: Int, max: Int) { + assert(max >= min) + assert(min > 0, "Min batch size should be larger than 0") + assert(max > 0, "Max batch size should be larger than 0") + } + + private object ResizeRange { + def parse(pattern: String): ResizeRange = { + assert(pattern.count(_ == '~') == 1, s"Invalid range pattern for batch resizing: $pattern") + val splits = pattern.split('~') + assert(splits.length == 2) + ResizeRange(splits(0).toInt, splits(1).toInt) + } + } - def veloxMinBatchSizeForShuffle: Int = { - val defaultSize: Int = (0.8 * conf.getConf(COLUMNAR_MAX_BATCH_SIZE)).toInt.max(1) + def veloxResizeBatchesShuffleInput: Boolean = + conf.getConf(COLUMNAR_VELOX_RESIZE_BATCHES_SHUFFLE_INPUT) + + def veloxResizeBatchesShuffleInputRange: ResizeRange = { + val standardSize = conf.getConf(COLUMNAR_MAX_BATCH_SIZE) + val defaultRange: ResizeRange = + ResizeRange((0.25 * standardSize).toInt.max(1), 4 * standardSize) conf - .getConf(COLUMNAR_VELOX_MIN_BATCH_SIZE_FOR_SHUFFLE) - .getOrElse(defaultSize) + .getConf(COLUMNAR_VELOX_RESIZE_BATCHES_SHUFFLE_INPUT_RANGE) + .map(ResizeRange.parse) + .getOrElse(defaultRange) } def chColumnarShufflePreferSpill: Boolean = conf.getConf(COLUMNAR_CH_SHUFFLE_PREFER_SPILL_ENABLED) @@ -1452,21 +1470,25 @@ object GlutenConfig { .checkValue(_ > 0, "must be a positive number") .createWithDefault(10000) - val COLUMNAR_VELOX_COALESCE_BATCHES_BEFORE_SHUFFLE = - buildConf("spark.gluten.sql.columnar.backend.velox.coalesceBatchesBeforeShuffle") + val COLUMNAR_VELOX_RESIZE_BATCHES_SHUFFLE_INPUT = + buildConf("spark.gluten.sql.columnar.backend.velox.resizeBatches.shuffleInput") .internal() .doc(s"If true, combine small columnar batches together before sending to shuffle. " + s"The default minimum output batch size is equal to 0.8 * $GLUTEN_MAX_BATCH_SIZE_KEY") .booleanConf .createWithDefault(true) - val COLUMNAR_VELOX_MIN_BATCH_SIZE_FOR_SHUFFLE = - buildConf("spark.gluten.sql.columnar.backend.velox.minBatchSizeForShuffle") + val COLUMNAR_VELOX_RESIZE_BATCHES_SHUFFLE_INPUT_RANGE = + buildConf("spark.gluten.sql.columnar.backend.velox.resizeBatches.shuffleInput.range") .internal() - .doc(s"The minimum batch size for shuffle. If the batch size is smaller than this value, " + - s"it will be combined with other batches before sending to shuffle. Only functions when " + - s"${COLUMNAR_VELOX_COALESCE_BATCHES_BEFORE_SHUFFLE.key} is set to true.") - .intConf + .doc( + s"The minimum and maximum batch sizes for shuffle. If the batch size is " + + s"smaller / bigger than minimum / maximum value, it will be combined with other " + + s"batches / split before sending to shuffle. Only functions when " + + s"${COLUMNAR_VELOX_RESIZE_BATCHES_SHUFFLE_INPUT.key} is set to true. " + + s"A valid value for the option is min~max. " + + s"E.g., s.g.s.c.b.v.resizeBatches.shuffleInput.range=100~10000") + .stringConf .createOptional val COLUMNAR_CH_SHUFFLE_PREFER_SPILL_ENABLED =