org.apache.gluten
@@ -368,6 +432,7 @@
src/main/delta-${delta.binary.version}/org/apache/spark/sql/delta/commands/*.scala
src/main/delta-${delta.binary.version}/org/apache/spark/sql/delta/commands/merge/*.scala
+ src/main/delta-${delta.binary.version}/org/apache/spark/sql/delta/stats/*.scala
src/main/delta-${delta.binary.version}/org/apache/spark/sql/delta/DeltaLog.scala
src/main/delta-${delta.binary.version}/org/apache/spark/sql/delta/Snapshot.scala
@@ -425,6 +490,18 @@
+
+ add-test-sources
+ generate-test-sources
+
+ add-test-source
+
+
+
+
+
+
+
diff --git a/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala b/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala
index 1ea6fe249e81..06b6fa8c4c20 100644
--- a/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala
+++ b/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala
@@ -36,7 +36,6 @@ import org.apache.commons.lang3.exception.ExceptionUtils
import scala.collection.mutable.ListBuffer
-object ClickhouseOptimisticTransaction {}
class ClickhouseOptimisticTransaction(
override val deltaLog: DeltaLog,
override val snapshot: Snapshot)(implicit override val clock: Clock)
diff --git a/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/rules/CHOptimizeMetadataOnlyDeltaQuery.scala b/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/rules/CHOptimizeMetadataOnlyDeltaQuery.scala
new file mode 100644
index 000000000000..a360fa8d7291
--- /dev/null
+++ b/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/rules/CHOptimizeMetadataOnlyDeltaQuery.scala
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+package org.apache.spark.sql.delta.rules
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.delta.metering.DeltaLogging
+
+class CHOptimizeMetadataOnlyDeltaQuery(protected val spark: SparkSession)
+ extends Rule[LogicalPlan]
+ with DeltaLogging {
+
+ // For Delta 2.0, it can not support to optimize query with the metadata
+ override def apply(plan: LogicalPlan): LogicalPlan = plan
+}
diff --git a/backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala b/backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala
index 2a97ef951506..11acfb026054 100644
--- a/backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala
+++ b/backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala
@@ -36,7 +36,6 @@ import org.apache.commons.lang3.exception.ExceptionUtils
import scala.collection.mutable.ListBuffer
-object ClickhouseOptimisticTransaction {}
class ClickhouseOptimisticTransaction(
override val deltaLog: DeltaLog,
override val snapshot: Snapshot)(implicit override val clock: Clock)
diff --git a/backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/rules/CHOptimizeMetadataOnlyDeltaQuery.scala b/backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/rules/CHOptimizeMetadataOnlyDeltaQuery.scala
new file mode 100644
index 000000000000..dbb5c4050a2c
--- /dev/null
+++ b/backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/rules/CHOptimizeMetadataOnlyDeltaQuery.scala
@@ -0,0 +1,77 @@
+/*
+ * 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.
+ */
+package org.apache.spark.sql.delta.rules
+
+import org.apache.gluten.backendsapi.clickhouse.CHBackendSettings
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, V2WriteCommand}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.delta.{OptimisticTransaction, Snapshot, SubqueryTransformerHelper}
+import org.apache.spark.sql.delta.files.TahoeLogFileIndex
+import org.apache.spark.sql.delta.metering.DeltaLogging
+import org.apache.spark.sql.delta.perf.OptimizeMetadataOnlyDeltaQuery
+import org.apache.spark.sql.delta.sources.DeltaSQLConf
+import org.apache.spark.sql.delta.stats.DeltaScanGenerator
+
+import org.apache.hadoop.fs.Path
+
+class CHOptimizeMetadataOnlyDeltaQuery(protected val spark: SparkSession)
+ extends Rule[LogicalPlan]
+ with DeltaLogging
+ with SubqueryTransformerHelper
+ with OptimizeMetadataOnlyDeltaQuery {
+
+ private val scannedSnapshots =
+ new java.util.concurrent.ConcurrentHashMap[(String, Path), Snapshot]
+
+ protected def getDeltaScanGenerator(index: TahoeLogFileIndex): DeltaScanGenerator = {
+ // The first case means that we've fixed the table snapshot for time travel
+ if (index.isTimeTravelQuery) return index.getSnapshot
+ OptimisticTransaction
+ .getActive()
+ .map(_.getDeltaScanGenerator(index))
+ .getOrElse {
+ // Will be called only when the log is accessed the first time
+ scannedSnapshots.computeIfAbsent(index.deltaLog.compositeId, _ => index.getSnapshot)
+ }
+ }
+
+ override def apply(plan: LogicalPlan): LogicalPlan = {
+ // Should not be applied to subqueries to avoid duplicate delta jobs.
+ val isSubquery = isSubqueryRoot(plan)
+ // Should not be applied to DataSourceV2 write plans, because they'll be planned later
+ // through a V1 fallback and only that later planning takes place within the transaction.
+ val isDataSourceV2 = plan.isInstanceOf[V2WriteCommand]
+ if (isSubquery || isDataSourceV2) {
+ return plan
+ }
+ // when 'stats.skipping' is off, it still use the metadata to optimize query for count/min/max
+ if (
+ spark.sessionState.conf
+ .getConfString(
+ CHBackendSettings.GLUTEN_CLICKHOUSE_DELTA_METADATA_OPTIMIZE,
+ CHBackendSettings.GLUTEN_CLICKHOUSE_DELTA_METADATA_OPTIMIZE_DEFAULT_VALUE)
+ .toBoolean &&
+ !spark.sessionState.conf.getConf(DeltaSQLConf.DELTA_STATS_SKIPPING, true)
+ ) {
+ optimizeQueryWithMetadata(plan)
+ } else {
+ plan
+ }
+ }
+}
diff --git a/backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/stats/PrepareDeltaScan.scala b/backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/stats/PrepareDeltaScan.scala
new file mode 100644
index 000000000000..21e31d35411e
--- /dev/null
+++ b/backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/stats/PrepareDeltaScan.scala
@@ -0,0 +1,406 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.delta.stats
+
+import java.util.Objects
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.delta._
+import org.apache.spark.sql.delta.actions.AddFile
+import org.apache.spark.sql.delta.files.{TahoeFileIndexWithSnapshot, TahoeLogFileIndex}
+import org.apache.spark.sql.delta.metering.DeltaLogging
+import org.apache.spark.sql.delta.perf.OptimizeMetadataOnlyDeltaQuery
+import org.apache.spark.sql.delta.sources.DeltaSQLConf
+import org.apache.hadoop.fs.Path
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.planning.PhysicalOperation
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.PROJECT
+import org.apache.spark.sql.execution.datasources.LogicalRelation
+
+/**
+ * Gluten overwrite Delta:
+ *
+ * This file is copied from Delta 2.3.0, it is modified to overcome the following issues:
+ * 1. Returns the plan directly even if stats.skipping is turned off
+ */
+
+/**
+ * Before query planning, we prepare any scans over delta tables by pushing
+ * any projections or filters in allowing us to gather more accurate statistics
+ * for CBO and metering.
+ *
+ * Note the following
+ * - This rule also ensures that all reads from the same delta log use the same snapshot of log
+ * thus providing snapshot isolation.
+ * - If this rule is invoked within an active [[OptimisticTransaction]], then the scans are
+ * generated using the transaction.
+ */
+trait PrepareDeltaScanBase extends Rule[LogicalPlan]
+ with PredicateHelper
+ with DeltaLogging
+ with OptimizeMetadataOnlyDeltaQuery
+ with PreprocessTableWithDVs { self: PrepareDeltaScan =>
+
+ /**
+ * Tracks the first-access snapshots of other logs planned by this rule. The snapshots are
+ * the keyed by the log's unique id. Note that the lifetime of this rule is a single
+ * query, therefore, the map tracks the snapshots only within a query.
+ */
+ private val scannedSnapshots =
+ new java.util.concurrent.ConcurrentHashMap[(String, Path), Snapshot]
+
+ /**
+ * Gets the [[DeltaScanGenerator]] for the given log, which will be used to generate
+ * [[DeltaScan]]s. Every time this method is called on a log within the lifetime of this
+ * rule (i.e., the lifetime of the query for which this rule was instantiated), the returned
+ * generator will read a snapshot that is pinned on the first access for that log.
+ *
+ * Internally, it will use the snapshot of the file index, the snapshot of the active transaction
+ * (if any), or the latest snapshot of the given log.
+ */
+ protected def getDeltaScanGenerator(index: TahoeLogFileIndex): DeltaScanGenerator = {
+ // The first case means that we've fixed the table snapshot for time travel
+ if (index.isTimeTravelQuery) return index.getSnapshot
+ val scanGenerator = OptimisticTransaction.getActive()
+ .map(_.getDeltaScanGenerator(index))
+ .getOrElse {
+ // Will be called only when the log is accessed the first time
+ scannedSnapshots.computeIfAbsent(index.deltaLog.compositeId, _ => index.getSnapshot)
+ }
+ import PrepareDeltaScanBase._
+ if (onGetDeltaScanGeneratorCallback != null) onGetDeltaScanGeneratorCallback(scanGenerator)
+ scanGenerator
+ }
+
+ /**
+ * Helper method to generate a [[PreparedDeltaFileIndex]]
+ */
+ protected def getPreparedIndex(
+ preparedScan: DeltaScan,
+ fileIndex: TahoeLogFileIndex): PreparedDeltaFileIndex = {
+ assert(fileIndex.partitionFilters.isEmpty,
+ "Partition filters should have been extracted by DeltaAnalysis.")
+ PreparedDeltaFileIndex(
+ spark,
+ fileIndex.deltaLog,
+ fileIndex.path,
+ preparedScan,
+ fileIndex.versionToUse)
+ }
+
+ /**
+ * Scan files using the given `filters` and return `DeltaScan`.
+ *
+ * Note: when `limitOpt` is non empty, `filters` must contain only partition filters. Otherwise,
+ * it can contain arbitrary filters. See `DeltaTableScan` for more details.
+ */
+ protected def filesForScan(
+ scanGenerator: DeltaScanGenerator,
+ limitOpt: Option[Int],
+ filters: Seq[Expression],
+ delta: LogicalRelation): DeltaScan = {
+ withStatusCode("DELTA", "Filtering files for query") {
+ if (limitOpt.nonEmpty) {
+ // If we trigger limit push down, the filters must be partition filters. Since
+ // there are no data filters, we don't need to apply Generated Columns
+ // optimization. See `DeltaTableScan` for more details.
+ return scanGenerator.filesForScan(limitOpt.get, filters)
+ }
+ val filtersForScan =
+ if (!GeneratedColumn.partitionFilterOptimizationEnabled(spark)) {
+ filters
+ } else {
+ val generatedPartitionFilters = GeneratedColumn.generatePartitionFilters(
+ spark, scanGenerator.snapshotToScan, filters, delta)
+ filters ++ generatedPartitionFilters
+ }
+ scanGenerator.filesForScan(filtersForScan)
+ }
+ }
+
+ /**
+ * Prepares delta scans sequentially.
+ */
+ protected def prepareDeltaScan(plan: LogicalPlan): LogicalPlan = {
+ // A map from the canonicalized form of a DeltaTableScan operator to its corresponding delta
+ // scan. This map is used to avoid fetching duplicate delta indexes for structurally-equal
+ // delta scans.
+ val deltaScans = new mutable.HashMap[LogicalPlan, DeltaScan]()
+
+ transformWithSubqueries(plan) {
+ case scan @ DeltaTableScan(planWithRemovedProjections, filters, fileIndex,
+ limit, delta) =>
+ val scanGenerator = getDeltaScanGenerator(fileIndex)
+ val preparedScan = deltaScans.getOrElseUpdate(planWithRemovedProjections.canonicalized,
+ filesForScan(scanGenerator, limit, filters, delta))
+ val preparedIndex = getPreparedIndex(preparedScan, fileIndex)
+ optimizeGeneratedColumns(scan, preparedIndex, filters, limit, delta)
+ }
+ }
+
+ protected def optimizeGeneratedColumns(
+ scan: LogicalPlan,
+ preparedIndex: PreparedDeltaFileIndex,
+ filters: Seq[Expression],
+ limit: Option[Int],
+ delta: LogicalRelation): LogicalPlan = {
+ if (limit.nonEmpty) {
+ // If we trigger limit push down, the filters must be partition filters. Since
+ // there are no data filters, we don't need to apply Generated Columns
+ // optimization. See `DeltaTableScan` for more details.
+ return DeltaTableUtils.replaceFileIndex(scan, preparedIndex)
+ }
+ if (!GeneratedColumn.partitionFilterOptimizationEnabled(spark)) {
+ DeltaTableUtils.replaceFileIndex(scan, preparedIndex)
+ } else {
+ val generatedPartitionFilters =
+ GeneratedColumn.generatePartitionFilters(spark, preparedIndex, filters, delta)
+ val scanWithFilters =
+ if (generatedPartitionFilters.nonEmpty) {
+ scan transformUp {
+ case delta @ DeltaTable(_: TahoeLogFileIndex) =>
+ Filter(generatedPartitionFilters.reduceLeft(And), delta)
+ }
+ } else {
+ scan
+ }
+ DeltaTableUtils.replaceFileIndex(scanWithFilters, preparedIndex)
+ }
+ }
+
+ override def apply(_plan: LogicalPlan): LogicalPlan = {
+ var plan = _plan
+
+ // --- modified start
+ // Should not be applied to subqueries to avoid duplicate delta jobs.
+ val isSubquery = isSubqueryRoot(plan)
+ // Should not be applied to DataSourceV2 write plans, because they'll be planned later
+ // through a V1 fallback and only that later planning takes place within the transaction.
+ val isDataSourceV2 = plan.isInstanceOf[V2WriteCommand]
+ if (isSubquery || isDataSourceV2) {
+ return plan
+ }
+
+ val shouldPrepareDeltaScan = (
+ spark.sessionState.conf.getConf(DeltaSQLConf.DELTA_STATS_SKIPPING)
+ )
+ val updatedPlan = if (shouldPrepareDeltaScan) {
+ if (spark.sessionState.conf.getConf(DeltaSQLConf.DELTA_OPTIMIZE_METADATA_QUERY_ENABLED)) {
+ plan = optimizeQueryWithMetadata(plan)
+ }
+ prepareDeltaScan(plan)
+ } else {
+ // If this query is running inside an active transaction and is touching the same table
+ // as the transaction, then mark that the entire table as tainted to be safe.
+ OptimisticTransaction.getActive.foreach { txn =>
+ val logsInPlan = plan.collect { case DeltaTable(fileIndex) => fileIndex.deltaLog }
+ if (logsInPlan.exists(_.isSameLogAs(txn.deltaLog))) {
+ txn.readWholeTable()
+ }
+ }
+
+ // Just return the plan if statistics based skipping is off.
+ // It will fall back to just partition pruning at planning time.
+ plan
+ }
+ // --- modified end
+ preprocessTablesWithDVs(updatedPlan)
+ }
+
+ /**
+ * This is an extractor object. See https://docs.scala-lang.org/tour/extractor-objects.html.
+ */
+ object DeltaTableScan {
+
+ /**
+ * The components of DeltaTableScanType are:
+ * - the plan with removed projections. We remove projections as a plan differentiator
+ * because it does not affect file listing results.
+ * - filter expressions collected by `PhysicalOperation`
+ * - the `TahoeLogFileIndex` of the matched DeltaTable`
+ * - integer value of limit expression, if any
+ * - matched `DeltaTable`
+ */
+ private type DeltaTableScanType =
+ (LogicalPlan, Seq[Expression], TahoeLogFileIndex, Option[Int], LogicalRelation)
+
+ /**
+ * This is an extractor method (basically, the opposite of a constructor) which takes in an
+ * object `plan` and tries to give back the arguments as a [[DeltaTableScanType]].
+ */
+ def unapply(plan: LogicalPlan): Option[DeltaTableScanType] = {
+ val limitPushdownEnabled = spark.conf.get(DeltaSQLConf.DELTA_LIMIT_PUSHDOWN_ENABLED)
+
+ // Remove projections as a plan differentiator because it does not affect file listing
+ // results. Plans with the same filters but different projections therefore will not have
+ // duplicate delta indexes.
+ def canonicalizePlanForDeltaFileListing(plan: LogicalPlan): LogicalPlan = {
+ val planWithRemovedProjections = plan.transformWithPruning(_.containsPattern(PROJECT)) {
+ case p: Project if p.projectList.forall(_.isInstanceOf[AttributeReference]) => p.child
+ }
+ planWithRemovedProjections
+ }
+
+ plan match {
+ case LocalLimit(IntegerLiteral(limit),
+ PhysicalOperation(_, filters, delta @ DeltaTable(fileIndex: TahoeLogFileIndex)))
+ if limitPushdownEnabled && containsPartitionFiltersOnly(filters, fileIndex) =>
+ Some((canonicalizePlanForDeltaFileListing(plan), filters, fileIndex, Some(limit), delta))
+ case PhysicalOperation(
+ _,
+ filters,
+ delta @ DeltaTable(fileIndex: TahoeLogFileIndex)) =>
+ val allFilters = fileIndex.partitionFilters ++ filters
+ Some((canonicalizePlanForDeltaFileListing(plan), allFilters, fileIndex, None, delta))
+
+ case _ => None
+ }
+ }
+
+ private def containsPartitionFiltersOnly(
+ filters: Seq[Expression],
+ fileIndex: TahoeLogFileIndex): Boolean = {
+ val partitionColumns = fileIndex.snapshotAtAnalysis.metadata.partitionColumns
+ import DeltaTableUtils._
+ filters.forall(expr => !containsSubquery(expr) &&
+ isPredicatePartitionColumnsOnly(expr, partitionColumns, spark))
+ }
+ }
+}
+
+class PrepareDeltaScan(protected val spark: SparkSession)
+ extends PrepareDeltaScanBase
+
+object PrepareDeltaScanBase {
+
+ /**
+ * Optional callback function that is called after `getDeltaScanGenerator` is called
+ * by the PrepareDeltaScan rule. This is primarily used for testing purposes.
+ */
+ @volatile private var onGetDeltaScanGeneratorCallback: DeltaScanGenerator => Unit = _
+
+ /**
+ * Run a thunk of code with the given callback function injected into the PrepareDeltaScan rule.
+ * The callback function is called after `getDeltaScanGenerator` is called
+ * by the PrepareDeltaScan rule. This is primarily used for testing purposes.
+ */
+ private[delta] def withCallbackOnGetDeltaScanGenerator[T](
+ callback: DeltaScanGenerator => Unit)(thunk: => T): T = {
+ try {
+ onGetDeltaScanGeneratorCallback = callback
+ thunk
+ } finally {
+ onGetDeltaScanGeneratorCallback = null
+ }
+ }
+}
+
+/**
+ * A [[TahoeFileIndex]] that uses a prepared scan to return the list of relevant files.
+ * This is injected into a query right before query planning by [[PrepareDeltaScan]] so that
+ * CBO and metering can accurately understand how much data will be read.
+ *
+ * @param versionScanned The version of the table that is being scanned, if a specific version
+ * has specifically been requested, e.g. by time travel.
+ */
+case class PreparedDeltaFileIndex(
+ override val spark: SparkSession,
+ override val deltaLog: DeltaLog,
+ override val path: Path,
+ preparedScan: DeltaScan,
+ versionScanned: Option[Long])
+ extends TahoeFileIndexWithSnapshot(spark, deltaLog, path, preparedScan.scannedSnapshot)
+ with DeltaLogging {
+
+ /**
+ * Returns all matching/valid files by the given `partitionFilters` and `dataFilters`
+ */
+ override def matchingFiles(
+ partitionFilters: Seq[Expression],
+ dataFilters: Seq[Expression]): Seq[AddFile] = {
+ val currentFilters = ExpressionSet(partitionFilters ++ dataFilters)
+ val (addFiles, eventData) = if (currentFilters == preparedScan.allFilters ||
+ currentFilters == preparedScan.filtersUsedForSkipping) {
+ // [[DeltaScan]] was created using `allFilters` out of which only `filtersUsedForSkipping`
+ // filters were used for skipping while creating the DeltaScan.
+ // If currentFilters is same as allFilters, then no need to recalculate files and we can use
+ // previous results.
+ // If currentFilters is same as filtersUsedForSkipping, then also we don't need to recalculate
+ // files as [[DeltaScan.files]] were calculates using filtersUsedForSkipping only. So if we
+ // recalculate, we will get same result. So we should use previous result in this case also.
+ val eventData = Map(
+ "reused" -> true,
+ "currentFiltersSameAsPreparedAllFilters" -> (currentFilters == preparedScan.allFilters),
+ "currentFiltersSameAsPreparedFiltersUsedForSkipping" ->
+ (currentFilters == preparedScan.filtersUsedForSkipping)
+ )
+ (preparedScan.files.distinct, eventData)
+ } else {
+ logInfo(
+ s"""
+ |Prepared scan does not match actual filters. Reselecting files to query.
+ |Prepared: ${preparedScan.allFilters}
+ |Actual: ${currentFilters}
+ """.stripMargin)
+ val eventData = Map(
+ "reused" -> false,
+ "preparedAllFilters" -> preparedScan.allFilters.mkString(","),
+ "preparedFiltersUsedForSkipping" -> preparedScan.filtersUsedForSkipping.mkString(","),
+ "currentFilters" -> currentFilters.mkString(",")
+ )
+ val files = preparedScan.scannedSnapshot.filesForScan(partitionFilters ++ dataFilters).files
+ (files, eventData)
+ }
+ recordDeltaEvent(deltaLog,
+ opType = "delta.preparedDeltaFileIndex.reuseSkippingResult",
+ data = eventData)
+ addFiles
+ }
+
+ /**
+ * Returns the list of files that will be read when scanning this relation. This call may be
+ * very expensive for large tables.
+ */
+ override def inputFiles: Array[String] =
+ preparedScan.files.map(f => absolutePath(f.path).toString).toArray
+
+ /** Refresh any cached file listings */
+ override def refresh(): Unit = { }
+
+ /** Sum of table file sizes, in bytes */
+ override def sizeInBytes: Long =
+ preparedScan.scanned.bytesCompressed
+ .getOrElse(spark.sessionState.conf.defaultSizeInBytes)
+
+ override def equals(other: Any): Boolean = other match {
+ case p: PreparedDeltaFileIndex =>
+ p.deltaLog == deltaLog && p.path == path && p.preparedScan == preparedScan &&
+ p.partitionSchema == partitionSchema && p.versionScanned == versionScanned
+ case _ => false
+ }
+
+ override def hashCode(): Int = {
+ Objects.hash(deltaLog, path, preparedScan, partitionSchema, versionScanned)
+ }
+
+}
diff --git a/backends-clickhouse/src/main/delta-32/io/delta/tables/ClickhouseTable.scala b/backends-clickhouse/src/main/delta-32/io/delta/tables/ClickhouseTable.scala
index e747c87c6a67..ba4c21df3a34 100644
--- a/backends-clickhouse/src/main/delta-32/io/delta/tables/ClickhouseTable.scala
+++ b/backends-clickhouse/src/main/delta-32/io/delta/tables/ClickhouseTable.scala
@@ -27,12 +27,7 @@ import scala.collection.JavaConverters._
class ClickhouseTable(
@transient private val _df: Dataset[Row],
@transient private val table: ClickHouseTableV2)
- extends DeltaTable(_df, table) {
-
- override def optimize(): DeltaOptimizeBuilder = {
- DeltaOptimizeBuilder(table)
- }
-}
+ extends DeltaTable(_df, table) {}
object ClickhouseTable {
diff --git a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala
index 2f5824b58092..05f7fdbfa423 100644
--- a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala
+++ b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala
@@ -16,6 +16,7 @@
*/
package org.apache.spark.sql.delta
+import org.apache.gluten.GlutenConfig
import org.apache.gluten.backendsapi.clickhouse.CHConf
import org.apache.spark.SparkException
@@ -24,12 +25,11 @@ import org.apache.spark.sql.catalyst.catalog.CatalogTable
import org.apache.spark.sql.delta.actions._
import org.apache.spark.sql.delta.catalog.ClickHouseTableV2
import org.apache.spark.sql.delta.constraints.{Constraint, Constraints}
-import org.apache.spark.sql.delta.files.{DelayedCommitProtocol, DeltaFileFormatWriter, MergeTreeDelayedCommitProtocol, TransactionalWrite}
+import org.apache.spark.sql.delta.files._
import org.apache.spark.sql.delta.hooks.AutoCompact
import org.apache.spark.sql.delta.schema.{InnerInvariantViolationException, InvariantViolationException}
import org.apache.spark.sql.delta.sources.DeltaSQLConf
-import org.apache.spark.sql.delta.stats.DeltaJobStatisticsTracker
-import org.apache.spark.sql.execution.{CHDelayedCommitProtocol, QueryExecution, SparkPlan, SQLExecution}
+import org.apache.spark.sql.execution.{QueryExecution, SparkPlan, SQLExecution}
import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec
import org.apache.spark.sql.execution.datasources.{BasicWriteJobStatsTracker, FileFormatWriter, GlutenWriterColumnarRules, WriteFiles, WriteJobStatsTracker}
import org.apache.spark.sql.execution.datasources.v1.MergeTreeWriterInjects
@@ -43,14 +43,15 @@ import org.apache.hadoop.fs.Path
import scala.collection.mutable.ListBuffer
-object ClickhouseOptimisticTransaction {}
-
class ClickhouseOptimisticTransaction(
override val deltaLog: DeltaLog,
override val catalogTable: Option[CatalogTable],
override val snapshot: Snapshot)
extends OptimisticTransaction(deltaLog, catalogTable, snapshot) {
+ private lazy val writingMergeTree =
+ ClickHouseConfig.isMergeTreeFormatEngine(metadata.configuration)
+
def this(
deltaLog: DeltaLog,
catalogTable: Option[CatalogTable],
@@ -65,120 +66,137 @@ class ClickhouseOptimisticTransaction(
override def writeFiles(
inputData: Dataset[_],
writeOptions: Option[DeltaOptions],
+ isOptimize: Boolean,
additionalConstraints: Seq[Constraint]): Seq[FileAction] = {
- if (ClickHouseConfig.isMergeTreeFormatEngine(metadata.configuration)) {
- hasWritten = true
-
- val spark = inputData.sparkSession
- val (data, partitionSchema) = performCDCPartition(inputData)
- val outputPath = deltaLog.dataPath
-
- val (queryExecution, output, generatedColumnConstraints, _) =
- normalizeData(deltaLog, writeOptions, data)
-
- val tableV2 = ClickHouseTableV2.getTable(deltaLog)
- val committer =
- new MergeTreeDelayedCommitProtocol(
- outputPath.toString,
- None,
- None,
- tableV2.dataBaseName,
- tableV2.tableName)
-
- // val (optionalStatsTracker, _) =
- // getOptionalStatsTrackerAndStatsCollection(output, outputPath, partitionSchema, data)
- val (optionalStatsTracker, _) = (None, None)
-
- val constraints =
- Constraints.getAll(metadata, spark) ++ generatedColumnConstraints ++ additionalConstraints
-
- SQLExecution.withNewExecutionId(queryExecution, Option("deltaTransactionalWrite")) {
- val queryPlan = queryExecution.executedPlan
- val (newQueryPlan, newOutput) =
- MergeTreeWriterInjects.insertFakeRowAdaptor(queryPlan, output)
- val outputSpec = FileFormatWriter.OutputSpec(outputPath.toString, Map.empty, newOutput)
- val partitioningColumns = getPartitioningColumns(partitionSchema, newOutput)
-
- val statsTrackers: ListBuffer[WriteJobStatsTracker] = ListBuffer()
-
- if (spark.conf.get(DeltaSQLConf.DELTA_HISTORY_METRICS_ENABLED)) {
- val basicWriteJobStatsTracker = new BasicWriteJobStatsTracker(
- new SerializableConfiguration(deltaLog.newDeltaHadoopConf()),
- BasicWriteJobStatsTracker.metrics)
- // registerSQLMetrics(spark, basicWriteJobStatsTracker.driverSideMetrics)
- statsTrackers.append(basicWriteJobStatsTracker)
- }
-
- // Iceberg spec requires partition columns in data files
- val writePartitionColumns = IcebergCompat.isAnyEnabled(metadata)
- // Retain only a minimal selection of Spark writer options to avoid any potential
- // compatibility issues
- var options = (writeOptions match {
- case None => Map.empty[String, String]
- case Some(writeOptions) =>
- writeOptions.options.filterKeys {
- key =>
- key.equalsIgnoreCase(DeltaOptions.MAX_RECORDS_PER_FILE) ||
- key.equalsIgnoreCase(DeltaOptions.COMPRESSION)
- }.toMap
- }) + (DeltaOptions.WRITE_PARTITION_COLUMNS -> writePartitionColumns.toString)
-
- spark.conf.getAll.foreach(
- entry => {
- if (
- CHConf.startWithSettingsPrefix(entry._1)
- || entry._1.equalsIgnoreCase(DeltaSQLConf.DELTA_OPTIMIZE_MIN_FILE_SIZE.key)
- ) {
- options += (entry._1 -> entry._2)
- }
- })
-
- try {
- val format = tableV2.getFileFormat(protocol, metadata)
- GlutenWriterColumnarRules.injectSparkLocalProperty(spark, Some(format.shortName()))
- MergeTreeFileFormatWriter.write(
- sparkSession = spark,
- plan = newQueryPlan,
- fileFormat = format,
- // formats.
- committer = committer,
- outputSpec = outputSpec,
- // scalastyle:off deltahadoopconfiguration
- hadoopConf = spark.sessionState
- .newHadoopConfWithOptions(metadata.configuration ++ deltaLog.options),
- // scalastyle:on deltahadoopconfiguration
- partitionColumns = partitioningColumns,
- bucketSpec =
- tableV2.normalizedBucketSpec(output.map(_.name), spark.sessionState.conf.resolver),
- statsTrackers = optionalStatsTracker.toSeq ++ statsTrackers,
- options = options,
- constraints = constraints
- )
- } catch {
- case s: SparkException =>
- // Pull an InvariantViolationException up to the top level if it was the root cause.
- val violationException = ExceptionUtils.getRootCause(s)
- if (violationException.isInstanceOf[InvariantViolationException]) {
- throw violationException
- } else {
- throw s
- }
- } finally {
- GlutenWriterColumnarRules.injectSparkLocalProperty(spark, None)
- }
+ val nativeWrite = GlutenConfig.getConf.enableNativeWriter.getOrElse(false)
+ if (writingMergeTree) {
+ if (isOptimize) {
+ throw new UnsupportedOperationException("Optimize is not supported for ClickHouse")
}
- committer.addedStatuses.toSeq ++ committer.changeFiles
+ // TODO: update FallbackByBackendSettings for mergetree always return true
+ val onePipeline = nativeWrite && CHConf.get.enableOnePipelineMergeTreeWrite
+ if (onePipeline)
+ pipelineWriteFiles(inputData, writeOptions, isOptimize = false, additionalConstraints)
+ else
+ writeMergeTree(inputData, writeOptions, additionalConstraints)
} else {
- // TODO: support native delta parquet write
- // 1. insert FakeRowAdaptor
- // 2. DeltaInvariantCheckerExec transform
- // 3. DeltaTaskStatisticsTracker collect null count / min values / max values
- // 4. set the parameters 'staticPartitionWriteOnly', 'isNativeApplicable',
- // 'nativeFormat' in the LocalProperty of the sparkcontext
- super.writeFiles(inputData, writeOptions, additionalConstraints)
+ if (isOptimize || !nativeWrite) {
+ super.writeFiles(inputData, writeOptions, isOptimize, additionalConstraints)
+ } else {
+ pipelineWriteFiles(inputData, writeOptions, isOptimize = false, additionalConstraints)
+ }
}
}
+ @deprecated("Use pipelineWriteFiles instead")
+ private def writeMergeTree(
+ inputData: Dataset[_],
+ writeOptions: Option[DeltaOptions],
+ additionalConstraints: Seq[Constraint]): Seq[FileAction] = {
+
+ hasWritten = true
+
+ val spark = inputData.sparkSession
+ val (data, partitionSchema) = performCDCPartition(inputData)
+ val outputPath = deltaLog.dataPath
+
+ val (queryExecution, output, generatedColumnConstraints, _) =
+ normalizeData(deltaLog, writeOptions, data)
+
+ val tableV2 = ClickHouseTableV2.getTable(deltaLog)
+ val committer =
+ new MergeTreeDelayedCommitProtocol(
+ outputPath.toString,
+ None,
+ None,
+ tableV2.dataBaseName,
+ tableV2.tableName)
+
+ // val (optionalStatsTracker, _) =
+ // getOptionalStatsTrackerAndStatsCollection(output, outputPath, partitionSchema, data)
+ val (optionalStatsTracker, _) = (None, None)
+
+ val constraints =
+ Constraints.getAll(metadata, spark) ++ generatedColumnConstraints ++ additionalConstraints
+
+ SQLExecution.withNewExecutionId(queryExecution, Option("deltaTransactionalWrite")) {
+ val queryPlan = queryExecution.executedPlan
+ val (newQueryPlan, newOutput) =
+ MergeTreeWriterInjects.insertFakeRowAdaptor(queryPlan, output)
+ val outputSpec = FileFormatWriter.OutputSpec(outputPath.toString, Map.empty, newOutput)
+ val partitioningColumns = getPartitioningColumns(partitionSchema, newOutput)
+
+ val statsTrackers: ListBuffer[WriteJobStatsTracker] = ListBuffer()
+
+ if (spark.conf.get(DeltaSQLConf.DELTA_HISTORY_METRICS_ENABLED)) {
+ val basicWriteJobStatsTracker = new BasicWriteJobStatsTracker(
+ new SerializableConfiguration(deltaLog.newDeltaHadoopConf()),
+ BasicWriteJobStatsTracker.metrics)
+ // registerSQLMetrics(spark, basicWriteJobStatsTracker.driverSideMetrics)
+ statsTrackers.append(basicWriteJobStatsTracker)
+ }
+
+ // Iceberg spec requires partition columns in data files
+ val writePartitionColumns = IcebergCompat.isAnyEnabled(metadata)
+ // Retain only a minimal selection of Spark writer options to avoid any potential
+ // compatibility issues
+ var options = (writeOptions match {
+ case None => Map.empty[String, String]
+ case Some(writeOptions) =>
+ writeOptions.options.filterKeys {
+ key =>
+ key.equalsIgnoreCase(DeltaOptions.MAX_RECORDS_PER_FILE) ||
+ key.equalsIgnoreCase(DeltaOptions.COMPRESSION)
+ }.toMap
+ }) + (DeltaOptions.WRITE_PARTITION_COLUMNS -> writePartitionColumns.toString)
+
+ spark.conf.getAll.foreach(
+ entry => {
+ if (
+ CHConf.startWithSettingsPrefix(entry._1)
+ || entry._1.equalsIgnoreCase(DeltaSQLConf.DELTA_OPTIMIZE_MIN_FILE_SIZE.key)
+ ) {
+ options += (entry._1 -> entry._2)
+ }
+ })
+
+ try {
+ val format = tableV2.getFileFormat(protocol, metadata)
+ GlutenWriterColumnarRules.injectSparkLocalProperty(spark, Some(format.shortName()))
+ MergeTreeFileFormatWriter.write(
+ sparkSession = spark,
+ plan = newQueryPlan,
+ fileFormat = format,
+ // formats.
+ committer = committer,
+ outputSpec = outputSpec,
+ // scalastyle:off deltahadoopconfiguration
+ hadoopConf = spark.sessionState
+ .newHadoopConfWithOptions(metadata.configuration ++ deltaLog.options),
+ // scalastyle:on deltahadoopconfiguration
+ partitionColumns = partitioningColumns,
+ bucketSpec =
+ tableV2.normalizedBucketSpec(output.map(_.name), spark.sessionState.conf.resolver),
+ statsTrackers = optionalStatsTracker.toSeq ++ statsTrackers,
+ options = options,
+ constraints = constraints
+ )
+ } catch {
+ case s: SparkException =>
+ // Pull an InvariantViolationException up to the top level if it was the root cause.
+ val violationException = ExceptionUtils.getRootCause(s)
+ if (violationException.isInstanceOf[InvariantViolationException]) {
+ throw violationException
+ } else {
+ throw s
+ }
+ } finally {
+ GlutenWriterColumnarRules.injectSparkLocalProperty(spark, None)
+ }
+ }
+ committer.addedStatuses.toSeq ++ committer.changeFiles
+ }
+
private def shouldOptimizeWrite(
writeOptions: Option[DeltaOptions],
sessionConf: SQLConf): Boolean = {
@@ -188,17 +206,23 @@ class ClickhouseOptimisticTransaction(
}
override protected def getCommitter(outputPath: Path): DelayedCommitProtocol =
- new CHDelayedCommitProtocol("delta", outputPath.toString, None, deltaDataSubdir)
+ new FileDelayedCommitProtocol("delta", outputPath.toString, None, deltaDataSubdir)
+
+ private def getCommitter2(outputPath: Path): DelayedCommitProtocol = {
+ val tableV2 = ClickHouseTableV2.getTable(deltaLog)
+ new MergeTreeDelayedCommitProtocol2(
+ outputPath.toString,
+ None,
+ deltaDataSubdir,
+ tableV2.dataBaseName,
+ tableV2.tableName)
+ }
- override def writeFiles(
+ private def pipelineWriteFiles(
inputData: Dataset[_],
writeOptions: Option[DeltaOptions],
isOptimize: Boolean,
additionalConstraints: Seq[Constraint]): Seq[FileAction] = {
-
- if (isOptimize)
- throw new UnsupportedOperationException("Optimize is not supported for ClickHouse")
-
hasWritten = true
val spark = inputData.sparkSession
@@ -230,14 +254,19 @@ class ClickhouseOptimisticTransaction(
WriteFiles(logicalPlan, fileFormat, partitioningColumns, None, options, Map.empty)
val queryExecution = new QueryExecution(spark, write)
- val committer = getCommitter(outputPath)
+ val (committer, collectStats) = fileFormat.toString match {
+ case "MergeTree" => (getCommitter2(outputPath), false)
+ case _ => (getCommitter(outputPath), true)
+ }
// If Statistics Collection is enabled, then create a stats tracker that will be injected during
// the FileFormatWriter.write call below and will collect per-file stats using
// StatisticsCollection
- // val (optionalStatsTracker, _) =
- // getOptionalStatsTrackerAndStatsCollection(output, outputPath, partitionSchema, data)
- val optionalStatsTracker: Option[DeltaJobStatisticsTracker] = None
+ val (optionalStatsTracker, _) = if (collectStats) {
+ getOptionalStatsTrackerAndStatsCollection(output, outputPath, partitionSchema, data)
+ } else {
+ (None, None)
+ }
val constraints =
Constraints.getAll(metadata, spark) ++ generatedColumnConstraints ++ additionalConstraints
@@ -258,7 +287,7 @@ class ClickhouseOptimisticTransaction(
DeltaOptimizedWriterExec(checkInvariants, metadata.partitionColumns, deltaLog)
} else {
checkInvariants
- }*/
+ } */
val statsTrackers: ListBuffer[WriteJobStatsTracker] = ListBuffer()
if (spark.conf.get(DeltaSQLConf.DELTA_HISTORY_METRICS_ENABLED)) {
@@ -304,10 +333,11 @@ class ClickhouseOptimisticTransaction(
committer.addedStatuses
})
.filter {
- // In some cases, we can write out an empty `inputData`. Some examples of this (though, they
- // may be fixed in the future) are the MERGE command when you delete with empty source, or
- // empty target, or on disjoint tables. This is hard to catch before the write without
- // collecting the DF ahead of time. Instead, we can return only the AddFiles that
+ // In some cases, we can write out an empty `inputData`. Some examples of this (though,
+ // they may be fixed in the future) are the MERGE command when you delete with empty
+ // source, or empty target, or on disjoint tables. This is hard to catch before
+ // the write without collecting the DF ahead of time. Instead,
+ // we can return only the AddFiles that
// a) actually add rows, or
// b) don't have any stats so we don't know the number of rows at all
case a: AddFile => a.numLogicalRecords.forall(_ > 0)
diff --git a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/DeltaLog.scala b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/DeltaLog.scala
index bac5231309b8..f64de28f4214 100644
--- a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/DeltaLog.scala
+++ b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/DeltaLog.scala
@@ -60,7 +60,7 @@ import org.apache.spark.util._
/**
* Gluten overwrite Delta:
*
- * This file is copied from Delta 3.2.0, it is modified to overcome the following issues:
+ * This file is copied from Delta 3.2.1, it is modified to overcome the following issues:
* 1. return ClickhouseOptimisticTransaction
* 2. return DeltaMergeTreeFileFormat
* 3. create HadoopFsRelation with the bucket options
diff --git a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/Snapshot.scala b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/Snapshot.scala
index 8836f7c88d23..5bfda914db67 100644
--- a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/Snapshot.scala
+++ b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/Snapshot.scala
@@ -45,7 +45,7 @@ import org.apache.spark.util.Utils
/**
* Gluten overwrite Delta:
*
- * This file is copied from Delta 3.2.0. It is modified to overcome the following issues:
+ * This file is copied from Delta 3.2.1. It is modified to overcome the following issues:
* 1. filesForScan() will cache the DeltaScan by the FilterExprsAsKey
* 2. filesForScan() should return DeltaScan of AddMergeTreeParts instead of AddFile
*/
diff --git a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/catalog/ClickHouseTableV2.scala b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/catalog/ClickHouseTableV2.scala
index 5f6a2dc3d712..d887e7a21b34 100644
--- a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/catalog/ClickHouseTableV2.scala
+++ b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/catalog/ClickHouseTableV2.scala
@@ -25,7 +25,6 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression}
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.connector.catalog.V1Table
import org.apache.spark.sql.connector.read.InputPartition
-import org.apache.spark.sql.connector.write.{LogicalWriteInfo, WriteBuilder}
import org.apache.spark.sql.delta.{ClickhouseSnapshot, DeltaErrors, DeltaLog, DeltaTableUtils, DeltaTimeTravelSpec, Snapshot, UnresolvedPathBasedDeltaTable}
import org.apache.spark.sql.delta.actions.{Metadata, Protocol}
import org.apache.spark.sql.delta.sources.DeltaDataSource
@@ -89,13 +88,6 @@ class ClickHouseTableV2(
ret
}
- override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = {
- new WriteIntoDeltaBuilder(
- this,
- info.options,
- spark.sessionState.conf.useNullsForMissingDefaultColumnValues)
- }
-
def getFileFormat(protocol: Protocol, meta: Metadata): DeltaMergeTreeFileFormat = {
new DeltaMergeTreeFileFormat(
protocol,
diff --git a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/commands/DeleteCommand.scala b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/commands/DeleteCommand.scala
index dec1f4b9c3f5..0a25346fc6c3 100644
--- a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/commands/DeleteCommand.scala
+++ b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/commands/DeleteCommand.scala
@@ -45,7 +45,7 @@ import org.apache.spark.sql.types.LongType
/**
* Gluten overwrite Delta:
*
- * This file is copied from Delta 3.2.0. It is modified to overcome the following issues:
+ * This file is copied from Delta 3.2.1. It is modified to overcome the following issues:
* 1. In Clickhouse backend, we can't implement input_file_name() correctly, we can only implement
* it so that it return a a list of filenames (concated by ',').
*/
diff --git a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/commands/OptimizeTableCommand.scala b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/commands/OptimizeTableCommand.scala
index 5b2170220228..439111df1b1c 100644
--- a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/commands/OptimizeTableCommand.scala
+++ b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/commands/OptimizeTableCommand.scala
@@ -49,7 +49,7 @@ import org.apache.spark.util.{SystemClock, ThreadUtils}
/**
* Gluten overwrite Delta:
*
- * This file is copied from Delta 3.2.0. It is modified in:
+ * This file is copied from Delta 3.2.1. It is modified in:
* 1. getDeltaTable supports to get ClickHouseTableV2
* 2. runOptimizeBinJobClickhouse
* 3. groupFilesIntoBinsClickhouse
diff --git a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/commands/UpdateCommand.scala b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/commands/UpdateCommand.scala
index 9a7fb96775f0..4e75b8461970 100644
--- a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/commands/UpdateCommand.scala
+++ b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/commands/UpdateCommand.scala
@@ -45,7 +45,7 @@ import org.apache.spark.sql.types.LongType
/**
* Gluten overwrite Delta:
*
- * This file is copied from Delta 3.2.0. It is modified to overcome the following issues:
+ * This file is copied from Delta 3.2.1. It is modified to overcome the following issues:
* 1. In Clickhouse backend, we can't implement input_file_name() correctly, we can only implement
* it so that it return a a list of filenames (concated by ',').
*/
diff --git a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/commands/VacuumCommand.scala b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/commands/VacuumCommand.scala
index 5d05bdb86896..7a350ae4d594 100644
--- a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/commands/VacuumCommand.scala
+++ b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/commands/VacuumCommand.scala
@@ -44,7 +44,7 @@ import org.apache.spark.util.{Clock, SerializableConfiguration, SystemClock}
/**
* Gluten overwrite Delta:
*
- * This file is copied from Delta 3.2.0. It is modified to overcome the following issues:
+ * This file is copied from Delta 3.2.1. It is modified to overcome the following issues:
* 1. In Gluten, part is a directory, but VacuumCommand assumes part is a file. So we need some
* modifications to make it work.
* 2. Set the 'gluten.enabledForCurrentThread' to false, now gluten can not support vacuum cmd.
@@ -255,7 +255,8 @@ object VacuumCommand extends VacuumCommandImpl with Serializable {
val originalEnabledGluten =
spark.sparkContext.getLocalProperty(GlutenSessionExtensions.GLUTEN_ENABLE_FOR_THREAD_KEY)
// gluten can not support vacuum command
- spark.sparkContext.setLocalProperty(GlutenSessionExtensions.GLUTEN_ENABLE_FOR_THREAD_KEY, "false")
+ spark.sparkContext
+ .setLocalProperty(GlutenSessionExtensions.GLUTEN_ENABLE_FOR_THREAD_KEY, "false")
// --- modified end
val validFiles =
diff --git a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/commands/merge/ClassicMergeExecutor.scala b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/commands/merge/ClassicMergeExecutor.scala
index 42a89d427197..aa1f94c5c9f5 100644
--- a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/commands/merge/ClassicMergeExecutor.scala
+++ b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/commands/merge/ClassicMergeExecutor.scala
@@ -35,7 +35,7 @@ import org.apache.spark.sql.functions.{coalesce, col, count, input_file_name, li
/**
* Gluten overwrite Delta:
*
- * This file is copied from Delta 3.2.0. It is modified to overcome the following issues:
+ * This file is copied from Delta 3.2.1. It is modified to overcome the following issues:
* 1. In Clickhouse backend, we can't implement input_file_name() correctly, we can only implement
* it so that it return a a list of filenames (concated by ','). In findTouchedFiles func.
*/
diff --git a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/files/MergeTreeDelayedCommitProtocol.scala b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/files/MergeTreeDelayedCommitProtocol.scala
index f2c22234a692..e90df0f02634 100644
--- a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/files/MergeTreeDelayedCommitProtocol.scala
+++ b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/files/MergeTreeDelayedCommitProtocol.scala
@@ -16,6 +16,8 @@
*/
package org.apache.spark.sql.delta.files
+import org.apache.hadoop.mapreduce.TaskAttemptContext
+
class MergeTreeDelayedCommitProtocol(
val outputPath: String,
randomPrefixLength: Option[Int],
@@ -24,3 +26,49 @@ class MergeTreeDelayedCommitProtocol(
val tableName: String)
extends DelayedCommitProtocol("delta-mergetree", outputPath, randomPrefixLength, subdir)
with MergeTreeFileCommitProtocol {}
+
+/**
+ * A Wrapper of [[DelayedCommitProtocol]] for accessing protected methods and fields in a pipeline
+ * write for parquet.
+ */
+class FileDelayedCommitProtocol(
+ jobId: String,
+ val outputPath: String,
+ randomPrefixLength: Option[Int],
+ subdir: Option[String])
+ extends DelayedCommitProtocol(jobId, outputPath, randomPrefixLength, subdir) {
+
+ override def getFileName(
+ taskContext: TaskAttemptContext,
+ ext: String,
+ partitionValues: Map[String, String]): String = {
+ super.getFileName(taskContext, ext, partitionValues)
+ }
+
+ def updateAddedFiles(files: Seq[(Map[String, String], String)]): Unit = {
+ assert(addedFiles.isEmpty)
+ addedFiles ++= files
+ }
+
+ override def parsePartitions(dir: String): Map[String, String] =
+ super.parsePartitions(dir)
+}
+
+/**
+ * A Wrapper of [[DelayedCommitProtocol]] for accessing protected methods and fields in a pipeline
+ * write for mergetree.
+ */
+class MergeTreeDelayedCommitProtocol2(
+ val outputPath: String,
+ randomPrefixLength: Option[Int],
+ subdir: Option[String],
+ val database: String,
+ val tableName: String)
+ extends DelayedCommitProtocol("delta-mergetree", outputPath, randomPrefixLength, subdir) {
+
+ override def newTaskTempFile(
+ taskContext: TaskAttemptContext,
+ dir: Option[String],
+ ext: String): String = outputPath
+
+}
diff --git a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/rules/CHOptimizeMetadataOnlyDeltaQuery.scala b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/rules/CHOptimizeMetadataOnlyDeltaQuery.scala
new file mode 100644
index 000000000000..dbb5c4050a2c
--- /dev/null
+++ b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/rules/CHOptimizeMetadataOnlyDeltaQuery.scala
@@ -0,0 +1,77 @@
+/*
+ * 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.
+ */
+package org.apache.spark.sql.delta.rules
+
+import org.apache.gluten.backendsapi.clickhouse.CHBackendSettings
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, V2WriteCommand}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.delta.{OptimisticTransaction, Snapshot, SubqueryTransformerHelper}
+import org.apache.spark.sql.delta.files.TahoeLogFileIndex
+import org.apache.spark.sql.delta.metering.DeltaLogging
+import org.apache.spark.sql.delta.perf.OptimizeMetadataOnlyDeltaQuery
+import org.apache.spark.sql.delta.sources.DeltaSQLConf
+import org.apache.spark.sql.delta.stats.DeltaScanGenerator
+
+import org.apache.hadoop.fs.Path
+
+class CHOptimizeMetadataOnlyDeltaQuery(protected val spark: SparkSession)
+ extends Rule[LogicalPlan]
+ with DeltaLogging
+ with SubqueryTransformerHelper
+ with OptimizeMetadataOnlyDeltaQuery {
+
+ private val scannedSnapshots =
+ new java.util.concurrent.ConcurrentHashMap[(String, Path), Snapshot]
+
+ protected def getDeltaScanGenerator(index: TahoeLogFileIndex): DeltaScanGenerator = {
+ // The first case means that we've fixed the table snapshot for time travel
+ if (index.isTimeTravelQuery) return index.getSnapshot
+ OptimisticTransaction
+ .getActive()
+ .map(_.getDeltaScanGenerator(index))
+ .getOrElse {
+ // Will be called only when the log is accessed the first time
+ scannedSnapshots.computeIfAbsent(index.deltaLog.compositeId, _ => index.getSnapshot)
+ }
+ }
+
+ override def apply(plan: LogicalPlan): LogicalPlan = {
+ // Should not be applied to subqueries to avoid duplicate delta jobs.
+ val isSubquery = isSubqueryRoot(plan)
+ // Should not be applied to DataSourceV2 write plans, because they'll be planned later
+ // through a V1 fallback and only that later planning takes place within the transaction.
+ val isDataSourceV2 = plan.isInstanceOf[V2WriteCommand]
+ if (isSubquery || isDataSourceV2) {
+ return plan
+ }
+ // when 'stats.skipping' is off, it still use the metadata to optimize query for count/min/max
+ if (
+ spark.sessionState.conf
+ .getConfString(
+ CHBackendSettings.GLUTEN_CLICKHOUSE_DELTA_METADATA_OPTIMIZE,
+ CHBackendSettings.GLUTEN_CLICKHOUSE_DELTA_METADATA_OPTIMIZE_DEFAULT_VALUE)
+ .toBoolean &&
+ !spark.sessionState.conf.getConf(DeltaSQLConf.DELTA_STATS_SKIPPING, true)
+ ) {
+ optimizeQueryWithMetadata(plan)
+ } else {
+ plan
+ }
+ }
+}
diff --git a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/execution/CHDeltaColumnarWrite.scala b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/execution/CHDeltaColumnarWrite.scala
deleted file mode 100644
index 66f502038fcd..000000000000
--- a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/execution/CHDeltaColumnarWrite.scala
+++ /dev/null
@@ -1,144 +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.
- */
-package org.apache.spark.sql.execution
-
-import org.apache.gluten.backendsapi.BackendsApiManager
-import org.apache.gluten.exception.GlutenNotSupportException
-
-import org.apache.spark.internal.Logging
-import org.apache.spark.internal.io.FileCommitProtocol
-import org.apache.spark.sql.delta.files.DelayedCommitProtocol
-import org.apache.spark.sql.execution.datasources.{ExecutedWriteSummary, WriteJobDescription, WriteTaskResult}
-import org.apache.spark.sql.vectorized.ColumnarBatch
-import org.apache.spark.util.Utils
-
-import org.apache.hadoop.mapreduce.TaskAttemptContext
-
-import scala.collection.mutable
-import scala.collection.mutable.ArrayBuffer
-
-/** A Wrapper of [[DelayedCommitProtocol]] for accessing protected methods and fields. */
-class CHDelayedCommitProtocol(
- jobId: String,
- val outputPath: String,
- randomPrefixLength: Option[Int],
- subdir: Option[String])
- extends DelayedCommitProtocol(jobId, outputPath, randomPrefixLength, subdir) {
-
- override def getFileName(
- taskContext: TaskAttemptContext,
- ext: String,
- partitionValues: Map[String, String]): String = {
- super.getFileName(taskContext, ext, partitionValues)
- }
-
- def updateAddedFiles(files: Seq[(Map[String, String], String)]): Unit = {
- assert(addedFiles.isEmpty)
- addedFiles ++= files
- }
-
- override def parsePartitions(dir: String): Map[String, String] =
- super.parsePartitions(dir)
-}
-
-case class CHDelayedCommitProtocolWrite(
- override val jobTrackerID: String,
- override val description: WriteJobDescription,
- override val committer: CHDelayedCommitProtocol)
- extends CHColumnarWrite[CHDelayedCommitProtocol]
- with Logging {
-
- override def doSetupNativeTask(): Unit = {
- assert(description.path == committer.outputPath)
- val nameSpec = CreateFileNameSpec(taskAttemptContext, description)
- val writePath = description.path
- val writeFileName = committer.getFileName(taskAttemptContext, nameSpec.suffix, Map.empty)
- logDebug(s"Native staging write path: $writePath and file name: $writeFileName")
- BackendsApiManager.getIteratorApiInstance.injectWriteFilesTempPath(writePath, writeFileName)
- }
-
- private def doCollectNativeResult(
- cb: ColumnarBatch): Option[(Seq[(Map[String, String], String)], ExecutedWriteSummary)] = {
- val numFiles = cb.numRows()
- // Write an empty iterator
- if (numFiles == 0) {
- None
- } else {
- val file_col = cb.column(0)
- val partition_col = cb.column(1)
- val count_col = cb.column(2)
-
- val partitions: mutable.Set[String] = mutable.Set[String]()
- val addedFiles: ArrayBuffer[(Map[String, String], String)] =
- new ArrayBuffer[(Map[String, String], String)]
-
- var numWrittenRows: Long = 0
- Range(0, cb.numRows()).foreach {
- i =>
- val fileName = file_col.getUTF8String(i).toString
- val partition = partition_col.getUTF8String(i).toString
- if (partition == "__NO_PARTITION_ID__") {
- addedFiles.append((Map.empty[String, String], fileName))
- } else {
- val partitionValues = committer.parsePartitions(partition)
- addedFiles.append((partitionValues, s"$partition/$fileName"))
- }
- numWrittenRows += count_col.getLong(i)
- }
- val updatedPartitions = partitions.toSet
- Some(
- (
- addedFiles.toSeq,
- ExecutedWriteSummary(
- updatedPartitions = updatedPartitions,
- stats = Seq(CreateBasicWriteTaskStats(numFiles, updatedPartitions, numWrittenRows)))))
- }
- }
-
- override def commitTask(batch: ColumnarBatch): Option[WriteTaskResult] = {
- doCollectNativeResult(batch).map {
- case (addedFiles, summary) =>
- require(addedFiles.nonEmpty, "No files to commit")
-
- committer.updateAddedFiles(addedFiles)
-
- val (taskCommitMessage, taskCommitTime) = Utils.timeTakenMs {
- committer.commitTask(taskAttemptContext)
- }
-
- // Just for update task commit time
- description.statsTrackers.foreach {
- stats => stats.newTaskInstance().getFinalStats(taskCommitTime)
- }
- WriteTaskResult(taskCommitMessage, summary)
- }
- }
-}
-
-object CHDeltaColumnarWrite {
- def apply(
- jobTrackerID: String,
- description: WriteJobDescription,
- committer: FileCommitProtocol): CHColumnarWrite[FileCommitProtocol] = committer match {
- case c: CHDelayedCommitProtocol =>
- CHDelayedCommitProtocolWrite(jobTrackerID, description, c)
- .asInstanceOf[CHColumnarWrite[FileCommitProtocol]]
- case _ =>
- throw new GlutenNotSupportException(
- s"Unsupported committer type: ${committer.getClass.getSimpleName}")
- }
-}
diff --git a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/execution/FileDeltaColumnarWrite.scala b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/execution/FileDeltaColumnarWrite.scala
new file mode 100644
index 000000000000..bf6b0c0074dc
--- /dev/null
+++ b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/execution/FileDeltaColumnarWrite.scala
@@ -0,0 +1,188 @@
+/*
+ * 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.
+ */
+package org.apache.spark.sql.execution
+
+import org.apache.gluten.backendsapi.clickhouse.RuntimeSettings
+import org.apache.gluten.exception.GlutenNotSupportException
+import org.apache.gluten.vectorized.NativeExpressionEvaluator
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.internal.io.FileCommitProtocol
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Projection, UnsafeProjection}
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, DeclarativeAggregate}
+import org.apache.spark.sql.delta.files.{FileDelayedCommitProtocol, MergeTreeDelayedCommitProtocol2}
+import org.apache.spark.sql.delta.stats.DeltaFileStatistics
+import org.apache.spark.sql.execution.datasources.{ExecutedWriteSummary, WriteJobDescription, WriteTaskResult}
+import org.apache.spark.sql.types.StringType
+import org.apache.spark.util.Utils
+
+import scala.collection.mutable.ArrayBuffer
+
+case class DeltaFileCommitInfo(committer: FileDelayedCommitProtocol)
+ extends (NativeFileWriteResult => Unit) {
+ val addedFiles: ArrayBuffer[(Map[String, String], String)] =
+ new ArrayBuffer[(Map[String, String], String)]
+ override def apply(stat: NativeFileWriteResult): Unit = {
+ if (stat.partition_id == "__NO_PARTITION_ID__") {
+ addedFiles.append((Map.empty[String, String], stat.filename))
+ } else {
+ val partitionValues = committer.parsePartitions(stat.partition_id)
+ addedFiles.append((partitionValues, stat.relativePath))
+ }
+ }
+
+ def result: Seq[(Map[String, String], String)] = addedFiles.toSeq
+}
+
+case class NativeDeltaStats(projection: Projection) extends (InternalRow => Unit) {
+ protected val results = new collection.mutable.HashMap[String, String]
+
+ override def apply(row: InternalRow): Unit = {
+ val filename = row.getString(0)
+ val jsonStats = projection(row).getString(0)
+ assert(!results.contains(filename), s"Duplicate filename: $filename")
+ results.put(filename, jsonStats)
+ }
+
+ def result: DeltaFileStatistics = DeltaFileStatistics(results.toMap)
+}
+case class FileDeltaColumnarWrite(
+ override val jobTrackerID: String,
+ override val description: WriteJobDescription,
+ override val committer: FileDelayedCommitProtocol)
+ extends CHColumnarWrite[FileDelayedCommitProtocol]
+ with Logging {
+
+ private lazy val nativeDeltaStats: Option[NativeDeltaStats] = {
+ deltaWriteJobStatsTracker
+ .map(
+ delta => {
+ val r = delta.statsColExpr.transform {
+ case ae: AggregateExpression
+ if ae.aggregateFunction.isInstanceOf[DeclarativeAggregate] =>
+ ae.aggregateFunction.asInstanceOf[DeclarativeAggregate].evaluateExpression
+ }
+ val z = Seq(
+ AttributeReference("filename", StringType, nullable = false)(),
+ AttributeReference("partition_id", StringType, nullable = false)())
+ val s =
+ delta.statsColExpr
+ .collect {
+ case ae: AggregateExpression
+ if ae.aggregateFunction.isInstanceOf[DeclarativeAggregate] =>
+ ae.aggregateFunction.asInstanceOf[DeclarativeAggregate]
+ }
+ .asInstanceOf[Seq[DeclarativeAggregate]]
+ .flatMap(_.aggBufferAttributes)
+ NativeDeltaStats(
+ UnsafeProjection.create(
+ exprs = Seq(r),
+ inputSchema = z :++ s
+ ))
+ })
+ }
+ override def doSetupNativeTask(): Unit = {
+ assert(description.path == committer.outputPath)
+ val nameSpec = CreateFileNameSpec(taskAttemptContext, description)
+ val writePath = description.path
+ val writeFileName = committer.getFileName(taskAttemptContext, nameSpec.suffix, Map.empty)
+
+ /**
+ * CDC files (CDC_PARTITION_COL = true) are named with "cdc-..." instead of "part-...".So, using
+ * pattern match to replace guid to {}.See the following example:
+ * {{{
+ * part-00000-7d672b28-c079-4b00-bb0a-196c15112918-c000.snappy.parquet
+ * =>
+ * part-00000-{id}.snappy.parquet
+ * }}}
+ */
+ val guidPattern =
+ """.*-([0-9a-fA-F]{8}-[0-9a-fA-F]{4}-[0-9a-fA-F]{4}-[0-9a-fA-F]{4}-[0-9a-fA-F]{12})(?:-c(\d+)\..*)?$""".r
+ val fileNamePattern =
+ guidPattern.replaceAllIn(
+ writeFileName,
+ m => writeFileName.replace(m.group(1), FileNamePlaceHolder.ID))
+
+ logDebug(s"Native staging write path: $writePath and with pattern: $fileNamePattern")
+ val settings =
+ Map(
+ RuntimeSettings.TASK_WRITE_TMP_DIR.key -> writePath,
+ RuntimeSettings.TASK_WRITE_FILENAME_PATTERN.key -> fileNamePattern
+ )
+ NativeExpressionEvaluator.updateQueryRuntimeSettings(settings)
+ }
+
+ private def doCollectNativeResult(stats: Seq[InternalRow])
+ : Option[(Seq[(Map[String, String], String)], ExecutedWriteSummary)] = {
+
+ // Write an empty iterator
+ if (stats.isEmpty) {
+ None
+ } else {
+ // stats.map(row => x.apply(row).getString(0)).foreach(println)
+ // process stats
+ val commitInfo = DeltaFileCommitInfo(committer)
+ val basicNativeStat = NativeBasicWriteTaskStatsTracker(description, basicWriteJobStatsTracker)
+ val basicNativeStats = Seq(commitInfo, basicNativeStat)
+ NativeStatCompute(stats)(basicNativeStats, nativeDeltaStats)
+
+ Some(
+ (
+ commitInfo.result,
+ ExecutedWriteSummary(
+ updatedPartitions = Set.empty,
+ stats = nativeDeltaStats.map(_.result).toSeq ++ Seq(basicNativeStat.result))))
+ }
+ }
+
+ override def commitTask(writeResults: Seq[InternalRow]): Option[WriteTaskResult] = {
+ doCollectNativeResult(writeResults).map {
+ case (addedFiles, summary) =>
+ require(addedFiles.nonEmpty, "No files to commit")
+
+ committer.updateAddedFiles(addedFiles)
+
+ val (taskCommitMessage, taskCommitTime) = Utils.timeTakenMs {
+ committer.commitTask(taskAttemptContext)
+ }
+
+ // Just for update task commit time
+ description.statsTrackers.foreach {
+ stats => stats.newTaskInstance().getFinalStats(taskCommitTime)
+ }
+ WriteTaskResult(taskCommitMessage, summary)
+ }
+ }
+}
+
+object CHDeltaColumnarWrite {
+ def apply(
+ jobTrackerID: String,
+ description: WriteJobDescription,
+ committer: FileCommitProtocol): CHColumnarWrite[FileCommitProtocol] = committer match {
+ case c: FileDelayedCommitProtocol =>
+ FileDeltaColumnarWrite(jobTrackerID, description, c)
+ .asInstanceOf[CHColumnarWrite[FileCommitProtocol]]
+ case m: MergeTreeDelayedCommitProtocol2 =>
+ MergeTreeDeltaColumnarWrite(jobTrackerID, description, m)
+ .asInstanceOf[CHColumnarWrite[FileCommitProtocol]]
+ case _ =>
+ throw new GlutenNotSupportException(
+ s"Unsupported committer type: ${committer.getClass.getSimpleName}")
+ }
+}
diff --git a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/execution/MergeTreeDeltaColumnarWrite.scala b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/execution/MergeTreeDeltaColumnarWrite.scala
new file mode 100644
index 000000000000..3ac9d4c305e7
--- /dev/null
+++ b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/execution/MergeTreeDeltaColumnarWrite.scala
@@ -0,0 +1,193 @@
+/*
+ * 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.
+ */
+package org.apache.spark.sql.execution
+
+import org.apache.gluten.backendsapi.clickhouse.RuntimeSettings
+import org.apache.gluten.vectorized.NativeExpressionEvaluator
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.internal.io.FileCommitProtocol.TaskCommitMessage
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.GenericInternalRow
+import org.apache.spark.sql.delta.actions.{AddFile, FileAction}
+import org.apache.spark.sql.delta.files.MergeTreeDelayedCommitProtocol2
+import org.apache.spark.sql.delta.stats.DeltaStatistics
+import org.apache.spark.sql.delta.util.{JsonUtils, MergeTreePartitionUtils}
+import org.apache.spark.sql.execution.datasources.{BasicWriteTaskStats, ExecutedWriteSummary, WriteJobDescription, WriteTaskResult}
+import org.apache.spark.sql.execution.datasources.mergetree.StorageMeta
+import org.apache.spark.util.Utils
+
+import org.apache.hadoop.fs.Path
+
+import java.util.UUID
+
+import scala.collection.mutable
+import scala.collection.mutable.ArrayBuffer
+import scala.language.implicitConversions
+
+/**
+ * {{{
+ * val schema =
+ * StructType(
+ * StructField("part_name", StringType, false) ::
+ * StructField("partition_id", StringType, false) ::
+ * StructField("record_count", LongType, false) ::
+ * StructField("marks_count", LongType, false) ::
+ * StructField("size_in_bytes", LongType, false) :: Nil)
+ * }}}
+ */
+case class MergeTreeWriteResult(
+ part_name: String,
+ partition_id: String,
+ record_count: Long,
+ marks_count: Long,
+ size_in_bytes: Long) {
+ def apply(
+ database: String,
+ table: String,
+ path: Path,
+ modificationTime: Long,
+ hostName: Seq[String]): FileAction = {
+ val partitionValues = if (partition_id == "__NO_PARTITION_ID__") {
+ Map.empty[String, String]
+ } else {
+ MergeTreePartitionUtils.parsePartitions(partition_id)
+ }
+ val tags = Map[String, String](
+ "database" -> database,
+ "table" -> table,
+ "engine" -> "MergeTree",
+ "path" -> path.toUri.getPath,
+ "targetNode" -> hostName.map(_.trim).mkString(","),
+ "partition" -> "",
+ "uuid" -> "",
+ "rows" -> record_count.toString,
+ "bytesOnDisk" -> size_in_bytes.toString,
+ "dataCompressedBytes" -> -1L.toString,
+ "dataUncompressedBytes" -> -1L.toString,
+ "modificationTime" -> modificationTime.toString,
+ "partitionId" -> "",
+ "minBlockNumber" -> -1L.toString,
+ "maxBlockNumber" -> -1L.toString,
+ "level" -> -1L.toString,
+ "dataVersion" -> -1L.toString,
+ "defaultCompressionCodec" -> "lz4",
+ "bucketNum" -> "",
+ "dirName" -> path.toString,
+ "marks" -> marks_count.toString
+ )
+
+ val stats = Map[String, Any](
+ DeltaStatistics.NUM_RECORDS -> record_count,
+ DeltaStatistics.MIN -> "",
+ DeltaStatistics.MAX -> "",
+ DeltaStatistics.NULL_COUNT -> ""
+ )
+ AddFile(
+ part_name,
+ partitionValues,
+ size_in_bytes,
+ modificationTime,
+ dataChange = true,
+ JsonUtils.toJson(stats),
+ tags)
+ }
+}
+
+object MergeTreeWriteResult {
+ implicit def apply(row: InternalRow): MergeTreeWriteResult = MergeTreeWriteResult(
+ row.getString(0),
+ row.getString(1),
+ row.getLong(2),
+ row.getLong(3),
+ row.getLong(4))
+}
+
+case class MergeTreeCommitInfo(committer: MergeTreeDelayedCommitProtocol2)
+ extends (MergeTreeWriteResult => Unit) {
+ private val modificationTime = System.currentTimeMillis()
+ private val hostName = Seq(Utils.localHostName())
+ private val addedFiles: ArrayBuffer[FileAction] = new ArrayBuffer[FileAction]
+ private val path = new Path(committer.outputPath)
+ def apply(stat: MergeTreeWriteResult): Unit = {
+ addedFiles.append(
+ stat(committer.database, committer.tableName, path, modificationTime, hostName))
+ }
+ def result: Seq[FileAction] = addedFiles.toSeq
+}
+
+case class MergeTreeBasicWriteTaskStatsTracker() extends (MergeTreeWriteResult => Unit) {
+ private val partitions: mutable.ArrayBuffer[InternalRow] = mutable.ArrayBuffer.empty
+ private var numRows: Long = 0
+ private var numBytes: Long = 0
+ private var numFiles: Int = 0
+
+ def apply(stat: MergeTreeWriteResult): Unit = {
+ if (stat.partition_id != "__NO_PARTITION_ID__") {
+ partitions.append(new GenericInternalRow(Array[Any](stat.partition_id)))
+ }
+ numFiles += 1
+ numRows += stat.record_count
+ numBytes += stat.size_in_bytes
+ }
+
+ def result: BasicWriteTaskStats =
+ BasicWriteTaskStats(partitions.toSeq, numFiles, numBytes, numRows)
+}
+
+case class MergeTreeDeltaColumnarWrite(
+ override val jobTrackerID: String,
+ override val description: WriteJobDescription,
+ override val committer: MergeTreeDelayedCommitProtocol2)
+ extends CHColumnarWrite[MergeTreeDelayedCommitProtocol2]
+ with Logging {
+ override def doSetupNativeTask(): Unit = {
+ assert(description.path == committer.outputPath)
+ val writePath = StorageMeta.normalizeRelativePath(committer.outputPath)
+ val split = taskAttemptContext.getTaskAttemptID.getTaskID.getId
+ val partPrefixWithoutPartitionAndBucket = s"${UUID.randomUUID.toString}_$split"
+ logDebug(
+ s"Pipeline write path: $writePath with part prefix: $partPrefixWithoutPartitionAndBucket")
+ val settings =
+ Map(
+ RuntimeSettings.TASK_WRITE_TMP_DIR.key -> writePath,
+ RuntimeSettings.PART_NAME_PREFIX.key -> partPrefixWithoutPartitionAndBucket)
+ NativeExpressionEvaluator.updateQueryRuntimeSettings(settings)
+ }
+
+ private def doCollectNativeResult(stats: Seq[InternalRow]): Option[WriteTaskResult] = {
+ if (stats.isEmpty) {
+ None
+ } else {
+ val commitInfo = MergeTreeCommitInfo(committer)
+ val mergeTreeStat = MergeTreeBasicWriteTaskStatsTracker()
+ val basicNativeStats = Seq(commitInfo, mergeTreeStat)
+ NativeStatCompute(stats)(basicNativeStats)
+
+ Some {
+ WriteTaskResult(
+ new TaskCommitMessage(commitInfo.result),
+ ExecutedWriteSummary(updatedPartitions = Set.empty, stats = Seq(mergeTreeStat.result))
+ )
+ }
+ }
+ }
+
+ override def commitTask(writeResults: Seq[InternalRow]): Option[WriteTaskResult] = {
+ doCollectNativeResult(writeResults)
+ }
+}
diff --git a/backends-clickhouse/src/main/java/org/apache/gluten/memory/CHThreadGroup.java b/backends-clickhouse/src/main/java/org/apache/gluten/memory/CHThreadGroup.java
index 119dc61893d9..36f7c388baee 100644
--- a/backends-clickhouse/src/main/java/org/apache/gluten/memory/CHThreadGroup.java
+++ b/backends-clickhouse/src/main/java/org/apache/gluten/memory/CHThreadGroup.java
@@ -28,7 +28,13 @@ public class CHThreadGroup implements TaskResource {
*/
public static void registerNewThreadGroup() {
if (TaskResources.isResourceRegistered(CHThreadGroup.class.getName())) return;
- CHThreadGroup group = new CHThreadGroup();
+
+ String taskId = "";
+ if (TaskResources.getLocalTaskContext() != null) {
+ taskId = String.valueOf(TaskResources.getLocalTaskContext().taskAttemptId());
+ }
+
+ CHThreadGroup group = new CHThreadGroup(taskId);
TaskResources.addResource(CHThreadGroup.class.getName(), group);
TaskContext.get()
.addTaskCompletionListener(
@@ -40,8 +46,8 @@ public static void registerNewThreadGroup() {
private long thread_group_id = 0;
private long peak_memory = -1;
- private CHThreadGroup() {
- thread_group_id = createThreadGroup();
+ private CHThreadGroup(String taskId) {
+ thread_group_id = createThreadGroup(taskId);
}
public long getPeakMemory() {
@@ -69,7 +75,7 @@ public String resourceName() {
return "CHThreadGroup";
}
- private static native long createThreadGroup();
+ private static native long createThreadGroup(String taskId);
private static native long threadGroupPeakMemory(long id);
diff --git a/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/CHNativeExpressionEvaluator.java b/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/CHNativeExpressionEvaluator.java
index a4f272a85355..2099ddbbf545 100644
--- a/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/CHNativeExpressionEvaluator.java
+++ b/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/CHNativeExpressionEvaluator.java
@@ -17,7 +17,6 @@
package org.apache.gluten.vectorized;
import org.apache.gluten.GlutenConfig;
-import org.apache.gluten.backend.Backend;
import org.apache.gluten.backendsapi.BackendsApiManager;
import org.apache.gluten.execution.ColumnarNativeIterator;
import org.apache.gluten.memory.CHThreadGroup;
@@ -25,7 +24,6 @@
import org.apache.spark.sql.internal.SQLConf;
-import java.nio.charset.StandardCharsets;
import java.util.List;
import java.util.Map;
@@ -36,11 +34,12 @@ private CHNativeExpressionEvaluator() {}
// Used to initialize the native computing.
public static void initNative(scala.collection.Map conf) {
Map nativeConfMap =
- GlutenConfig.getNativeBackendConf(Backend.get().name(), conf);
+ GlutenConfig.getNativeBackendConf(BackendsApiManager.getBackendName(), conf);
// Get the customer config from SparkConf for each backend
BackendsApiManager.getTransformerApiInstance()
- .postProcessNativeConfig(nativeConfMap, GlutenConfig.prefixOf(Backend.get().name()));
+ .postProcessNativeConfig(
+ nativeConfMap, GlutenConfig.prefixOf(BackendsApiManager.getBackendName()));
nativeInitNative(ConfigUtil.serialize(nativeConfMap));
}
@@ -55,12 +54,8 @@ public static boolean doValidate(byte[] subPlan) {
}
private static Map getNativeBackendConf() {
- return GlutenConfig.getNativeBackendConf(Backend.get().name(), SQLConf.get().getAllConfs());
- }
-
- public static void injectWriteFilesTempPath(String path, String fileName) {
- ExpressionEvaluatorJniWrapper.injectWriteFilesTempPath(
- path.getBytes(StandardCharsets.UTF_8), fileName.getBytes(StandardCharsets.UTF_8));
+ return GlutenConfig.getNativeBackendConf(
+ BackendsApiManager.getBackendName(), SQLConf.get().getAllConfs());
}
// Used by WholeStageTransform to create the native computing pipeline and
diff --git a/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/ExpressionEvaluatorJniWrapper.java b/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/ExpressionEvaluatorJniWrapper.java
index 8cc3046436ab..76fddcafdd41 100644
--- a/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/ExpressionEvaluatorJniWrapper.java
+++ b/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/ExpressionEvaluatorJniWrapper.java
@@ -16,12 +16,7 @@
*/
package org.apache.gluten.vectorized;
-import org.apache.gluten.backendsapi.clickhouse.CHConf;
import org.apache.gluten.execution.ColumnarNativeIterator;
-import org.apache.gluten.utils.ConfigUtil;
-
-import java.util.Map;
-import java.util.stream.Collectors;
/**
* This class is implemented in JNI. This provides the Java interface to invoke functions in JNI.
@@ -48,23 +43,5 @@ public static native long nativeCreateKernelWithIterator(
byte[] confArray,
boolean materializeInput);
- /**
- * Set the temp path for writing files.
- *
- * @param path the temp path for writing files
- */
- public static native void injectWriteFilesTempPath(byte[] path, byte[] filename);
-
- /// The following methods are used to update the query settings in the native engine.
- public static void updateQueryRuntimeSettings(Map settings) {
- Map newSettings =
- settings.entrySet().stream()
- .filter(entry -> CHConf.startWithSettingsPrefix(entry.getKey()))
- .collect(
- Collectors.toMap(
- e -> CHConf.removeSettingsPrefix(e.getKey()), Map.Entry::getValue));
- updateQueryRuntimeSettings(ConfigUtil.serialize(newSettings));
- }
-
- private static native void updateQueryRuntimeSettings(byte[] settings);
+ public static native void updateQueryRuntimeSettings(byte[] settings);
}
diff --git a/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/SimpleExpressionEval.java b/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/SimpleExpressionEval.java
index b09cccb4580f..c73e1076ddf8 100644
--- a/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/SimpleExpressionEval.java
+++ b/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/SimpleExpressionEval.java
@@ -34,7 +34,7 @@ public class SimpleExpressionEval implements AutoCloseable, Iterator {
public SimpleExpressionEval(ColumnarNativeIterator blockStream, PlanNode planNode) {
Plan plan = planNode.toProtobuf();
if (LOG.isDebugEnabled()) {
- LOG.debug(String.format("SimpleExpressionEval exec plan: %s", plan.toString()));
+ LOG.debug("SimpleExpressionEval exec plan: {}", plan.toString());
}
byte[] planData = plan.toByteArray();
instance = createNativeInstance(blockStream, planData);
diff --git a/backends-clickhouse/src/main/java/org/apache/spark/sql/execution/datasources/CHDatasourceJniWrapper.java b/backends-clickhouse/src/main/java/org/apache/spark/sql/execution/datasources/CHDatasourceJniWrapper.java
index fb2b50798836..ee000d5ae157 100644
--- a/backends-clickhouse/src/main/java/org/apache/spark/sql/execution/datasources/CHDatasourceJniWrapper.java
+++ b/backends-clickhouse/src/main/java/org/apache/spark/sql/execution/datasources/CHDatasourceJniWrapper.java
@@ -26,10 +26,8 @@ public CHDatasourceJniWrapper(String filePath, WriteRel write) {
this.instance = createFilerWriter(filePath, write.toByteArray());
}
- public CHDatasourceJniWrapper(
- String prefix, String partition, String bucket, WriteRel write, byte[] confArray) {
- this.instance =
- createMergeTreeWriter(prefix, partition, bucket, write.toByteArray(), confArray);
+ public CHDatasourceJniWrapper(WriteRel write, byte[] confArray) {
+ this.instance = createMergeTreeWriter(write.toByteArray(), confArray);
}
public void write(long blockAddress) {
@@ -48,8 +46,7 @@ public void close() {
private native long createFilerWriter(String filePath, byte[] writeRel);
/// MergeTreeWriter
- private native long createMergeTreeWriter(
- String prefix, String partition, String bucket, byte[] writeRel, byte[] confArray);
+ private native long createMergeTreeWriter(byte[] writeRel, byte[] confArray);
public static native String nativeMergeMTParts(
byte[] splitInfo, String partition_dir, String bucket_dir);
diff --git a/backends-clickhouse/src/main/java/org/apache/spark/sql/execution/datasources/clickhouse/WriteReturnedMetric.java b/backends-clickhouse/src/main/java/org/apache/spark/sql/execution/datasources/clickhouse/WriteReturnedMetric.java
index 223e92c28d1d..49311504ced0 100644
--- a/backends-clickhouse/src/main/java/org/apache/spark/sql/execution/datasources/clickhouse/WriteReturnedMetric.java
+++ b/backends-clickhouse/src/main/java/org/apache/spark/sql/execution/datasources/clickhouse/WriteReturnedMetric.java
@@ -18,8 +18,6 @@
import com.fasterxml.jackson.annotation.JsonProperty;
-import java.util.Map;
-
public class WriteReturnedMetric {
@JsonProperty("part_name")
@@ -38,7 +36,7 @@ public class WriteReturnedMetric {
protected String bucketId;
@JsonProperty("partition_values")
- protected Map partitionValues;
+ protected String partitionValues;
public String getPartName() {
return partName;
@@ -72,11 +70,11 @@ public void setRowCount(long rowCount) {
this.rowCount = rowCount;
}
- public Map getPartitionValues() {
+ public String getPartitionValues() {
return partitionValues;
}
- public void setPartitionValues(Map partitionValues) {
+ public void setPartitionValues(String partitionValues) {
this.partitionValues = partitionValues;
}
diff --git a/backends-clickhouse/src/main/resources/org/apache/spark/sql/execution/datasources/v1/write_optimization.proto b/backends-clickhouse/src/main/resources/org/apache/spark/sql/execution/datasources/v1/write_optimization.proto
index 8404f81f19f4..89f606e4ffd3 100644
--- a/backends-clickhouse/src/main/resources/org/apache/spark/sql/execution/datasources/v1/write_optimization.proto
+++ b/backends-clickhouse/src/main/resources/org/apache/spark/sql/execution/datasources/v1/write_optimization.proto
@@ -6,6 +6,8 @@ package local_engine;
option java_package = "org.apache.spark.sql.execution.datasources.v1";
option java_multiple_files = true;
+//TODO : set compression codec
+
message Write {
message Common {
string format = 1;
diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala
index 59d912d8e75d..83a92db51897 100644
--- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala
+++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala
@@ -18,13 +18,13 @@ package org.apache.gluten.backendsapi.clickhouse
import org.apache.gluten.GlutenBuildInfo._
import org.apache.gluten.GlutenConfig
-import org.apache.gluten.backend.Backend
+import org.apache.gluten.backend.Component.BuildInfo
import org.apache.gluten.backendsapi._
import org.apache.gluten.columnarbatch.CHBatch
import org.apache.gluten.execution.WriteFilesExecTransformer
import org.apache.gluten.expression.WindowFunctionsBuilder
import org.apache.gluten.extension.ValidationResult
-import org.apache.gluten.extension.columnar.transition.Convention
+import org.apache.gluten.extension.columnar.transition.{Convention, ConventionFunc}
import org.apache.gluten.substrait.rel.LocalFilesNode.ReadFileFormat
import org.apache.gluten.substrait.rel.LocalFilesNode.ReadFileFormat._
@@ -34,9 +34,8 @@ import org.apache.spark.sql.catalyst.catalog.BucketSpec
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
import org.apache.spark.sql.catalyst.plans._
-import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning}
import org.apache.spark.sql.execution.SparkPlan
-import org.apache.spark.sql.execution.aggregate.HashAggregateExec
+import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec
import org.apache.spark.sql.execution.datasources.FileFormat
import org.apache.spark.sql.execution.datasources.orc.OrcFileFormat
import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat
@@ -48,10 +47,11 @@ import java.util.Locale
import scala.util.control.Breaks.{break, breakable}
class CHBackend extends SubstraitBackend {
+ import CHBackend._
override def name(): String = CHConf.BACKEND_NAME
- override def defaultBatchType: Convention.BatchType = CHBatch
- override def buildInfo(): Backend.BuildInfo =
- Backend.BuildInfo("ClickHouse", CH_BRANCH, CH_COMMIT, "UNKNOWN")
+ override def buildInfo(): BuildInfo =
+ BuildInfo("ClickHouse", CH_BRANCH, CH_COMMIT, "UNKNOWN")
+ override def convFuncOverride(): ConventionFunc.Override = new ConvFunc()
override def iteratorApi(): IteratorApi = new CHIteratorApi
override def sparkPlanExecApi(): SparkPlanExecApi = new CHSparkPlanExecApi
override def transformerApi(): TransformerApi = new CHTransformerApi
@@ -62,7 +62,17 @@ class CHBackend extends SubstraitBackend {
override def settings(): BackendSettingsApi = CHBackendSettings
}
+object CHBackend {
+ private class ConvFunc() extends ConventionFunc.Override {
+ override def batchTypeOf: PartialFunction[SparkPlan, Convention.BatchType] = {
+ case a: AdaptiveSparkPlanExec if a.supportsColumnar =>
+ CHBatch
+ }
+ }
+}
+
object CHBackendSettings extends BackendSettingsApi with Logging {
+ override def primaryBatchType: Convention.BatchType = CHBatch
private val GLUTEN_CLICKHOUSE_SEP_SCAN_RDD = "spark.gluten.sql.columnar.separate.scan.rdd.for.ch"
private val GLUTEN_CLICKHOUSE_SEP_SCAN_RDD_DEFAULT = "false"
@@ -133,6 +143,14 @@ object CHBackendSettings extends BackendSettingsApi with Logging {
val GLUTEN_CLICKHOUSE_TABLE_PATH_TO_MTPS_CACHE_SIZE: String =
CHConf.prefixOf("table.path.to.mtps.cache.size")
+ val GLUTEN_CLICKHOUSE_DELTA_METADATA_OPTIMIZE: String =
+ CHConf.prefixOf("delta.metadata.optimize")
+ val GLUTEN_CLICKHOUSE_DELTA_METADATA_OPTIMIZE_DEFAULT_VALUE: String = "true"
+
+ val GLUTEN_CLICKHOUSE_CONVERT_LEFT_ANTI_SEMI_TO_RIGHT: String =
+ CHConf.prefixOf("convert.left.anti_semi.to.right")
+ val GLUTEN_CLICKHOUSE_CONVERT_LEFT_ANTI_SEMI_TO_RIGHT_DEFAULT_VALUE: String = "false"
+
def affinityMode: String = {
SparkEnv.get.conf
.get(
@@ -142,10 +160,11 @@ object CHBackendSettings extends BackendSettingsApi with Logging {
.toLowerCase(Locale.getDefault)
}
- override def validateScan(
+ override def validateScanExec(
format: ReadFileFormat,
fields: Array[StructField],
- rootPaths: Seq[String]): ValidationResult = {
+ rootPaths: Seq[String],
+ properties: Map[String, String]): ValidationResult = {
// Validate if all types are supported.
def hasComplexType: Boolean = {
@@ -227,45 +246,16 @@ object CHBackendSettings extends BackendSettingsApi with Logging {
}
}
- def validateBucketSpec(): Option[String] = {
- if (bucketSpec.nonEmpty) {
- Some("Unsupported native write: bucket write is not supported.")
- } else {
- None
- }
- }
-
validateCompressionCodec()
.orElse(validateFileFormat())
.orElse(validateFieldMetadata())
.orElse(validateDateTypes())
- .orElse(validateWriteFilesOptions())
- .orElse(validateBucketSpec()) match {
+ .orElse(validateWriteFilesOptions()) match {
case Some(reason) => ValidationResult.failed(reason)
case _ => ValidationResult.succeeded
}
}
- override def supportShuffleWithProject(
- outputPartitioning: Partitioning,
- child: SparkPlan): Boolean = {
- child match {
- case hash: HashAggregateExec =>
- if (hash.aggregateExpressions.isEmpty) {
- true
- } else {
- outputPartitioning match {
- case hashPartitioning: HashPartitioning =>
- hashPartitioning.expressions.exists(x => !x.isInstanceOf[AttributeReference])
- case _ =>
- false
- }
- }
- case _ =>
- true
- }
- }
-
override def supportSortExec(): Boolean = {
GlutenConfig.getConf.enableColumnarSort
}
@@ -348,18 +338,47 @@ object CHBackendSettings extends BackendSettingsApi with Logging {
)
}
+ // It try to move the expand node after the pre-aggregate node. That is to make the plan from
+ // expand -> pre-aggregate -> shuffle -> final-aggregate
+ // to
+ // pre-aggregate -> expand -> shuffle -> final-aggregate
+ // It could reduce the overhead of pre-aggregate node.
+ def enableLazyAggregateExpand(): Boolean = {
+ SparkEnv.get.conf.getBoolean(
+ CHConf.runtimeConfig("enable_lazy_aggregate_expand"),
+ defaultValue = true
+ )
+ }
+
+ // If the partition keys are high cardinality, the aggregation method is slower.
+ def enableConvertWindowGroupLimitToAggregate(): Boolean = {
+ SparkEnv.get.conf.getBoolean(
+ CHConf.runtimeConfig("enable_window_group_limit_to_aggregate"),
+ defaultValue = true
+ )
+ }
+
override def enableNativeWriteFiles(): Boolean = {
GlutenConfig.getConf.enableNativeWriter.getOrElse(false)
}
override def supportCartesianProductExec(): Boolean = true
+ override def supportCartesianProductExecWithCondition(): Boolean = false
+
override def supportHashBuildJoinTypeOnLeft: JoinType => Boolean = {
t =>
if (super.supportHashBuildJoinTypeOnLeft(t)) {
true
} else {
t match {
+ case LeftAnti | LeftSemi
+ if (SQLConf.get
+ .getConfString(
+ GLUTEN_CLICKHOUSE_CONVERT_LEFT_ANTI_SEMI_TO_RIGHT,
+ GLUTEN_CLICKHOUSE_CONVERT_LEFT_ANTI_SEMI_TO_RIGHT_DEFAULT_VALUE)
+ .toBoolean) =>
+ true
case LeftOuter => true
case _ => false
}
diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHConf.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHConf.scala
index 3f653ad200c0..83cc75bc0ac7 100644
--- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHConf.scala
+++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHConf.scala
@@ -19,6 +19,7 @@ package org.apache.gluten.backendsapi.clickhouse
import org.apache.gluten.GlutenConfig
import org.apache.spark.SparkConf
+import org.apache.spark.sql.internal.SQLConf
object CHConf {
private[clickhouse] val BACKEND_NAME: String = "ch"
@@ -47,10 +48,26 @@ object CHConf {
}
}
+ /** CH configuration prefix at Java side */
def prefixOf(key: String): String = s"$CONF_PREFIX.$key"
def runtimeConfig(key: String): String = s"$RUNTIME_CONFIG.$key"
def runtimeSettings(key: String): String = s"$RUNTIME_SETTINGS.$key"
def startWithSettingsPrefix(key: String): Boolean = key.startsWith(RUNTIME_SETTINGS)
def removeSettingsPrefix(key: String): String = key.substring(RUNTIME_SETTINGS.length + 1)
+
+ def get: CHConf = new CHConf(SQLConf.get)
+
+ import SQLConf._
+
+ val ENABLE_ONEPIPELINE_MERGETREE_WRITE =
+ buildConf(prefixOf("mergetree.write.pipeline"))
+ .doc("Using one pipeline to write data to MergeTree table in Spark 3.5")
+ .booleanConf
+ .createWithDefault(false)
+}
+
+class CHConf(conf: SQLConf) extends GlutenConfig(conf) {
+ def enableOnePipelineMergeTreeWrite: Boolean =
+ conf.getConf(CHConf.ENABLE_ONEPIPELINE_MERGETREE_WRITE)
}
diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHIteratorApi.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHIteratorApi.scala
index 2fa2e4402b79..878e27a5b8c2 100644
--- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHIteratorApi.scala
+++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHIteratorApi.scala
@@ -21,7 +21,6 @@ import org.apache.gluten.backendsapi.IteratorApi
import org.apache.gluten.execution._
import org.apache.gluten.expression.ConverterUtils
import org.apache.gluten.logging.LogLevelUtil
-import org.apache.gluten.memory.CHThreadGroup
import org.apache.gluten.metrics.IMetrics
import org.apache.gluten.sql.shims.SparkShimLoader
import org.apache.gluten.substrait.plan.PlanNode
@@ -44,7 +43,6 @@ import org.apache.spark.sql.execution.metric.SQLMetric
import org.apache.spark.sql.types._
import org.apache.spark.sql.utils.SparkInputMetricsUtil.InputMetricsWrapper
import org.apache.spark.sql.vectorized.ColumnarBatch
-import org.apache.spark.util.SerializableConfiguration
import java.lang.{Long => JLong}
import java.net.URI
@@ -134,8 +132,7 @@ class CHIteratorApi extends IteratorApi with Logging with LogLevelUtil {
partitionSchema: StructType,
fileFormat: ReadFileFormat,
metadataColumnNames: Seq[String],
- properties: Map[String, String],
- serializableHadoopConf: SerializableConfiguration): SplitInfo = {
+ properties: Map[String, String]): SplitInfo = {
partition match {
case p: GlutenMergeTreePartition =>
ExtensionTableBuilder
@@ -325,10 +322,6 @@ class CHIteratorApi extends IteratorApi with Logging with LogLevelUtil {
createNativeIterator(splitInfoByteArray, wsPlan, materializeInput, inputIterators))
}
- override def injectWriteFilesTempPath(path: String, fileName: String): Unit = {
- CHThreadGroup.registerNewThreadGroup()
- CHNativeExpressionEvaluator.injectWriteFilesTempPath(path, fileName)
- }
}
class CollectMetricIterator(
diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHListenerApi.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHListenerApi.scala
index 6ae957912ac5..b93c002561f0 100644
--- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHListenerApi.scala
+++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHListenerApi.scala
@@ -71,7 +71,7 @@ class CHListenerApi extends ListenerApi with Logging {
private def initialize(conf: SparkConf, isDriver: Boolean): Unit = {
// Force batch type initializations.
- CHBatch.getClass
+ CHBatch.ensureRegistered()
SparkDirectoryUtil.init(conf)
val libPath = conf.get(GlutenConfig.GLUTEN_LIB_PATH, StringUtils.EMPTY)
if (StringUtils.isBlank(libPath)) {
diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHMetricsApi.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHMetricsApi.scala
index 6a4f0c9a6f77..a0576a807b98 100644
--- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHMetricsApi.scala
+++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHMetricsApi.scala
@@ -23,7 +23,8 @@ import org.apache.gluten.substrait.{AggregationParams, JoinParams}
import org.apache.spark.SparkContext
import org.apache.spark.internal.Logging
-import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.{ColumnarInputAdapter, SparkPlan}
+import org.apache.spark.sql.execution.adaptive.QueryStageExec
import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics}
import java.lang.{Long => JLong}
@@ -39,21 +40,40 @@ class CHMetricsApi extends MetricsApi with Logging with LogLevelUtil {
}
override def genInputIteratorTransformerMetrics(
- sparkContext: SparkContext): Map[String, SQLMetric] = {
+ child: SparkPlan,
+ sparkContext: SparkContext,
+ forBroadcast: Boolean): Map[String, SQLMetric] = {
+ def metricsPlan(plan: SparkPlan): SparkPlan = {
+ plan match {
+ case ColumnarInputAdapter(child) => metricsPlan(child)
+ case q: QueryStageExec => metricsPlan(q.plan)
+ case _ => plan
+ }
+ }
+
+ val outputMetrics = if (forBroadcast) {
+ metricsPlan(child).metrics
+ .filterKeys(key => key.equals("numOutputRows"))
+ } else {
+ Map(
+ "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")
+ )
+ }
+
Map(
"iterReadTime" -> SQLMetrics.createTimingMetric(
sparkContext,
"time of reading from iterator"),
"numInputRows" -> SQLMetrics.createMetric(sparkContext, "number of input rows"),
- "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"),
"fillingRightJoinSideTime" -> SQLMetrics.createTimingMetric(
sparkContext,
"filling right join side time")
- )
+ ) ++ outputMetrics
}
override def genInputIteratorTransformerMetricsUpdater(
- metrics: Map[String, SQLMetric]): MetricsUpdater = {
+ metrics: Map[String, SQLMetric],
+ forBroadcast: Boolean): MetricsUpdater = {
InputIteratorMetricsUpdater(metrics)
}
@@ -430,6 +450,14 @@ class CHMetricsApi extends MetricsApi with Logging with LogLevelUtil {
s"SampleTransformer metrics update is not supported in CH backend")
}
+ override def genUnionTransformerMetrics(sparkContext: SparkContext): Map[String, SQLMetric] =
+ throw new UnsupportedOperationException(
+ "UnionExecTransformer metrics update is not supported in CH backend")
+
+ override def genUnionTransformerMetricsUpdater(metrics: Map[String, SQLMetric]): MetricsUpdater =
+ throw new UnsupportedOperationException(
+ "UnionExecTransformer metrics update is not supported in CH backend")
+
def genWriteFilesTransformerMetrics(sparkContext: SparkContext): Map[String, SQLMetric] =
Map(
"physicalWrittenBytes" -> SQLMetrics.createMetric(sparkContext, "number of written bytes"),
diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala
index 91698d4cde85..141778688967 100644
--- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala
+++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala
@@ -16,13 +16,19 @@
*/
package org.apache.gluten.backendsapi.clickhouse
+import org.apache.gluten.GlutenConfig
import org.apache.gluten.backendsapi.RuleApi
+import org.apache.gluten.columnarbatch.CHBatch
import org.apache.gluten.extension._
import org.apache.gluten.extension.columnar._
-import org.apache.gluten.extension.columnar.MiscColumnarRules.{RemoveGlutenTableCacheColumnarToRow, RemoveTopmostColumnarToRow, RewriteSubqueryBroadcast, TransformPreOverrides}
-import org.apache.gluten.extension.columnar.rewrite.RewriteSparkPlanRulesManager
+import org.apache.gluten.extension.columnar.MiscColumnarRules.{RemoveGlutenTableCacheColumnarToRow, RemoveTopmostColumnarToRow, RewriteSubqueryBroadcast}
+import org.apache.gluten.extension.columnar.heuristic.{ExpandFallbackPolicy, HeuristicTransform}
+import org.apache.gluten.extension.columnar.offload.{OffloadExchange, OffloadJoin, OffloadOthers}
+import org.apache.gluten.extension.columnar.rewrite._
import org.apache.gluten.extension.columnar.transition.{InsertTransitions, RemoveTransitions}
-import org.apache.gluten.extension.injector.{RuleInjector, SparkInjector}
+import org.apache.gluten.extension.columnar.validator.Validator
+import org.apache.gluten.extension.columnar.validator.Validators.ValidatorBuilderImplicits
+import org.apache.gluten.extension.injector.{Injector, SparkInjector}
import org.apache.gluten.extension.injector.GlutenInjector.{LegacyInjector, RasInjector}
import org.apache.gluten.parser.{GlutenCacheFilesSqlParser, GlutenClickhouseSqlParser}
import org.apache.gluten.sql.shims.SparkShimLoader
@@ -30,22 +36,23 @@ import org.apache.gluten.sql.shims.SparkShimLoader
import org.apache.spark.sql.catalyst.{CHAggregateFunctionRewriteRule, EqualToRewrite}
import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.delta.DeltaLogFileIndex
-import org.apache.spark.sql.execution.{ColumnarCollapseTransformStages, CommandResultExec, FileSourceScanExec, GlutenFallbackReporter, RDDScanExec, SparkPlan}
+import org.apache.spark.sql.delta.rules.CHOptimizeMetadataOnlyDeltaQuery
+import org.apache.spark.sql.execution._
import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper
import org.apache.spark.sql.execution.datasources.v2.V2CommandExec
import org.apache.spark.util.SparkPlanRules
class CHRuleApi extends RuleApi {
import CHRuleApi._
- override def injectRules(injector: RuleInjector): Unit = {
+ override def injectRules(injector: Injector): Unit = {
injectSpark(injector.spark)
injectLegacy(injector.gluten.legacy)
injectRas(injector.gluten.ras)
}
}
-private object CHRuleApi {
- def injectSpark(injector: SparkInjector): Unit = {
+object CHRuleApi {
+ private def injectSpark(injector: SparkInjector): Unit = {
// Inject the regular Spark rules directly.
injector.injectQueryStagePrepRule(FallbackBroadcastHashJoinPrepQueryStage.apply)
injector.injectQueryStagePrepRule(spark => CHAQEPropagateEmptyRelation(spark))
@@ -56,38 +63,62 @@ private object CHRuleApi {
injector.injectResolutionRule(spark => new RewriteToDateExpresstionRule(spark))
injector.injectResolutionRule(spark => new RewriteDateTimestampComparisonRule(spark))
injector.injectOptimizerRule(spark => new CommonSubexpressionEliminateRule(spark))
+ injector.injectOptimizerRule(spark => new ExtendedColumnPruning(spark))
injector.injectOptimizerRule(spark => CHAggregateFunctionRewriteRule(spark))
injector.injectOptimizerRule(_ => CountDistinctWithoutExpand)
injector.injectOptimizerRule(_ => EqualToRewrite)
+ injector.injectPreCBORule(spark => new CHOptimizeMetadataOnlyDeltaQuery(spark))
}
- def injectLegacy(injector: LegacyInjector): Unit = {
-
- // Gluten columnar: Transform rules.
- injector.injectTransform(_ => RemoveTransitions)
- injector.injectTransform(_ => PushDownInputFileExpression.PreOffload)
- injector.injectTransform(c => FallbackOnANSIMode.apply(c.session))
- injector.injectTransform(c => FallbackMultiCodegens.apply(c.session))
- injector.injectTransform(_ => RewriteSubqueryBroadcast())
- injector.injectTransform(c => FallbackBroadcastHashJoin.apply(c.session))
- injector.injectTransform(c => MergeTwoPhasesHashBaseAggregate.apply(c.session))
- injector.injectTransform(_ => intercept(RewriteSparkPlanRulesManager()))
- injector.injectTransform(_ => intercept(AddFallbackTagRule()))
- injector.injectTransform(_ => intercept(TransformPreOverrides()))
- injector.injectTransform(_ => RemoveNativeWriteFilesSortAndProject())
- injector.injectTransform(c => intercept(RewriteTransformer.apply(c.session)))
- injector.injectTransform(_ => PushDownFilterToScan)
- injector.injectTransform(_ => PushDownInputFileExpression.PostOffload)
- injector.injectTransform(_ => EnsureLocalSortRequirements)
- injector.injectTransform(_ => EliminateLocalSort)
- injector.injectTransform(_ => CollapseProjectExecTransformer)
- injector.injectTransform(c => RewriteSortMergeJoinToHashJoinRule.apply(c.session))
- injector.injectTransform(c => PushdownAggregatePreProjectionAheadExpand.apply(c.session))
+ private def injectLegacy(injector: LegacyInjector): Unit = {
+ // Legacy: Pre-transform rules.
+ injector.injectPreTransform(_ => RemoveTransitions)
+ injector.injectPreTransform(_ => PushDownInputFileExpression.PreOffload)
+ injector.injectPreTransform(c => FallbackOnANSIMode.apply(c.session))
+ injector.injectPreTransform(c => FallbackMultiCodegens.apply(c.session))
+ injector.injectPreTransform(_ => RewriteSubqueryBroadcast())
+ injector.injectPreTransform(c => FallbackBroadcastHashJoin.apply(c.session))
+ injector.injectPreTransform(c => MergeTwoPhasesHashBaseAggregate.apply(c.session))
+ injector.injectPreTransform(_ => WriteFilesWithBucketValue)
+
+ // Legacy: The legacy transform rule.
+ val validatorBuilder: GlutenConfig => Validator = conf =>
+ Validator
+ .builder()
+ .fallbackByHint()
+ .fallbackIfScanOnlyWithFilterPushed(conf.enableScanOnly)
+ .fallbackComplexExpressions()
+ .fallbackByBackendSettings()
+ .fallbackByUserOptions()
+ .fallbackByTestInjects()
+ .fallbackByNativeValidation()
+ .build()
+ val rewrites =
+ Seq(RewriteIn, RewriteMultiChildrenCount, RewriteJoin, PullOutPreProject, PullOutPostProject)
+ val offloads = Seq(OffloadOthers(), OffloadExchange(), OffloadJoin())
injector.injectTransform(
+ c => intercept(HeuristicTransform.Single(validatorBuilder(c.glutenConf), rewrites, offloads)))
+
+ // Legacy: Post-transform rules.
+ injector.injectPostTransform(_ => PruneNestedColumnsInHiveTableScan)
+ injector.injectPostTransform(_ => RemoveNativeWriteFilesSortAndProject())
+ injector.injectPostTransform(c => intercept(RewriteTransformer.apply(c.session)))
+ injector.injectPostTransform(_ => PushDownFilterToScan)
+ injector.injectPostTransform(_ => PushDownInputFileExpression.PostOffload)
+ injector.injectPostTransform(_ => EnsureLocalSortRequirements)
+ injector.injectPostTransform(_ => EliminateLocalSort)
+ injector.injectPostTransform(_ => CollapseProjectExecTransformer)
+ injector.injectPostTransform(c => RewriteSortMergeJoinToHashJoinRule.apply(c.session))
+ injector.injectPostTransform(c => PushdownAggregatePreProjectionAheadExpand.apply(c.session))
+ injector.injectPostTransform(c => LazyAggregateExpandRule.apply(c.session))
+ injector.injectPostTransform(c => ConverRowNumbertWindowToAggregateRule(c.session))
+ injector.injectPostTransform(
c =>
intercept(
- SparkPlanRules.extendedColumnarRule(c.conf.extendedColumnarTransformRules)(c.session)))
- injector.injectTransform(c => InsertTransitions(c.outputsColumnar))
+ SparkPlanRules.extendedColumnarRule(c.glutenConf.extendedColumnarTransformRules)(
+ c.session)))
+ injector.injectPostTransform(c => InsertTransitions.create(c.outputsColumnar, CHBatch))
+ injector.injectPostTransform(c => RemoveDuplicatedColumns.apply(c.session))
// Gluten columnar: Fallback policies.
injector.injectFallbackPolicy(
@@ -98,21 +129,22 @@ private object CHRuleApi {
SparkShimLoader.getSparkShims
.getExtendedColumnarPostRules()
.foreach(each => injector.injectPost(c => intercept(each(c.session))))
- injector.injectPost(c => ColumnarCollapseTransformStages(c.conf))
- injector.injectTransform(
+ injector.injectPost(c => ColumnarCollapseTransformStages(c.glutenConf))
+ injector.injectPost(
c =>
- intercept(SparkPlanRules.extendedColumnarRule(c.conf.extendedColumnarPostRules)(c.session)))
+ intercept(
+ SparkPlanRules.extendedColumnarRule(c.glutenConf.extendedColumnarPostRules)(c.session)))
// Gluten columnar: Final rules.
injector.injectFinal(c => RemoveGlutenTableCacheColumnarToRow(c.session))
- injector.injectFinal(c => GlutenFallbackReporter(c.conf, c.session))
+ injector.injectFinal(c => GlutenFallbackReporter(c.glutenConf, c.session))
injector.injectFinal(_ => RemoveFallbackTagRule())
}
- def injectRas(injector: RasInjector): Unit = {
+ private def injectRas(injector: RasInjector): Unit = {
// CH backend doesn't work with RAS at the moment. Inject a rule that aborts any
// execution calls.
- injector.inject(
+ injector.injectPreTransform(
_ =>
new SparkPlanRules.AbortRule(
"Clickhouse backend doesn't yet have RAS support, please try disabling RAS and" +
diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala
index ba165d936eed..de0680df10ab 100644
--- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala
+++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala
@@ -23,8 +23,7 @@ import org.apache.gluten.execution._
import org.apache.gluten.expression._
import org.apache.gluten.expression.ExpressionNames.MONOTONICALLY_INCREASING_ID
import org.apache.gluten.extension.ExpressionExtensionTrait
-import org.apache.gluten.extension.columnar.AddFallbackTagRule
-import org.apache.gluten.extension.columnar.MiscColumnarRules.TransformPreOverrides
+import org.apache.gluten.extension.columnar.heuristic.HeuristicTransform
import org.apache.gluten.sql.shims.SparkShimLoader
import org.apache.gluten.substrait.expression.{ExpressionBuilder, ExpressionNode, WindowFunctionNode}
import org.apache.gluten.utils.{CHJoinValidateUtil, UnknownJoinStrategy}
@@ -158,16 +157,21 @@ class CHSparkPlanExecApi extends SparkPlanExecApi with Logging {
aggregateAttributes: Seq[Attribute],
initialInputBufferOffset: Int,
resultExpressions: Seq[NamedExpression],
- child: SparkPlan): HashAggregateExecBaseTransformer =
+ child: SparkPlan): HashAggregateExecBaseTransformer = {
+ val replacedResultExpressions = CHHashAggregateExecTransformer.getCHAggregateResultExpressions(
+ groupingExpressions,
+ aggregateExpressions,
+ resultExpressions)
CHHashAggregateExecTransformer(
requiredChildDistributionExpressions,
- groupingExpressions.distinct,
+ groupingExpressions,
aggregateExpressions,
aggregateAttributes,
initialInputBufferOffset,
- resultExpressions.distinct,
+ replacedResultExpressions,
child
)
+ }
/** Generate HashAggregateExecPullOutHelper */
override def genHashAggregateExecPullOutHelper(
@@ -219,9 +223,10 @@ class CHSparkPlanExecApi extends SparkPlanExecApi with Logging {
}
// FIXME: The operation happens inside ReplaceSingleNode().
// Caller may not know it adds project on top of the shuffle.
- val project = TransformPreOverrides().apply(
- AddFallbackTagRule().apply(
- ProjectExec(plan.child.output ++ projectExpressions, plan.child)))
+ // FIXME: HeuristicTransform is costly. Re-applying it may cause performance issues.
+ val project =
+ HeuristicTransform.static()(
+ ProjectExec(plan.child.output ++ projectExpressions, plan.child))
var newExprs = Seq[Expression]()
for (i <- exprs.indices) {
val pos = newExpressionsPosition(i)
@@ -244,9 +249,10 @@ class CHSparkPlanExecApi extends SparkPlanExecApi with Logging {
}
// FIXME: The operation happens inside ReplaceSingleNode().
// Caller may not know it adds project on top of the shuffle.
- val project = TransformPreOverrides().apply(
- AddFallbackTagRule().apply(
- ProjectExec(plan.child.output ++ projectExpressions, plan.child)))
+ // FIXME: HeuristicTransform is costly. Re-applying it may cause performance issues.
+ val project =
+ HeuristicTransform.static()(
+ ProjectExec(plan.child.output ++ projectExpressions, plan.child))
var newOrderings = Seq[SortOrder]()
for (i <- orderings.indices) {
val oldOrdering = orderings(i)
@@ -267,9 +273,7 @@ class CHSparkPlanExecApi extends SparkPlanExecApi with Logging {
override def genColumnarShuffleExchange(shuffle: ShuffleExchangeExec): SparkPlan = {
val child = shuffle.child
- if (
- BackendsApiManager.getSettings.supportShuffleWithProject(shuffle.outputPartitioning, child)
- ) {
+ if (CHValidatorApi.supportShuffleWithProject(shuffle.outputPartitioning, child)) {
val (projectColumnNumber, newPartitioning, newChild) =
addProjectionForShuffleExchange(shuffle)
@@ -359,15 +363,10 @@ class CHSparkPlanExecApi extends SparkPlanExecApi with Logging {
left: SparkPlan,
right: SparkPlan,
condition: Option[Expression]): CartesianProductExecTransformer =
- if (!condition.isEmpty) {
- throw new GlutenNotSupportException(
- "CartesianProductExecTransformer with condition is not supported in ch backend.")
- } else {
- CartesianProductExecTransformer(
- ColumnarCartesianProductBridge(left),
- ColumnarCartesianProductBridge(right),
- condition)
- }
+ CartesianProductExecTransformer(
+ ColumnarCartesianProductBridge(left),
+ ColumnarCartesianProductBridge(right),
+ condition)
override def genBroadcastNestedLoopJoinExecTransformer(
left: SparkPlan,
diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHTransformerApi.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHTransformerApi.scala
index e5b71825853b..0be8cf2c25bf 100644
--- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHTransformerApi.scala
+++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHTransformerApi.scala
@@ -17,7 +17,7 @@
package org.apache.gluten.backendsapi.clickhouse
import org.apache.gluten.backendsapi.TransformerApi
-import org.apache.gluten.execution.{CHHashAggregateExecTransformer, WriteFilesExecTransformer}
+import org.apache.gluten.execution.CHHashAggregateExecTransformer
import org.apache.gluten.expression.ConverterUtils
import org.apache.gluten.substrait.expression.{BooleanLiteralNode, ExpressionBuilder, ExpressionNode}
import org.apache.gluten.utils.{CHInputPartitionsUtil, ExpressionDocUtil}
@@ -26,18 +26,23 @@ import org.apache.spark.internal.Logging
import org.apache.spark.rpc.GlutenDriverEndpoint
import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression}
import org.apache.spark.sql.connector.read.InputPartition
+import org.apache.spark.sql.delta.MergeTreeFileFormat
import org.apache.spark.sql.delta.catalog.ClickHouseTableV2
import org.apache.spark.sql.delta.files.TahoeFileIndex
import org.apache.spark.sql.execution.SparkPlan
import org.apache.spark.sql.execution.aggregate.HashAggregateExec
import org.apache.spark.sql.execution.datasources.{FileFormat, HadoopFsRelation, PartitionDirectory}
+import org.apache.spark.sql.execution.datasources.orc.OrcFileFormat
+import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat
+import org.apache.spark.sql.execution.datasources.v1.Write
import org.apache.spark.sql.execution.datasources.v2.clickhouse.source.DeltaMergeTreeFileFormat
import org.apache.spark.sql.sources.DataSourceRegister
import org.apache.spark.sql.types._
import org.apache.spark.util.collection.BitSet
import com.google.common.collect.Lists
-import com.google.protobuf.{Any, Message, StringValue}
+import com.google.protobuf.{Any, Message}
+import org.apache.hadoop.fs.Path
import java.util
@@ -177,10 +182,13 @@ class CHTransformerApi extends TransformerApi with Logging {
// output name will be different from grouping expressions,
// so using output attribute instead of grouping expression
val groupingExpressions = hash.output.splitAt(hash.groupingExpressions.size)._1
- val aggResultAttributes = CHHashAggregateExecTransformer.getAggregateResultAttributes(
- groupingExpressions,
- hash.aggregateExpressions
- )
+ val aggResultAttributes = CHHashAggregateExecTransformer
+ .getCHAggregateResultExpressions(
+ groupingExpressions,
+ hash.aggregateExpressions,
+ hash.resultExpressions
+ )
+ .map(_.toAttribute)
if (aggResultAttributes.size == hash.output.size) {
aggResultAttributes
} else {
@@ -243,16 +251,27 @@ class CHTransformerApi extends TransformerApi with Logging {
register.shortName
case _ => "UnknownFileFormat"
}
- val compressionCodec =
- WriteFilesExecTransformer.getCompressionCodec(writeOptions).capitalize
- val writeParametersStr = new StringBuffer("WriteParameters:")
- writeParametersStr.append("is").append(compressionCodec).append("=1")
- writeParametersStr.append(";format=").append(fileFormatStr).append("\n")
+ val write = Write
+ .newBuilder()
+ .setCommon(
+ Write.Common
+ .newBuilder()
+ .setFormat(fileFormatStr)
+ .setJobTaskAttemptId("") // we can get job and task id at the driver side
+ .build())
- packPBMessage(
- StringValue
- .newBuilder()
- .setValue(writeParametersStr.toString)
- .build())
+ fileFormat match {
+ case d: MergeTreeFileFormat =>
+ write.setMergetree(MergeTreeFileFormat.createWrite(d.metadata))
+ case _: ParquetFileFormat =>
+ write.setParquet(Write.ParquetWrite.newBuilder().build())
+ case _: OrcFileFormat =>
+ write.setOrc(Write.OrcWrite.newBuilder().build())
+ }
+ packPBMessage(write.build())
}
+
+ /** use Hadoop Path class to encode the file path */
+ override def encodeFilePathIfNeed(filePath: String): String =
+ (new Path(filePath)).toUri.toASCIIString
}
diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHValidatorApi.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHValidatorApi.scala
index eed493cffe1e..29d26410b9b0 100644
--- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHValidatorApi.scala
+++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHValidatorApi.scala
@@ -17,7 +17,7 @@
package org.apache.gluten.backendsapi.clickhouse
import org.apache.gluten.GlutenConfig
-import org.apache.gluten.backendsapi.{BackendsApiManager, ValidatorApi}
+import org.apache.gluten.backendsapi.ValidatorApi
import org.apache.gluten.expression.ExpressionConverter
import org.apache.gluten.extension.ValidationResult
import org.apache.gluten.substrait.SubstraitContext
@@ -28,12 +28,14 @@ import org.apache.gluten.vectorized.CHNativeExpressionEvaluator
import org.apache.spark.internal.Logging
import org.apache.spark.shuffle.utils.RangePartitionerBoundsGenerator
-import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression}
+import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Expression}
import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, RangePartitioning}
import org.apache.spark.sql.execution.SparkPlan
import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper
+import org.apache.spark.sql.execution.aggregate.HashAggregateExec
class CHValidatorApi extends ValidatorApi with AdaptiveSparkPlanHelper with Logging {
+ import CHValidatorApi._
override def doNativeValidateWithFailureReason(plan: PlanNode): ValidationResult = {
if (CHNativeExpressionEvaluator.doValidate(plan.toProtobuf.toByteArray)) {
@@ -86,10 +88,7 @@ class CHValidatorApi extends ValidatorApi with AdaptiveSparkPlanHelper with Logg
.doTransform(substraitContext.registeredFunction)
node.isInstanceOf[SelectionNode]
}
- if (
- allSelectionNodes ||
- BackendsApiManager.getSettings.supportShuffleWithProject(outputPartitioning, child)
- ) {
+ if (allSelectionNodes || supportShuffleWithProject(outputPartitioning, child)) {
None
} else {
Some("expressions are not supported in HashPartitioning")
@@ -107,3 +106,31 @@ class CHValidatorApi extends ValidatorApi with AdaptiveSparkPlanHelper with Logg
}
}
}
+
+object CHValidatorApi {
+
+ /**
+ * A shuffle key may be an expression. We would add a projection for this expression shuffle key
+ * and make it into a new column which the shuffle will refer to. But we need to remove it from
+ * the result columns from the shuffle.
+ *
+ * Since https://github.com/apache/incubator-gluten/pull/1071.
+ */
+ def supportShuffleWithProject(outputPartitioning: Partitioning, child: SparkPlan): Boolean = {
+ child match {
+ case hash: HashAggregateExec =>
+ if (hash.aggregateExpressions.isEmpty) {
+ true
+ } else {
+ outputPartitioning match {
+ case hashPartitioning: HashPartitioning =>
+ hashPartitioning.expressions.exists(x => !x.isInstanceOf[AttributeReference])
+ case _ =>
+ false
+ }
+ }
+ case _ =>
+ true
+ }
+ }
+}
diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/RuntimeConfig.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/RuntimeConfig.scala
new file mode 100644
index 000000000000..12bb8d05d953
--- /dev/null
+++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/RuntimeConfig.scala
@@ -0,0 +1,45 @@
+/*
+ * 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.
+ */
+package org.apache.gluten.backendsapi.clickhouse
+
+import org.apache.spark.sql.internal.SQLConf
+
+object RuntimeConfig {
+ import CHConf._
+ import SQLConf._
+
+ val PATH =
+ buildConf(runtimeConfig("path"))
+ .doc(
+ "https://clickhouse.com/docs/en/operations/server-configuration-parameters/settings#path")
+ .stringConf
+ .createWithDefault("/")
+
+ // scalastyle:off line.size.limit
+ val TMP_PATH =
+ buildConf(runtimeConfig("tmp_path"))
+ .doc("https://clickhouse.com/docs/en/operations/server-configuration-parameters/settings#tmp-path")
+ .stringConf
+ .createWithDefault("/tmp/libch")
+ // scalastyle:on line.size.limit
+
+ val USE_CURRENT_DIRECTORY_AS_TMP =
+ buildConf(runtimeConfig("use_current_directory_as_tmp"))
+ .doc("Use the current directory as the temporary directory.")
+ .booleanConf
+ .createWithDefault(false)
+}
diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/RuntimeSettings.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/RuntimeSettings.scala
new file mode 100644
index 000000000000..c2747cf1eb53
--- /dev/null
+++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/RuntimeSettings.scala
@@ -0,0 +1,61 @@
+/*
+ * 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.
+ */
+package org.apache.gluten.backendsapi.clickhouse
+
+import org.apache.spark.sql.internal.SQLConf
+
+object RuntimeSettings {
+
+ import CHConf._
+ import SQLConf._
+
+ val NATIVE_WRITE_RESERVE_PARTITION_COLUMNS =
+ buildConf(runtimeSettings("gluten.write.reserve_partition_columns"))
+ .doc("Whether reserve partition columns for Native write or not, default is false")
+ .booleanConf
+ .createWithDefault(false)
+
+ val TASK_WRITE_TMP_DIR =
+ buildConf(runtimeSettings("gluten.task_write_tmp_dir"))
+ .doc("The temporary directory for writing data")
+ .stringConf
+ .createWithDefault("")
+
+ val TASK_WRITE_FILENAME_PATTERN =
+ buildConf(runtimeSettings("gluten.task_write_filename_pattern"))
+ .doc("The pattern to generate file name for writing delta parquet in spark 3.5")
+ .stringConf
+ .createWithDefault("")
+
+ val PART_NAME_PREFIX =
+ buildConf(runtimeSettings("gluten.part_name_prefix"))
+ .doc("The part name prefix for writing data")
+ .stringConf
+ .createWithDefault("")
+
+ val PARTITION_DIR =
+ buildConf(runtimeSettings("gluten.partition_dir"))
+ .doc("The partition directory for writing data")
+ .stringConf
+ .createWithDefault("")
+
+ val BUCKET_DIR =
+ buildConf(runtimeSettings("gluten.bucket_dir"))
+ .doc("The bucket directory for writing data")
+ .stringConf
+ .createWithDefault("")
+}
diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/columnarbatch/CHBatch.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/columnarbatch/CHBatch.scala
index 870a731b1176..ac0ca5f8b4b7 100644
--- a/backends-clickhouse/src/main/scala/org/apache/gluten/columnarbatch/CHBatch.scala
+++ b/backends-clickhouse/src/main/scala/org/apache/gluten/columnarbatch/CHBatch.scala
@@ -38,6 +38,8 @@ import org.apache.spark.sql.execution.{CHColumnarToRowExec, RowToCHNativeColumna
* }}}
*/
object CHBatch extends Convention.BatchType {
- fromRow(RowToCHNativeColumnarExec.apply)
- toRow(CHColumnarToRowExec.apply)
+ override protected def registerTransitions(): Unit = {
+ fromRow(RowToCHNativeColumnarExec.apply)
+ toRow(CHColumnarToRowExec.apply)
+ }
}
diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHAggregateGroupLimitExecTransformer.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHAggregateGroupLimitExecTransformer.scala
new file mode 100644
index 000000000000..83bb33bfa225
--- /dev/null
+++ b/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHAggregateGroupLimitExecTransformer.scala
@@ -0,0 +1,174 @@
+/*
+ * 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.
+ */
+package org.apache.gluten.execution
+
+import org.apache.gluten.backendsapi.BackendsApiManager
+import org.apache.gluten.exception.GlutenNotSupportException
+import org.apache.gluten.expression._
+import org.apache.gluten.expression.{ConverterUtils, ExpressionConverter}
+import org.apache.gluten.extension.ValidationResult
+import org.apache.gluten.metrics.MetricsUpdater
+import org.apache.gluten.substrait.`type`.TypeBuilder
+import org.apache.gluten.substrait.SubstraitContext
+import org.apache.gluten.substrait.extensions.ExtensionBuilder
+import org.apache.gluten.substrait.rel.{RelBuilder, RelNode}
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.plans.physical.{AllTuples, ClusteredDistribution, Distribution, Partitioning}
+import org.apache.spark.sql.execution.SparkPlan
+
+import com.google.protobuf.StringValue
+import io.substrait.proto.SortField
+
+import scala.collection.JavaConverters._
+
+case class CHAggregateGroupLimitExecTransformer(
+ partitionSpec: Seq[Expression],
+ orderSpec: Seq[SortOrder],
+ rankLikeFunction: Expression,
+ resultAttributes: Seq[Attribute],
+ limit: Int,
+ child: SparkPlan)
+ extends UnaryTransformSupport {
+
+ @transient override lazy val metrics =
+ BackendsApiManager.getMetricsApiInstance.genWindowTransformerMetrics(sparkContext)
+
+ override protected def withNewChildInternal(newChild: SparkPlan): SparkPlan =
+ copy(child = newChild)
+
+ override def metricsUpdater(): MetricsUpdater =
+ BackendsApiManager.getMetricsApiInstance.genWindowTransformerMetricsUpdater(metrics)
+
+ override def output: Seq[Attribute] = resultAttributes
+
+ override def requiredChildDistribution: Seq[Distribution] = {
+ if (partitionSpec.isEmpty) {
+ // Only show warning when the number of bytes is larger than 100 MiB?
+ logWarning(
+ "No Partition Defined for Window operation! Moving all data to a single "
+ + "partition, this can cause serious performance degradation.")
+ AllTuples :: Nil
+ } else ClusteredDistribution(partitionSpec) :: Nil
+ }
+
+ override def requiredChildOrdering: Seq[Seq[SortOrder]] = {
+ Seq(Nil)
+ }
+
+ override def outputOrdering: Seq[SortOrder] = {
+ if (requiredChildOrdering.forall(_.isEmpty)) {
+ Nil
+ } else {
+ child.outputOrdering
+ }
+ }
+
+ override def outputPartitioning: Partitioning = child.outputPartitioning
+
+ def getWindowGroupLimitRel(
+ context: SubstraitContext,
+ originalInputAttributes: Seq[Attribute],
+ operatorId: Long,
+ input: RelNode,
+ validation: Boolean): RelNode = {
+ val args = context.registeredFunction
+ // Partition By Expressions
+ val partitionsExpressions = partitionSpec
+ .map(
+ ExpressionConverter
+ .replaceWithExpressionTransformer(_, attributeSeq = child.output)
+ .doTransform(args))
+ .asJava
+
+ // Sort By Expressions
+ val sortFieldList =
+ orderSpec.map {
+ order =>
+ val builder = SortField.newBuilder()
+ val exprNode = ExpressionConverter
+ .replaceWithExpressionTransformer(order.child, attributeSeq = child.output)
+ .doTransform(args)
+ builder.setExpr(exprNode.toProtobuf)
+ builder.setDirectionValue(SortExecTransformer.transformSortDirection(order))
+ builder.build()
+ }.asJava
+ if (!validation) {
+ val windowFunction = rankLikeFunction match {
+ case _: RowNumber => ExpressionNames.ROW_NUMBER
+ case _: Rank => ExpressionNames.RANK
+ case _: DenseRank => ExpressionNames.DENSE_RANK
+ case _ => throw new GlutenNotSupportException(s"Unknow window function $rankLikeFunction")
+ }
+ val parametersStr = new StringBuffer("WindowGroupLimitParameters:")
+ parametersStr
+ .append("window_function=")
+ .append(windowFunction)
+ .append("\n")
+ .append("is_aggregate_group_limit=true\n")
+ val message = StringValue.newBuilder().setValue(parametersStr.toString).build()
+ val extensionNode = ExtensionBuilder.makeAdvancedExtension(
+ BackendsApiManager.getTransformerApiInstance.packPBMessage(message),
+ null)
+ RelBuilder.makeWindowGroupLimitRel(
+ input,
+ partitionsExpressions,
+ sortFieldList,
+ limit,
+ extensionNode,
+ context,
+ operatorId)
+ } else {
+ // Use a extension node to send the input types through Substrait plan for validation.
+ val inputTypeNodeList = originalInputAttributes
+ .map(attr => ConverterUtils.getTypeNode(attr.dataType, attr.nullable))
+ .asJava
+ val extensionNode = ExtensionBuilder.makeAdvancedExtension(
+ BackendsApiManager.getTransformerApiInstance.packPBMessage(
+ TypeBuilder.makeStruct(false, inputTypeNodeList).toProtobuf))
+
+ RelBuilder.makeWindowGroupLimitRel(
+ input,
+ partitionsExpressions,
+ sortFieldList,
+ limit,
+ extensionNode,
+ context,
+ operatorId)
+ }
+ }
+
+ override protected def doValidateInternal(): ValidationResult = {
+ val substraitContext = new SubstraitContext
+ val operatorId = substraitContext.nextOperatorId(this.nodeName)
+
+ val relNode =
+ getWindowGroupLimitRel(substraitContext, child.output, operatorId, null, validation = true)
+
+ doNativeValidation(substraitContext, relNode)
+ }
+
+ override protected def doTransform(context: SubstraitContext): TransformContext = {
+ val childCtx = child.asInstanceOf[TransformSupport].transform(context)
+ val operatorId = context.nextOperatorId(this.nodeName)
+
+ val currRel =
+ getWindowGroupLimitRel(context, child.output, operatorId, childCtx.root, validation = false)
+ assert(currRel != null, "Window Group Limit Rel should be valid")
+ TransformContext(output, currRel)
+ }
+}
diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHHashAggregateExecTransformer.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHHashAggregateExecTransformer.scala
index f5e64330cd15..48b0d7336103 100644
--- a/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHHashAggregateExecTransformer.scala
+++ b/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHHashAggregateExecTransformer.scala
@@ -43,6 +43,45 @@ import scala.collection.JavaConverters._
import scala.collection.mutable.ListBuffer
object CHHashAggregateExecTransformer {
+ // The result attributes of aggregate expressions from vanilla may be different from CH native.
+ // For example, the result attributes of `avg(x)` are `sum(x)` and `count(x)`. This could bring
+ // some unexpected issues. So we need to make the result attributes consistent with CH native.
+ def getCHAggregateResultExpressions(
+ groupingExpressions: Seq[NamedExpression],
+ aggregateExpressions: Seq[AggregateExpression],
+ resultExpressions: Seq[NamedExpression]): Seq[NamedExpression] = {
+ var adjustedResultExpressions = resultExpressions.slice(0, groupingExpressions.length)
+ var resultExpressionIndex = groupingExpressions.length
+ adjustedResultExpressions ++ aggregateExpressions.flatMap {
+ aggExpr =>
+ aggExpr.mode match {
+ case Partial | PartialMerge =>
+ // For partial aggregate, the size of the result expressions of an aggregate expression
+ // is the same as aggBufferAttributes' length
+ val aggBufferAttributesCount = aggExpr.aggregateFunction.aggBufferAttributes.length
+ aggExpr.aggregateFunction match {
+ case avg: Average =>
+ val res = Seq(aggExpr.resultAttribute)
+ resultExpressionIndex += aggBufferAttributesCount
+ res
+ case sum: Sum if (sum.dataType.isInstanceOf[DecimalType]) =>
+ val res = Seq(resultExpressions(resultExpressionIndex))
+ resultExpressionIndex += aggBufferAttributesCount
+ res
+ case _ =>
+ val res = resultExpressions
+ .slice(resultExpressionIndex, resultExpressionIndex + aggBufferAttributesCount)
+ resultExpressionIndex += aggBufferAttributesCount
+ res
+ }
+ case _ =>
+ val res = Seq(resultExpressions(resultExpressionIndex))
+ resultExpressionIndex += 1
+ res
+ }
+ }
+ }
+
def getAggregateResultAttributes(
groupingExpressions: Seq[NamedExpression],
aggregateExpressions: Seq[AggregateExpression]): Seq[Attribute] = {
@@ -170,7 +209,7 @@ case class CHHashAggregateExecTransformer(
RelBuilder.makeReadRelForInputIteratorWithoutRegister(typeList, nameList, context)
(getAggRel(context, operatorId, aggParams, readRel), inputAttrs, outputAttrs)
}
- TransformContext(inputAttributes, outputAttributes, relNode)
+ TransformContext(outputAttributes, relNode)
}
override def getAggRel(
diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHHashJoinExecTransformer.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHHashJoinExecTransformer.scala
index 43f19c30e28f..6bf2248ebe3a 100644
--- a/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHHashJoinExecTransformer.scala
+++ b/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHHashJoinExecTransformer.scala
@@ -58,16 +58,23 @@ object JoinTypeTransform {
} else {
JoinRel.JoinType.JOIN_TYPE_RIGHT
}
- case LeftSemi | ExistenceJoin(_) =>
+ case LeftSemi =>
if (!buildRight) {
- throw new IllegalArgumentException("LeftSemi join should not switch children")
+ JoinRel.JoinType.JOIN_TYPE_RIGHT_SEMI
+ } else {
+ JoinRel.JoinType.JOIN_TYPE_LEFT_SEMI
}
- JoinRel.JoinType.JOIN_TYPE_LEFT_SEMI
case LeftAnti =>
if (!buildRight) {
- throw new IllegalArgumentException("LeftAnti join should not switch children")
+ JoinRel.JoinType.JOIN_TYPE_RIGHT_ANTI
+ } else {
+ JoinRel.JoinType.JOIN_TYPE_LEFT_ANTI
+ }
+ case ExistenceJoin(_) =>
+ if (!buildRight) {
+ throw new IllegalArgumentException("Existence join should not switch children")
}
- JoinRel.JoinType.JOIN_TYPE_LEFT_ANTI
+ JoinRel.JoinType.JOIN_TYPE_LEFT_SEMI
case _ =>
// TODO: Support cross join with Cross Rel
JoinRel.JoinType.UNRECOGNIZED
diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHWindowGroupLimitExecTransformer.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHWindowGroupLimitExecTransformer.scala
index c2648f29ec4c..793d733abf96 100644
--- a/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHWindowGroupLimitExecTransformer.scala
+++ b/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHWindowGroupLimitExecTransformer.scala
@@ -182,6 +182,6 @@ case class CHWindowGroupLimitExecTransformer(
val currRel =
getWindowGroupLimitRel(context, child.output, operatorId, childCtx.root, validation = false)
assert(currRel != null, "Window Group Limit Rel should be valid")
- TransformContext(childCtx.outputAttributes, output, currRel)
+ TransformContext(output, currRel)
}
}
diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/extension/ConvertWindowToAggregate.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/ConvertWindowToAggregate.scala
new file mode 100644
index 000000000000..ad2b22ba6cc6
--- /dev/null
+++ b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/ConvertWindowToAggregate.scala
@@ -0,0 +1,138 @@
+/*
+ * 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.
+ */
+package org.apache.gluten.extension
+
+import org.apache.gluten.backendsapi.clickhouse.CHBackendSettings
+import org.apache.gluten.exception.GlutenException
+import org.apache.gluten.execution._
+import org.apache.gluten.expression.WindowFunctionsBuilder
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.rules.Rule
+// import org.apache.spark.sql.execution._
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.types._
+
+// When to find the first rows of partitions by window function, we can convert it to aggregate
+// function.
+case class ConverRowNumbertWindowToAggregateRule(spark: SparkSession)
+ extends Rule[SparkPlan]
+ with Logging {
+
+ override def apply(plan: SparkPlan): SparkPlan = {
+ if (!CHBackendSettings.enableConvertWindowGroupLimitToAggregate) {
+ return plan
+ }
+ plan.transformUp {
+ case filter @ FilterExecTransformer(
+ condition,
+ window @ WindowExecTransformer(
+ windowExpressions,
+ partitionSpec,
+ orderSpec,
+ sort @ SortExecTransformer(_, _, _, _))) =>
+ if (
+ !isSupportedWindowFunction(windowExpressions) || !isTopKLimitFilter(
+ condition,
+ windowExpressions(0))
+ ) {
+ logDebug(
+ s"xxx Not Supported case for converting window to aggregate. is topk limit: " +
+ s"${isTopKLimitFilter(condition, windowExpressions(0))}. is supported window " +
+ s"function: ${isSupportedWindowFunction(windowExpressions)}")
+ filter
+ } else {
+ val limit = getLimit(condition.asInstanceOf[BinaryComparison])
+ if (limit < 1 || limit > 100) {
+ filter
+ } else {
+ val groupLimit = CHAggregateGroupLimitExecTransformer(
+ partitionSpec,
+ orderSpec,
+ extractWindowFunction(windowExpressions(0)),
+ sort.child.output ++ Seq(windowExpressions(0).toAttribute),
+ limit,
+ sort.child
+ )
+ groupLimit
+ }
+ }
+ }
+ }
+
+ def getLimit(e: BinaryComparison): Int = {
+ e match {
+ case _: EqualTo => evalIntLiteral(e.right).get
+ case _: LessThanOrEqual => evalIntLiteral(e.right).get
+ case _: LessThan => evalIntLiteral(e.right).get - 1
+ case _ => throw new GlutenException(s"Unsupported comparison: $e")
+ }
+ }
+
+ def evalIntLiteral(expr: Expression): Option[Int] = {
+ expr match {
+ case int @ Literal(value, IntegerType) => Some(value.asInstanceOf[Int])
+ case long @ Literal(value, LongType) => Some(value.asInstanceOf[Long].toInt)
+ case _ => None
+ }
+ }
+
+ def extractWindowFunction(windowExpression: NamedExpression): Expression = {
+ val aliasExpr = windowExpression.asInstanceOf[Alias]
+ WindowFunctionsBuilder.extractWindowExpression(aliasExpr.child).windowFunction
+ }
+
+ def isSupportedWindowFunction(windowExpressions: Seq[NamedExpression]): Boolean = {
+ if (windowExpressions.length != 1) {
+ return false
+ }
+ val windowFunction = extractWindowFunction(windowExpressions(0))
+ windowFunction match {
+ case _: RowNumber => true
+ case _ => false
+ }
+ }
+
+ // If the filter condition is a constant limit condition, return the limit value.
+ def isTopKLimitFilter(condition: Expression, windowExpression: NamedExpression): Boolean = {
+ def isWindowFunctionResult(
+ condition: Expression,
+ windowExpression: NamedExpression): Boolean = {
+ condition match {
+ case attr: Attribute =>
+ attr.semanticEquals(windowExpression.toAttribute)
+ case _ => false
+ }
+ }
+ if (!condition.isInstanceOf[BinaryComparison]) {
+ return false
+ }
+ val binaryComparison = condition.asInstanceOf[BinaryComparison]
+ val constLimit = evalIntLiteral(binaryComparison.right)
+ if (!isWindowFunctionResult(binaryComparison.left, windowExpression) || !constLimit.isDefined) {
+ return false
+ }
+ binaryComparison match {
+ case _: EqualTo => constLimit.get == 1
+ case _: LessThanOrEqual | _: LessThan => true
+ case _ => false
+ }
+ }
+
+}
diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/extension/ExtendedColumnPruning.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/ExtendedColumnPruning.scala
new file mode 100644
index 000000000000..f2a0a549bc7e
--- /dev/null
+++ b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/ExtendedColumnPruning.scala
@@ -0,0 +1,366 @@
+/*
+ * 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.
+ */
+package org.apache.gluten.extension
+
+import org.apache.gluten.GlutenConfig
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateFunction
+import org.apache.spark.sql.catalyst.optimizer.GeneratorNestedColumnAliasing.canPruneGenerator
+import org.apache.spark.sql.catalyst.optimizer.NestedColumnAliasing
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.AlwaysProcess
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+
+import scala.collection.mutable
+
+object ExtendedGeneratorNestedColumnAliasing {
+ def unapply(plan: LogicalPlan): Option[LogicalPlan] = plan match {
+ case pj @ Project(projectList, f @ Filter(condition, g: Generate))
+ if canPruneGenerator(g.generator) &&
+ GlutenConfig.getConf.enableExtendedColumnPruning &&
+ (SQLConf.get.nestedPruningOnExpressions || SQLConf.get.nestedSchemaPruningEnabled) =>
+ val attrToExtractValues =
+ getAttributeToExtractValues(projectList ++ g.generator.children :+ condition, Seq.empty)
+ if (attrToExtractValues.isEmpty) {
+ return None
+ }
+
+ val generatorOutputSet = AttributeSet(g.qualifiedGeneratorOutput)
+ var (attrToExtractValuesOnGenerator, attrToExtractValuesNotOnGenerator) =
+ attrToExtractValues.partition {
+ case (attr, _) =>
+ attr.references.subsetOf(generatorOutputSet)
+ }
+
+ val pushedThrough = rewritePlanWithAliases(pj, attrToExtractValuesNotOnGenerator)
+
+ // We cannot push through if the child of generator is `MapType`.
+ g.generator.children.head.dataType match {
+ case _: MapType => return Some(pushedThrough)
+ case _ =>
+ }
+
+ if (!g.generator.isInstanceOf[ExplodeBase]) {
+ return Some(pushedThrough)
+ }
+
+ // In spark3.2, we could not reuse [[NestedColumnAliasing.getAttributeToExtractValues]]
+ // which only accepts 2 arguments. Instead we redefine it in current file to avoid moving
+ // this rule to gluten-shims
+ attrToExtractValuesOnGenerator = getAttributeToExtractValues(
+ attrToExtractValuesOnGenerator.flatMap(_._2).toSeq,
+ Seq.empty,
+ collectNestedGetStructFields)
+
+ val nestedFieldsOnGenerator = attrToExtractValuesOnGenerator.values.flatten.toSet
+ if (nestedFieldsOnGenerator.isEmpty) {
+ return Some(pushedThrough)
+ }
+
+ // Multiple or single nested column accessors.
+ // E.g. df.select(explode($"items").as("item")).select($"item.a", $"item.b")
+ pushedThrough match {
+ case p2 @ Project(_, f2 @ Filter(_, g2: Generate)) =>
+ val nestedFieldsOnGeneratorSeq = nestedFieldsOnGenerator.toSeq
+ val nestedFieldToOrdinal = nestedFieldsOnGeneratorSeq.zipWithIndex.toMap
+ val rewrittenG = g2.transformExpressions {
+ case e: ExplodeBase =>
+ val extractors = nestedFieldsOnGeneratorSeq.map(replaceGenerator(e, _))
+ val names = extractors.map {
+ case g: GetStructField => Literal(g.extractFieldName)
+ case ga: GetArrayStructFields => Literal(ga.field.name)
+ case other =>
+ throw new IllegalStateException(
+ s"Unreasonable extractor " +
+ "after replaceGenerator: $other")
+ }
+ val zippedArray = ArraysZip(extractors, names)
+ e.withNewChildren(Seq(zippedArray))
+ }
+ // As we change the child of the generator, its output data type must be updated.
+ val updatedGeneratorOutput = rewrittenG.generatorOutput
+ .zip(
+ rewrittenG.generator.elementSchema.map(
+ f => AttributeReference(f.name, f.dataType, f.nullable, f.metadata)()))
+ .map {
+ case (oldAttr, newAttr) =>
+ newAttr.withExprId(oldAttr.exprId).withName(oldAttr.name)
+ }
+ assert(
+ updatedGeneratorOutput.length == rewrittenG.generatorOutput.length,
+ "Updated generator output must have the same length " +
+ "with original generator output."
+ )
+ val updatedGenerate = rewrittenG.copy(generatorOutput = updatedGeneratorOutput)
+
+ // Replace nested column accessor with generator output.
+ val attrExprIdsOnGenerator = attrToExtractValuesOnGenerator.keys.map(_.exprId).toSet
+ val updatedFilter = f2.withNewChildren(Seq(updatedGenerate)).transformExpressions {
+ case f: GetStructField if nestedFieldsOnGenerator.contains(f) =>
+ replaceGetStructField(
+ f,
+ updatedGenerate.output,
+ attrExprIdsOnGenerator,
+ nestedFieldToOrdinal)
+ }
+
+ val updatedProject = p2.withNewChildren(Seq(updatedFilter)).transformExpressions {
+ case f: GetStructField if nestedFieldsOnGenerator.contains(f) =>
+ replaceGetStructField(
+ f,
+ updatedFilter.output,
+ attrExprIdsOnGenerator,
+ nestedFieldToOrdinal)
+ }
+
+ Some(updatedProject)
+ case other =>
+ throw new IllegalStateException(s"Unreasonable plan after optimization: $other")
+ }
+ case _ =>
+ None
+ }
+
+ /**
+ * Returns two types of expressions:
+ * - Root references that are individually accessed
+ * - [[GetStructField]] or [[GetArrayStructFields]] on top of other [[ExtractValue]]s or special
+ * expressions.
+ */
+ private def collectRootReferenceAndExtractValue(e: Expression): Seq[Expression] = e match {
+ case _: AttributeReference => Seq(e)
+ case GetStructField(_: ExtractValue | _: AttributeReference, _, _) => Seq(e)
+ case GetArrayStructFields(
+ _: MapValues | _: MapKeys | _: ExtractValue | _: AttributeReference,
+ _,
+ _,
+ _,
+ _) =>
+ Seq(e)
+ case es if es.children.nonEmpty => es.children.flatMap(collectRootReferenceAndExtractValue)
+ case _ => Seq.empty
+ }
+
+ /**
+ * Creates a map from root [[Attribute]]s to non-redundant nested [[ExtractValue]]s. Nested field
+ * accessors of `exclusiveAttrs` are not considered in nested fields aliasing.
+ */
+ private def getAttributeToExtractValues(
+ exprList: Seq[Expression],
+ exclusiveAttrs: Seq[Attribute],
+ extractor: (Expression) => Seq[Expression] = collectRootReferenceAndExtractValue)
+ : Map[Attribute, Seq[ExtractValue]] = {
+
+ val nestedFieldReferences = new mutable.ArrayBuffer[ExtractValue]()
+ val otherRootReferences = new mutable.ArrayBuffer[AttributeReference]()
+ exprList.foreach {
+ e =>
+ extractor(e).foreach {
+ // we can not alias the attr from lambda variable whose expr id is not available
+ case ev: ExtractValue if ev.find(_.isInstanceOf[NamedLambdaVariable]).isEmpty =>
+ if (ev.references.size == 1) {
+ nestedFieldReferences.append(ev)
+ }
+ case ar: AttributeReference => otherRootReferences.append(ar)
+ case _ => // ignore
+ }
+ }
+ val exclusiveAttrSet = AttributeSet(exclusiveAttrs ++ otherRootReferences)
+
+ // Remove cosmetic variations when we group extractors by their references
+ nestedFieldReferences
+ .filter(!_.references.subsetOf(exclusiveAttrSet))
+ .groupBy(_.references.head.canonicalized.asInstanceOf[Attribute])
+ .flatMap {
+ case (attr: Attribute, nestedFields: collection.Seq[ExtractValue]) =>
+ // Check if `ExtractValue` expressions contain any aggregate functions in their tree.
+ // Those that do should not have an alias generated as it can lead to pushing the
+ // aggregate down into a projection.
+ def containsAggregateFunction(ev: ExtractValue): Boolean =
+ ev.find(_.isInstanceOf[AggregateFunction]).isDefined
+
+ // Remove redundant [[ExtractValue]]s if they share the same parent nest field.
+ // For example, when `a.b` and `a.b.c` are in project list, we only need to alias `a.b`.
+ // Because `a.b` requires all of the inner fields of `b`, we cannot prune `a.b.c`.
+ val dedupNestedFields = nestedFields
+ .filter {
+ // See [[collectExtractValue]]: we only need to deal with [[GetArrayStructFields]] and
+ // [[GetStructField]]
+ case e @ (_: GetStructField | _: GetArrayStructFields) =>
+ val child = e.children.head
+ nestedFields.forall(f => child.find(_.semanticEquals(f)).isEmpty)
+ case _ => true
+ }
+ .distinct
+ // Discard [[ExtractValue]]s that contain aggregate functions.
+ .filterNot(containsAggregateFunction)
+
+ // If all nested fields of `attr` are used, we don't need to introduce new aliases.
+ // By default, the [[ColumnPruning]] rule uses `attr` already.
+ // Note that we need to remove cosmetic variations first, so we only count a
+ // nested field once.
+ val numUsedNestedFields = dedupNestedFields
+ .map(_.canonicalized)
+ .distinct
+ .map(nestedField => totalFieldNum(nestedField.dataType))
+ .sum
+ if (dedupNestedFields.nonEmpty && numUsedNestedFields < totalFieldNum(attr.dataType)) {
+ Some((attr, dedupNestedFields.toSeq))
+ } else {
+ None
+ }
+ }
+ }
+
+ /**
+ * Return total number of fields of this type. This is used as a threshold to use nested column
+ * pruning. It's okay to underestimate. If the number of reference is bigger than this, the parent
+ * reference is used instead of nested field references.
+ */
+ private def totalFieldNum(dataType: DataType): Int = dataType match {
+ case StructType(fields) => fields.map(f => totalFieldNum(f.dataType)).sum
+ case ArrayType(elementType, _) => totalFieldNum(elementType)
+ case MapType(keyType, valueType, _) => totalFieldNum(keyType) + totalFieldNum(valueType)
+ case _ => 1 // UDT and others
+ }
+
+ private def replaceGetStructField(
+ g: GetStructField,
+ input: Seq[Attribute],
+ attrExprIdsOnGenerator: Set[ExprId],
+ nestedFieldToOrdinal: Map[ExtractValue, Int]): Expression = {
+ val attr = input.find(a => attrExprIdsOnGenerator.contains(a.exprId))
+ attr match {
+ case Some(a) =>
+ val ordinal = nestedFieldToOrdinal(g)
+ GetStructField(a, ordinal, g.name)
+ case None => g
+ }
+ }
+
+ /** Replace the reference attribute of extractor expression with generator input. */
+ private def replaceGenerator(generator: ExplodeBase, expr: Expression): Expression = {
+ expr match {
+ case a: Attribute if expr.references.contains(a) =>
+ generator.child
+ case g: GetStructField =>
+ // We cannot simply do a transformUp instead because if we replace the attribute
+ // `extractFieldName` could cause `ClassCastException` error. We need to get the
+ // field name before replacing down the attribute/other extractor.
+ val fieldName = g.extractFieldName
+ val newChild = replaceGenerator(generator, g.child)
+ ExtractValue(newChild, Literal(fieldName), SQLConf.get.resolver)
+ case other =>
+ other.mapChildren(replaceGenerator(generator, _))
+ }
+ }
+
+ // This function collects all GetStructField*(attribute) from the passed in expression.
+ // GetStructField* means arbitrary levels of nesting.
+ private def collectNestedGetStructFields(e: Expression): Seq[Expression] = {
+ // The helper function returns a tuple of
+ // (nested GetStructField including the current level, all other nested GetStructField)
+ def helper(e: Expression): (Seq[Expression], Seq[Expression]) = e match {
+ case _: AttributeReference => (Seq(e), Seq.empty)
+ case gsf: GetStructField =>
+ val child_res = helper(gsf.child)
+ (child_res._1.map(p => gsf.withNewChildren(Seq(p))), child_res._2)
+ case other =>
+ val child_res = other.children.map(helper)
+ val child_res_combined = (child_res.flatMap(_._1), child_res.flatMap(_._2))
+ (Seq.empty, child_res_combined._1 ++ child_res_combined._2)
+ }
+
+ val res = helper(e)
+ (res._1 ++ res._2).filterNot(_.isInstanceOf[Attribute])
+ }
+
+ private def rewritePlanWithAliases(
+ plan: LogicalPlan,
+ attributeToExtractValues: Map[Attribute, Seq[ExtractValue]]): LogicalPlan = {
+ val attributeToExtractValuesAndAliases =
+ attributeToExtractValues.map {
+ case (attr, evSeq) =>
+ val evAliasSeq = evSeq.map {
+ ev =>
+ val fieldName = ev match {
+ case g: GetStructField => g.extractFieldName
+ case g: GetArrayStructFields => g.field.name
+ }
+ ev -> Alias(ev, s"_extract_$fieldName")()
+ }
+
+ attr -> evAliasSeq
+ }
+
+ val nestedFieldToAlias = attributeToExtractValuesAndAliases.values.flatten.map {
+ case (field, alias) => field.canonicalized -> alias
+ }.toMap
+
+ // A reference attribute can have multiple aliases for nested fields.
+ val attrToAliases =
+ AttributeMap(attributeToExtractValuesAndAliases.mapValues(_.map(_._2)).toSeq)
+
+ plan match {
+ // Project(Filter(Generate))
+ case Project(projectList, f @ Filter(condition, g: Generate)) =>
+ val newProjectList = NestedColumnAliasing.getNewProjectList(projectList, nestedFieldToAlias)
+ val newCondition = getNewExpression(condition, nestedFieldToAlias)
+ val newGenerator = getNewExpression(g.generator, nestedFieldToAlias).asInstanceOf[Generator]
+
+ val tmpG = NestedColumnAliasing
+ .replaceWithAliases(g, nestedFieldToAlias, attrToAliases)
+ .asInstanceOf[Generate]
+ val newG = Generate(
+ newGenerator,
+ tmpG.unrequiredChildIndex,
+ tmpG.outer,
+ tmpG.qualifier,
+ tmpG.generatorOutput,
+ tmpG.children.head)
+ val newF = Filter(newCondition, newG)
+ val newP = Project(newProjectList, newF)
+ newP
+ case _ => plan
+ }
+ }
+
+ private def getNewExpression(
+ expr: Expression,
+ nestedFieldToAlias: Map[Expression, Alias]): Expression = {
+ expr.transform {
+ case f: ExtractValue if nestedFieldToAlias.contains(f.canonicalized) =>
+ nestedFieldToAlias(f.canonicalized).toAttribute
+ }
+ }
+}
+
+// ExtendedColumnPruning process Project(Filter(Generate)),
+// which is ignored by vanilla spark in optimization rule: ColumnPruning
+class ExtendedColumnPruning(spark: SparkSession) extends Rule[LogicalPlan] with Logging {
+
+ override def apply(plan: LogicalPlan): LogicalPlan =
+ plan.transformWithPruning(AlwaysProcess.fn) {
+ case ExtendedGeneratorNestedColumnAliasing(rewrittenPlan) => rewrittenPlan
+ case p => p
+ }
+}
diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/extension/FallbackBroadcastHashJoinRules.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/FallbackBroadcastHashJoinRules.scala
index ec465a3c1506..6a788617a6c4 100644
--- a/backends-clickhouse/src/main/scala/org/apache/gluten/extension/FallbackBroadcastHashJoinRules.scala
+++ b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/FallbackBroadcastHashJoinRules.scala
@@ -18,8 +18,7 @@ package org.apache.gluten.extension
import org.apache.gluten.GlutenConfig
import org.apache.gluten.backendsapi.BackendsApiManager
-import org.apache.gluten.extension.columnar._
-import org.apache.gluten.extension.columnar.FallbackTags.EncodeFallbackTagImplicits
+import org.apache.gluten.extension.columnar.FallbackTags
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight}
@@ -38,7 +37,7 @@ import scala.util.control.Breaks.{break, breakable}
// queryStagePrepRules.
case class FallbackBroadcastHashJoinPrepQueryStage(session: SparkSession) extends Rule[SparkPlan] {
override def apply(plan: SparkPlan): SparkPlan = {
- val columnarConf: GlutenConfig = GlutenConfig.getConf
+ val glutenConf: GlutenConfig = GlutenConfig.getConf
plan.foreach {
case bhj: BroadcastHashJoinExec =>
val buildSidePlan = bhj.buildSide match {
@@ -53,8 +52,8 @@ case class FallbackBroadcastHashJoinPrepQueryStage(session: SparkSession) extend
case Some(exchange @ BroadcastExchangeExec(mode, child)) =>
val isTransformable =
if (
- !columnarConf.enableColumnarBroadcastExchange ||
- !columnarConf.enableColumnarBroadcastJoin
+ !glutenConf.enableColumnarBroadcastExchange ||
+ !glutenConf.enableColumnarBroadcastJoin
) {
ValidationResult.failed(
"columnar broadcast exchange is disabled or " +
@@ -117,19 +116,19 @@ case class FallbackBroadcastHashJoinPrepQueryStage(session: SparkSession) extend
if (FallbackTags.nonEmpty(bnlj)) {
ValidationResult.failed("broadcast join is already tagged as not transformable")
} else {
- val transformer = BackendsApiManager.getSparkPlanExecApiInstance
+ val bnljTransformer = BackendsApiManager.getSparkPlanExecApiInstance
.genBroadcastNestedLoopJoinExecTransformer(
bnlj.left,
bnlj.right,
bnlj.buildSide,
bnlj.joinType,
bnlj.condition)
- val isTransformable = transformer.doValidate()
- if (isTransformable.ok()) {
+ val isBnljTransformable = bnljTransformer.doValidate()
+ if (isBnljTransformable.ok()) {
val exchangeTransformer = ColumnarBroadcastExchangeExec(mode, child)
exchangeTransformer.doValidate()
} else {
- isTransformable
+ isBnljTransformable
}
}
}
diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/extension/LazyAggregateExpandRule.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/LazyAggregateExpandRule.scala
new file mode 100644
index 000000000000..0f7752339707
--- /dev/null
+++ b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/LazyAggregateExpandRule.scala
@@ -0,0 +1,388 @@
+/*
+ * 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.
+ */
+package org.apache.gluten.extension
+
+import org.apache.gluten.backendsapi.clickhouse.CHBackendSettings
+import org.apache.gluten.execution._
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate._
+import org.apache.spark.sql.catalyst.plans.physical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.execution._
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.types._
+
+/*
+ * For aggregation with grouping sets, we need to expand the grouping sets
+ * to individual group by.
+ * 1. It need to make copies of the original data.
+ * 2. run the aggregation on the multi copied data.
+ * Both of these two are expensive.
+ *
+ * We could do this as following
+ * 1. Run the aggregation on full grouping keys.
+ * 2. Expand the aggregation result to the full grouping sets.
+ * 3. Run the aggregation on the expanded data.
+ *
+ * So the plan is transformed from
+ * expand -> partial aggregating -> shuffle -> final merge aggregating
+ * to
+ * partial aggregating -> expand -> shuffle -> final merge aggregating
+ *
+ * Notice:
+ * If the aggregation involves distinct, we can't do this optimization.
+ */
+
+case class LazyAggregateExpandRule(session: SparkSession) extends Rule[SparkPlan] with Logging {
+ override def apply(plan: SparkPlan): SparkPlan = {
+ logDebug(s"xxx enable lazy aggregate expand: ${CHBackendSettings.enableLazyAggregateExpand}")
+ if (!CHBackendSettings.enableLazyAggregateExpand) {
+ return plan
+ }
+ plan.transformUp {
+ case shuffle @ ColumnarShuffleExchangeExec(
+ HashPartitioning(hashExpressions, _),
+ CHHashAggregateExecTransformer(
+ _,
+ groupingExpressions,
+ aggregateExpressions,
+ _,
+ _,
+ resultExpressions,
+ ExpandExecTransformer(projections, output, child)),
+ _,
+ _,
+ _
+ ) =>
+ logDebug(s"xxx match plan:$shuffle")
+ val partialAggregate = shuffle.child.asInstanceOf[CHHashAggregateExecTransformer]
+ val expand = partialAggregate.child.asInstanceOf[ExpandExecTransformer]
+ logDebug(
+ s"xxx partialAggregate: groupingExpressions:" +
+ s"${partialAggregate.groupingExpressions}\n" +
+ s"aggregateAttributes:${partialAggregate.aggregateAttributes}\n" +
+ s"aggregateExpressions:${partialAggregate.aggregateExpressions}\n" +
+ s"resultExpressions:${partialAggregate.resultExpressions}")
+ if (doValidation(partialAggregate, expand, shuffle)) {
+
+ val attributesToReplace = buildReplaceAttributeMap(expand)
+ logDebug(s"xxx attributesToReplace: $attributesToReplace")
+
+ val newPartialAggregate = buildAheadAggregateExec(
+ partialAggregate,
+ expand,
+ attributesToReplace
+ )
+
+ val newExpand = buildPostExpandExec(
+ expand,
+ partialAggregate,
+ newPartialAggregate,
+ attributesToReplace
+ )
+
+ val newShuffle = shuffle.copy(child = newExpand)
+ logDebug(s"xxx new plan: $newShuffle")
+ newShuffle
+ } else {
+ shuffle
+ }
+ case shuffle @ ColumnarShuffleExchangeExec(
+ HashPartitioning(hashExpressions, _),
+ CHHashAggregateExecTransformer(
+ _,
+ groupingExpressions,
+ aggregateExpressions,
+ _,
+ _,
+ resultExpressions,
+ FilterExecTransformer(_, ExpandExecTransformer(projections, output, child))),
+ _,
+ _,
+ _
+ ) =>
+ val partialAggregate = shuffle.child.asInstanceOf[CHHashAggregateExecTransformer]
+ val filter = partialAggregate.child.asInstanceOf[FilterExecTransformer]
+ val expand = filter.child.asInstanceOf[ExpandExecTransformer]
+ logDebug(
+ s"xxx partialAggregate: groupingExpressions:" +
+ s"${partialAggregate.groupingExpressions}\n" +
+ s"aggregateAttributes:${partialAggregate.aggregateAttributes}\n" +
+ s"aggregateExpressions:${partialAggregate.aggregateExpressions}\n" +
+ s"resultExpressions:${partialAggregate.resultExpressions}")
+ if (doValidation(partialAggregate, expand, shuffle)) {
+ val attributesToReplace = buildReplaceAttributeMap(expand)
+ logDebug(s"xxx attributesToReplace: $attributesToReplace")
+
+ val newPartialAggregate = buildAheadAggregateExec(
+ partialAggregate,
+ expand,
+ attributesToReplace
+ )
+
+ val newExpand = buildPostExpandExec(
+ expand,
+ partialAggregate,
+ newPartialAggregate,
+ attributesToReplace
+ )
+
+ val newFilter = filter.copy(child = newExpand)
+
+ val newShuffle = shuffle.copy(child = newFilter)
+ logDebug(s"xxx new plan: $newShuffle")
+ newShuffle
+
+ } else {
+ shuffle
+ }
+ }
+ }
+
+ // Just enable for simple cases. Some of cases that are not supported:
+ // 1. select count(a),count(b), count(1), count(distinct(a)), count(distinct(b)) from values
+ // (1, null), (2,2) as data(a,b);
+ // 2. select n_name, count(distinct n_regionkey) as col1,
+ // count(distinct concat(n_regionkey, n_nationkey)) as col2 from
+ // nation group by n_name;
+ def doValidation(
+ aggregate: CHHashAggregateExecTransformer,
+ expand: ExpandExecTransformer,
+ shuffle: ColumnarShuffleExchangeExec): Boolean = {
+ // all grouping keys must be attribute references
+ val expandOutputAttributes = expand.child.output.toSet
+ if (
+ !aggregate.groupingExpressions.forall(
+ e => e.isInstanceOf[Attribute] || e.isInstanceOf[Literal])
+ ) {
+ logDebug(s"xxx Not all grouping expression are attribute references")
+ return false
+ }
+ // all shuffle keys must be attribute references
+ if (
+ !shuffle.outputPartitioning
+ .asInstanceOf[HashPartitioning]
+ .expressions
+ .forall(e => e.isInstanceOf[Attribute] || e.isInstanceOf[Literal])
+ ) {
+ logDebug(s"xxx Not all shuffle hash expression are attribute references")
+ return false
+ }
+
+ // 1. for safty, we don't enbale this optimization for all aggregate functions.
+ // 2. if any aggregate function uses attributes which is not from expand's child, we don't
+ // enable this
+ if (
+ !aggregate.aggregateExpressions.forall {
+ e =>
+ isValidAggregateFunction(e) &&
+ e.aggregateFunction.references.forall(
+ attr => expandOutputAttributes.find(_.semanticEquals(attr)).isDefined)
+ }
+ ) {
+ logDebug(s"xxx Some aggregate functions are not supported")
+ return false
+ }
+
+ // get the group id's position in the expand's output
+ val gidIndex = findGroupingIdIndex(expand)
+ gidIndex != -1
+ }
+
+ // group id column doesn't have a fixed position, so we need to find it.
+ def findGroupingIdIndex(expand: ExpandExecTransformer): Int = {
+ def isValidGroupIdColumn(e: Expression, gids: Set[Long]): Long = {
+ if (!e.isInstanceOf[Literal]) {
+ return -1
+ }
+ val literalValue = e.asInstanceOf[Literal].value
+ e.dataType match {
+ case _: LongType =>
+ if (gids.contains(literalValue.asInstanceOf[Long])) {
+ -1
+ } else {
+ literalValue.asInstanceOf[Long]
+ }
+ case _: IntegerType =>
+ if (gids.contains(literalValue.asInstanceOf[Int].toLong)) {
+ -1
+ } else {
+ literalValue.asInstanceOf[Int].toLong
+ }
+ case _ => -1
+ }
+ }
+
+ var groupIdIndexes = Seq[Int]()
+ for (col <- 0 until expand.output.length) {
+ val expandCol = expand.projections(0)(col)
+ // gids should be unique
+ var gids = Set[Long]()
+ if (isValidGroupIdColumn(expandCol, gids) != -1) {
+ if (
+ expand.projections.forall {
+ projection =>
+ val res = isValidGroupIdColumn(projection(col), gids)
+ gids += res
+ res != -1
+ }
+ ) {
+ groupIdIndexes +:= col
+ }
+ }
+ }
+ if (groupIdIndexes.length == 1) {
+ logDebug(s"xxx gid is at pos ${groupIdIndexes(0)}")
+ groupIdIndexes(0)
+ } else {
+ -1
+ }
+ }
+
+ // Some of aggregate functions' output columns are not consistent with the output of gluten.
+ // - average: in partial aggregation, the outputs are sum and count, but gluten only generates one
+ // column, avg.
+ // - sum: if the input's type is decimal, the output are sum and isEmpty, but gluten doesn't use
+ // the isEmpty column.
+ def isValidAggregateFunction(aggregateExpression: AggregateExpression): Boolean = {
+ if (aggregateExpression.filter.isDefined) {
+ return false
+ }
+ aggregateExpression.aggregateFunction match {
+ case _: Count => true
+ case _: Max => true
+ case _: Min => true
+ case _: Average => true
+ case _: Sum => true
+ case _ => false
+ }
+ }
+
+ def getReplaceAttribute(
+ toReplace: Attribute,
+ attributesToReplace: Map[Attribute, Attribute]): Attribute = {
+ val kv = attributesToReplace.find(kv => kv._1.semanticEquals(toReplace))
+ kv match {
+ case Some((_, v)) => v
+ case None => toReplace
+ }
+ }
+
+ def buildReplaceAttributeMap(expand: ExpandExecTransformer): Map[Attribute, Attribute] = {
+ var fullExpandProjection = Seq[Expression]()
+ for (i <- 0 until expand.projections(0).length) {
+ val attr = expand.projections.find(x => x(i).isInstanceOf[Attribute]) match {
+ case Some(projection) => projection(i).asInstanceOf[Attribute]
+ case None => null
+ }
+ fullExpandProjection = fullExpandProjection :+ attr
+ }
+
+ var attributeMap = Map[Attribute, Attribute]()
+ for (i <- 0 until expand.output.length) {
+ if (fullExpandProjection(i).isInstanceOf[Attribute]) {
+ attributeMap += (expand.output(i) -> fullExpandProjection(i).asInstanceOf[Attribute])
+ }
+ }
+ attributeMap
+ }
+
+ def buildPostExpandProjections(
+ originalExpandProjections: Seq[Seq[Expression]],
+ originalExpandOutput: Seq[Attribute],
+ newExpandOutput: Seq[Attribute]): Seq[Seq[Expression]] = {
+ val newExpandProjections = originalExpandProjections.map {
+ projection =>
+ newExpandOutput.map {
+ attr =>
+ val index = originalExpandOutput.indexWhere(_.semanticEquals(attr))
+ if (index != -1) {
+ projection(index)
+ } else {
+ attr
+ }
+ }
+ }
+ newExpandProjections
+ }
+
+ // 1. make expand's child be aggregate's child
+ // 2. replace the attributes in groupingExpressions and resultExpressions as needed
+ def buildAheadAggregateExec(
+ partialAggregate: CHHashAggregateExecTransformer,
+ expand: ExpandExecTransformer,
+ attributesToReplace: Map[Attribute, Attribute]): SparkPlan = {
+ val groupIdAttribute = expand.output(findGroupingIdIndex(expand))
+
+ // New grouping expressions should include the group id column
+ val groupingExpressions =
+ partialAggregate.groupingExpressions
+ .filter(
+ e =>
+ !e.toAttribute.semanticEquals(groupIdAttribute) &&
+ attributesToReplace.find(kv => kv._1.semanticEquals(e.toAttribute)).isDefined)
+ .map(e => getReplaceAttribute(e.toAttribute, attributesToReplace))
+ .distinct
+ logDebug(
+ s"xxx newGroupingExpresion: $groupingExpressions,\n" +
+ s"groupingExpressions: ${partialAggregate.groupingExpressions}")
+
+ // Remove group id column from result expressions
+ val groupingKeysCount =
+ partialAggregate.resultExpressions.length - partialAggregate.aggregateExpressions.length
+ var resultExpressions = partialAggregate.resultExpressions
+ .slice(0, groupingKeysCount)
+ .filter(
+ e =>
+ !e.toAttribute.semanticEquals(groupIdAttribute) &&
+ attributesToReplace.find(kv => kv._1.semanticEquals(e.toAttribute)).isDefined)
+ .map(e => getReplaceAttribute(e.toAttribute, attributesToReplace))
+ resultExpressions = resultExpressions ++ partialAggregate.resultExpressions
+ .slice(groupingKeysCount, partialAggregate.resultExpressions.length)
+ .map(e => getReplaceAttribute(e.toAttribute, attributesToReplace))
+ logDebug(
+ s"xxx newResultExpressions: $resultExpressions\n" +
+ s"resultExpressions:${partialAggregate.resultExpressions}")
+ partialAggregate.copy(
+ groupingExpressions = groupingExpressions,
+ resultExpressions = resultExpressions.distinct,
+ child = expand.child)
+ }
+
+ def buildPostExpandExec(
+ expand: ExpandExecTransformer,
+ partialAggregate: CHHashAggregateExecTransformer,
+ child: SparkPlan,
+ attributesToReplace: Map[Attribute, Attribute]): SparkPlan = {
+ // The output of the native plan is not completely consistent with Spark.
+ val aggregateOutput = partialAggregate.output
+ logDebug(s"xxx aggregateResultAttributes: ${partialAggregate.aggregateResultAttributes}")
+ logDebug(s"xxx aggregateOutput: $aggregateOutput")
+
+ val expandProjections = buildPostExpandProjections(
+ expand.projections,
+ expand.output,
+ aggregateOutput
+ )
+ logDebug(s"xxx expandProjections: $expandProjections\nprojections:${expand.projections}")
+ ExpandExecTransformer(expandProjections, aggregateOutput, child)
+ }
+
+}
diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/extension/PushdownAggregatePreProjectionAheadExpand.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/PushdownAggregatePreProjectionAheadExpand.scala
index a3fab3c954ee..9335fd8829a9 100644
--- a/backends-clickhouse/src/main/scala/org/apache/gluten/extension/PushdownAggregatePreProjectionAheadExpand.scala
+++ b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/PushdownAggregatePreProjectionAheadExpand.scala
@@ -27,7 +27,7 @@ import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.execution.SparkPlan
// If there is an expression (not a attribute) in an aggregation function's
-// parameters. It will introduce a pr-projection to calculate the expression
+// parameters. It will introduce a pre-projection to calculate the expression
// at first, and make all the parameters be attributes.
// If it's a aggregation with grouping set, this pre-projection is placed after
// expand operator. This is not efficient, we cannot move this pre-projection
@@ -83,7 +83,7 @@ case class PushdownAggregatePreProjectionAheadExpand(session: SparkSession)
val originInputAttributes = aheadProjectExprs.filter(e => isAttributeOrLiteral(e))
val preProjectExprs = aheadProjectExprs.filter(e => !isAttributeOrLiteral(e))
- if (preProjectExprs.length == 0) {
+ if (preProjectExprs.isEmpty) {
return hashAggregate
}
@@ -93,11 +93,31 @@ case class PushdownAggregatePreProjectionAheadExpand(session: SparkSession)
return hashAggregate
}
+ def projectInputExists(expr: Expression, inputs: Seq[Attribute]): Boolean = {
+ expr.children.foreach {
+ case a: Attribute =>
+ return inputs.exists(i => i.name.equals(a.name) && i.exprId.equals(a.exprId))
+ case p: Expression =>
+ return projectInputExists(p, inputs)
+ case _ =>
+ return true
+ }
+ true
+ }
+
+ val couldPushDown = preProjectExprs.forall {
+ case p: Expression => projectInputExists(p, rootChild.output)
+ case _ => true
+ }
+
+ if (!couldPushDown) {
+ return hashAggregate;
+ }
+
// The new ahead project node will take rootChild's output and preProjectExprs as the
// the projection expressions.
val aheadProject = ProjectExecTransformer(rootChild.output ++ preProjectExprs, rootChild)
val aheadProjectOuput = aheadProject.output
-
val preProjectOutputAttrs = aheadProjectOuput.filter(
e =>
!originInputAttributes.exists(_.exprId.equals(e.asInstanceOf[NamedExpression].exprId)))
diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/extension/RemoveDuplicatedColumns.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/RemoveDuplicatedColumns.scala
new file mode 100644
index 000000000000..7f378b5a41a0
--- /dev/null
+++ b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/RemoveDuplicatedColumns.scala
@@ -0,0 +1,126 @@
+/*
+ * 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.
+ */
+package org.apache.gluten.extension
+
+import org.apache.gluten.execution._
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.execution._
+import org.apache.spark.sql.execution.CHColumnarToRowExec
+
+/*
+ * CH doesn't support will for duplicate columns in the a block.
+ * Most of the cases that introduce duplicate columns are from group by.
+ */
+case class RemoveDuplicatedColumns(session: SparkSession) extends Rule[SparkPlan] with Logging {
+ override def apply(plan: SparkPlan): SparkPlan = {
+ visitPlan(plan)
+ }
+
+ private def visitPlan(plan: SparkPlan): SparkPlan = {
+ plan match {
+ case c2r @ CHColumnarToRowExec(hashAgg: CHHashAggregateExecTransformer) =>
+ // This is a special case. Use the result from aggregation as the input of sink.
+ // We need to make the schema same as the input of sink.
+ val newChildren = hashAgg.children.map(visitPlan)
+ val newHashAgg = uniqueHashAggregateColumns(hashAgg)
+ .withNewChildren(newChildren)
+ .asInstanceOf[CHHashAggregateExecTransformer]
+ if (newHashAgg.resultExpressions.length != hashAgg.resultExpressions.length) {
+ val project = ProjectExecTransformer(hashAgg.resultExpressions, newHashAgg)
+ c2r.copy(child = project)
+ } else {
+ c2r.copy(child = newHashAgg)
+ }
+ case hashAgg: CHHashAggregateExecTransformer =>
+ val newChildren = hashAgg.children.map(visitPlan)
+ val newHashAgg = uniqueHashAggregateColumns(hashAgg)
+ newHashAgg.withNewChildren(newChildren)
+ case _ =>
+ plan.withNewChildren(plan.children.map(visitPlan))
+ }
+ }
+
+ private def unwrapAliasNamedExpression(e: NamedExpression): NamedExpression = {
+ e match {
+ case a: Alias =>
+ if (a.child.isInstanceOf[NamedExpression]) {
+ a.child.asInstanceOf[NamedExpression]
+ } else {
+ a
+ }
+ case _ => e
+ }
+ }
+ private def unwrapAliasExpression(e: Expression): Expression = {
+ e match {
+ case a: Alias =>
+ if (a.child.isInstanceOf[Expression]) {
+ a.child.asInstanceOf[Expression]
+ } else {
+ a
+ }
+ case _ => e
+ }
+ }
+
+ private def uniqueNamedExpressions(
+ groupingExpressions: Seq[NamedExpression]): Seq[NamedExpression] = {
+ var addedExpression = Seq[NamedExpression]()
+ groupingExpressions.foreach {
+ e =>
+ val unwrapped = unwrapAliasNamedExpression(e)
+ if (
+ !addedExpression.exists(_.semanticEquals(unwrapped)) && !unwrapped.isInstanceOf[Literal]
+ ) {
+ addedExpression = addedExpression :+ unwrapped
+ }
+ }
+ addedExpression
+ }
+
+ private def uniqueExpressions(expressions: Seq[Expression]): Seq[Expression] = {
+ var addedExpression = Seq[Expression]()
+ expressions.foreach {
+ e =>
+ val unwrapped = unwrapAliasExpression(e)
+ if (
+ !addedExpression.exists(_.semanticEquals(unwrapped)) && !unwrapped.isInstanceOf[Literal]
+ ) {
+ addedExpression = addedExpression :+ unwrapped
+ }
+ }
+ addedExpression
+ }
+
+ private def uniqueHashAggregateColumns(
+ hashAgg: CHHashAggregateExecTransformer): CHHashAggregateExecTransformer = {
+ val newGroupingExpressions = uniqueNamedExpressions(hashAgg.groupingExpressions)
+ val newResultExpressions = uniqueNamedExpressions(hashAgg.resultExpressions)
+ if (newResultExpressions.length != hashAgg.resultExpressions.length) {
+ hashAgg
+ .copy(
+ groupingExpressions = newGroupingExpressions,
+ resultExpressions = newResultExpressions)
+ } else {
+ hashAgg
+ }
+ }
+}
diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/extension/WriteFilesWithBucketValue.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/WriteFilesWithBucketValue.scala
new file mode 100644
index 000000000000..8ab78dcff9bc
--- /dev/null
+++ b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/WriteFilesWithBucketValue.scala
@@ -0,0 +1,76 @@
+/*
+ * 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.
+ */
+package org.apache.gluten.extension
+
+import org.apache.gluten.GlutenConfig
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, BitwiseAnd, Expression, HiveHash, Literal, Pmod}
+import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.execution.{ProjectExec, SparkPlan}
+import org.apache.spark.sql.execution.datasources.WriteFilesExec
+
+/**
+ * Wrap with bucket value to specify the bucket file name in native write. Native writer will remove
+ * this value in the final output.
+ */
+object WriteFilesWithBucketValue extends Rule[SparkPlan] {
+
+ val optionForHiveCompatibleBucketWrite = "__hive_compatible_bucketed_table_insertion__"
+
+ override def apply(plan: SparkPlan): SparkPlan = {
+ if (
+ GlutenConfig.getConf.enableGluten
+ && GlutenConfig.getConf.enableNativeWriter.getOrElse(false)
+ ) {
+ plan.transformDown {
+ case writeFiles: WriteFilesExec if writeFiles.bucketSpec.isDefined =>
+ val bucketIdExp = getWriterBucketIdExp(writeFiles)
+ val wrapBucketValue = ProjectExec(
+ writeFiles.child.output :+ Alias(bucketIdExp, "__bucket_value__")(),
+ writeFiles.child)
+ writeFiles.copy(child = wrapBucketValue)
+ }
+ } else {
+ plan
+ }
+ }
+
+ private def getWriterBucketIdExp(writeFilesExec: WriteFilesExec): Expression = {
+ val partitionColumns = writeFilesExec.partitionColumns
+ val outputColumns = writeFilesExec.child.output
+ val dataColumns = outputColumns.filterNot(partitionColumns.contains)
+ val bucketSpec = writeFilesExec.bucketSpec.get
+ val bucketColumns = bucketSpec.bucketColumnNames.map(c => dataColumns.find(_.name == c).get)
+ if (writeFilesExec.options.getOrElse(optionForHiveCompatibleBucketWrite, "false") == "true") {
+ val hashId = BitwiseAnd(HiveHash(bucketColumns), Literal(Int.MaxValue))
+ Pmod(hashId, Literal(bucketSpec.numBuckets))
+ // The bucket file name prefix is following Hive, Presto and Trino conversion, so this
+ // makes sure Hive bucketed table written by Spark, can be read by other SQL engines.
+ //
+ // Hive: `org.apache.hadoop.hive.ql.exec.Utilities#getBucketIdFromFile()`.
+ // Trino: `io.trino.plugin.hive.BackgroundHiveSplitLoader#BUCKET_PATTERNS`.
+
+ } else {
+ // Spark bucketed table: use `HashPartitioning.partitionIdExpression` as bucket id
+ // expression, so that we can guarantee the data distribution is same between shuffle and
+ // bucketed data source, which enables us to only shuffle one side when join a bucketed
+ // table and a normal one.
+ HashPartitioning(bucketColumns, bucketSpec.numBuckets).partitionIdExpression
+ }
+ }
+}
diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/metrics/MetricsUtil.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/metrics/MetricsUtil.scala
index e1e0f7c11a09..7d81467e978f 100644
--- a/backends-clickhouse/src/main/scala/org/apache/gluten/metrics/MetricsUtil.scala
+++ b/backends-clickhouse/src/main/scala/org/apache/gluten/metrics/MetricsUtil.scala
@@ -38,6 +38,9 @@ object MetricsUtil extends Logging {
j.metricsUpdater(),
// must put the buildPlan first
Seq(treeifyMetricsUpdaters(j.buildPlan), treeifyMetricsUpdaters(j.streamedPlan)))
+ case t: TransformSupport if t.metricsUpdater() == MetricsUpdater.None =>
+ assert(t.children.size == 1, "MetricsUpdater.None can only be used on unary operator")
+ treeifyMetricsUpdaters(t.children.head)
case t: TransformSupport =>
MetricsUpdaterTree(t.metricsUpdater(), t.children.map(treeifyMetricsUpdaters))
case _ =>
diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/utils/PlanNodesUtil.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/utils/PlanNodesUtil.scala
index 9dcb7ee3c41d..d6511f7a4a29 100644
--- a/backends-clickhouse/src/main/scala/org/apache/gluten/utils/PlanNodesUtil.scala
+++ b/backends-clickhouse/src/main/scala/org/apache/gluten/utils/PlanNodesUtil.scala
@@ -22,13 +22,17 @@ import org.apache.gluten.substrait.expression.ExpressionNode
import org.apache.gluten.substrait.plan.{PlanBuilder, PlanNode}
import org.apache.gluten.substrait.rel.RelBuilder
-import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, BoundReference, Expression}
+import org.apache.spark.sql.catalyst.expressions.{Attribute, BoundReference, Expression}
import com.google.common.collect.Lists
+import java.util
+
+import scala.collection.JavaConverters._
+
object PlanNodesUtil {
- def genProjectionsPlanNode(key: Expression, output: Seq[Attribute]): PlanNode = {
+ def genProjectionsPlanNode(key: Seq[Expression], output: Seq[Attribute]): PlanNode = {
val context = new SubstraitContext
var operatorId = context.nextOperatorId("ClickHouseBuildSideRelationReadIter")
@@ -36,41 +40,36 @@ object PlanNodesUtil {
val nameList = ConverterUtils.collectAttributeNamesWithExprId(output)
val readRel = RelBuilder.makeReadRelForInputIterator(typeList, nameList, context, operatorId)
- // replace attribute to BoundRefernce according to the output
- val newBoundRefKey = key.transformDown {
- case expression: AttributeReference =>
- val columnInOutput = output.zipWithIndex.filter {
- p: (Attribute, Int) => p._1.exprId == expression.exprId || p._1.name == expression.name
- }
- if (columnInOutput.isEmpty) {
- throw new IllegalStateException(
- s"Key $expression not found from build side relation output: $output")
- }
- if (columnInOutput.size != 1) {
- throw new IllegalStateException(
- s"More than one key $expression found from build side relation output: $output")
- }
- val boundReference = columnInOutput.head
- BoundReference(boundReference._2, boundReference._1.dataType, boundReference._1.nullable)
- case other => other
- }
-
// project
operatorId = context.nextOperatorId("ClickHouseBuildSideRelationProjection")
val args = context.registeredFunction
val columnarProjExpr = ExpressionConverter
- .replaceWithExpressionTransformer(newBoundRefKey, attributeSeq = output)
+ .replaceWithExpressionTransformer(key, attributeSeq = output)
val projExprNodeList = new java.util.ArrayList[ExpressionNode]()
- projExprNodeList.add(columnarProjExpr.doTransform(args))
+ columnarProjExpr.foreach(e => projExprNodeList.add(e.doTransform(args)))
PlanBuilder.makePlan(
context,
Lists.newArrayList(
RelBuilder.makeProjectRel(readRel, projExprNodeList, context, operatorId, output.size)),
- Lists.newArrayList(
- ConverterUtils.genColumnNameWithExprId(ConverterUtils.getAttrFromExpr(key)))
+ Lists.newArrayList(genColumnNameWithExprId(key, output))
)
}
+
+ private def genColumnNameWithExprId(
+ key: Seq[Expression],
+ output: Seq[Attribute]): util.List[String] = {
+ key
+ .map {
+ k =>
+ val reference = k.collectFirst { case BoundReference(ordinal, _, _) => output(ordinal) }
+ assert(reference.isDefined)
+ reference.get
+ }
+ .map(ConverterUtils.genColumnNameWithExprId)
+ .toList
+ .asJava
+ }
}
diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/vectorized/NativeExpressionEvaluator.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/vectorized/NativeExpressionEvaluator.scala
new file mode 100644
index 000000000000..35533a4b3e9c
--- /dev/null
+++ b/backends-clickhouse/src/main/scala/org/apache/gluten/vectorized/NativeExpressionEvaluator.scala
@@ -0,0 +1,39 @@
+/*
+ * 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.
+ */
+package org.apache.gluten.vectorized
+
+import org.apache.gluten.backendsapi.clickhouse.CHConf
+import org.apache.gluten.utils.ConfigUtil
+
+import scala.collection.JavaConverters._
+
+// TODO: move CHNativeExpressionEvaluator to NativeExpressionEvaluator
+/** Scala Wrapper for ExpressionEvaluatorJniWrapper */
+object NativeExpressionEvaluator {
+
+ def updateQueryRuntimeSettings(settings: Map[String, String]): Unit = {
+ ExpressionEvaluatorJniWrapper.updateQueryRuntimeSettings(
+ ConfigUtil.serialize(
+ settings
+ .filter(t => CHConf.startWithSettingsPrefix(t._1) && t._2.nonEmpty)
+ .map {
+ case (k, v) =>
+ (CHConf.removeSettingsPrefix(k), v)
+ }
+ .asJava))
+ }
+}
diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/shuffle/utils/RangePartitionerBoundsGenerator.scala b/backends-clickhouse/src/main/scala/org/apache/spark/shuffle/utils/RangePartitionerBoundsGenerator.scala
index 87c6ae343d4c..706cc5f34108 100644
--- a/backends-clickhouse/src/main/scala/org/apache/spark/shuffle/utils/RangePartitionerBoundsGenerator.scala
+++ b/backends-clickhouse/src/main/scala/org/apache/spark/shuffle/utils/RangePartitionerBoundsGenerator.scala
@@ -16,7 +16,7 @@
*/
package org.apache.spark.shuffle.utils
-import org.apache.gluten.backendsapi.BackendsApiManager
+import org.apache.gluten.backendsapi.clickhouse.CHValidatorApi
import org.apache.gluten.execution.SortExecTransformer
import org.apache.gluten.expression.ExpressionConverter
import org.apache.gluten.substrait.SubstraitContext
@@ -261,8 +261,9 @@ object RangePartitionerBoundsGenerator {
break
}
if (
- !ordering.child.isInstanceOf[Attribute] && !BackendsApiManager.getSettings
- .supportShuffleWithProject(rangePartitioning, child)
+ !ordering.child.isInstanceOf[Attribute] && !CHValidatorApi.supportShuffleWithProject(
+ rangePartitioning,
+ child)
) {
enableRangePartitioning = false
break
diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/delta/MergeTreeFileFormat.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/delta/MergeTreeFileFormat.scala
index dd24d475340b..bea22404eafe 100644
--- a/backends-clickhouse/src/main/scala/org/apache/spark/sql/delta/MergeTreeFileFormat.scala
+++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/delta/MergeTreeFileFormat.scala
@@ -21,7 +21,8 @@ import org.apache.gluten.execution.datasource.GlutenFormatFactory
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.delta.actions.Metadata
import org.apache.spark.sql.execution.datasources.{FileFormat, OutputWriter, OutputWriterFactory}
-import org.apache.spark.sql.execution.datasources.mergetree.DeltaMetaReader
+import org.apache.spark.sql.execution.datasources.mergetree.{DeltaMetaReader, StorageMeta}
+import org.apache.spark.sql.execution.datasources.v1.Write
import org.apache.spark.sql.sources.DataSourceRegister
import org.apache.spark.sql.types.StructType
@@ -64,3 +65,28 @@ trait MergeTreeFileFormat extends FileFormat with DataSourceRegister {
}
}
}
+
+object MergeTreeFileFormat {
+ def createWrite(outputPath: Option[String], conf: Map[String, String]): Write.MergeTreeWrite = {
+ Write.MergeTreeWrite
+ .newBuilder()
+ .setDatabase(conf(StorageMeta.DB))
+ .setTable(conf(StorageMeta.TABLE))
+ .setSnapshotId(conf(StorageMeta.SNAPSHOT_ID))
+ .setOrderByKey(conf(StorageMeta.ORDER_BY_KEY))
+ .setLowCardKey(conf(StorageMeta.LOW_CARD_KEY))
+ .setMinmaxIndexKey(conf(StorageMeta.MINMAX_INDEX_KEY))
+ .setBfIndexKey(conf(StorageMeta.BF_INDEX_KEY))
+ .setSetIndexKey(conf(StorageMeta.SET_INDEX_KEY))
+ .setPrimaryKey(conf(StorageMeta.PRIMARY_KEY))
+ .setRelativePath(outputPath.map(StorageMeta.normalizeRelativePath).getOrElse(""))
+ .setAbsolutePath("")
+ .setStoragePolicy(conf(StorageMeta.POLICY))
+ .build()
+ }
+ def createWrite(metadata: Metadata): Write.MergeTreeWrite = {
+ // we can get the output path at the driver side
+ val deltaMetaReader = DeltaMetaReader(metadata)
+ createWrite(None, deltaMetaReader.storageConf)
+ }
+}
diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/delta/files/MergeTreeFileCommitProtocol.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/delta/files/MergeTreeFileCommitProtocol.scala
index bce43b0ef288..13a9efa35989 100644
--- a/backends-clickhouse/src/main/scala/org/apache/spark/sql/delta/files/MergeTreeFileCommitProtocol.scala
+++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/delta/files/MergeTreeFileCommitProtocol.scala
@@ -16,9 +16,9 @@
*/
package org.apache.spark.sql.delta.files
-import org.apache.gluten.backendsapi.clickhouse.CHConf
+import org.apache.gluten.backendsapi.clickhouse.RuntimeSettings
import org.apache.gluten.memory.CHThreadGroup
-import org.apache.gluten.vectorized.ExpressionEvaluatorJniWrapper
+import org.apache.gluten.vectorized.NativeExpressionEvaluator
import org.apache.spark.internal.io.FileCommitProtocol
import org.apache.spark.internal.io.FileCommitProtocol.TaskCommitMessage
@@ -31,8 +31,6 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext
import java.util.UUID
-import scala.collection.JavaConverters._
-
trait MergeTreeFileCommitProtocol extends FileCommitProtocol {
def outputPath: String
@@ -45,8 +43,8 @@ trait MergeTreeFileCommitProtocol extends FileCommitProtocol {
val jobID = taskContext.getJobID.toString
val taskAttemptID = taskContext.getTaskAttemptID.toString
MergeTreeCommiterHelper.prepareTaskWriteInfo(jobID, taskAttemptID)
- val settings = Map(CHConf.runtimeSettings("gluten.write.reserve_partition_columns") -> "true")
- ExpressionEvaluatorJniWrapper.updateQueryRuntimeSettings(settings.asJava)
+ val settings = Map(RuntimeSettings.NATIVE_WRITE_RESERVE_PARTITION_COLUMNS.key -> "true")
+ NativeExpressionEvaluator.updateQueryRuntimeSettings(settings)
}
override def newTaskTempFile(
@@ -54,22 +52,22 @@ trait MergeTreeFileCommitProtocol extends FileCommitProtocol {
dir: Option[String],
ext: String): String = {
- taskContext.getConfiguration.set(
- "mapreduce.task.gluten.mergetree.partition",
- dir.map(p => new Path(p).toUri.toString).getOrElse(""))
-
+ val partitionStr = dir.map(p => new Path(p).toUri.toString)
val bucketIdStr = ext.split("\\.").headOption.filter(_.startsWith("_")).map(_.substring(1))
- taskContext.getConfiguration.set(
- "mapreduce.task.gluten.mergetree.bucketid",
- bucketIdStr.getOrElse(""))
+ val split = taskContext.getTaskAttemptID.getTaskID.getId
- val partition = dir.map(p => new Path(p).toUri.toString + "/").getOrElse("")
+ // The partPrefix is used to generate the part name in the MergeTree table.
+ // outputPath/partition-dir/bucket-id/UUID_split
+ val partition = partitionStr.map(_ + "/").getOrElse("")
val bucket = bucketIdStr.map(_ + "/").getOrElse("")
- val taskID = taskContext.getTaskAttemptID.getTaskID.getId.toString
- val partPrefix = s"$partition$bucket${UUID.randomUUID.toString}_$taskID"
-
- taskContext.getConfiguration.set("mapreduce.task.gluten.mergetree.partPrefix", partPrefix)
+ val partPrefix = s"$partition$bucket${UUID.randomUUID.toString}_$split"
+ val settings = Map(
+ RuntimeSettings.PART_NAME_PREFIX.key -> partPrefix,
+ RuntimeSettings.PARTITION_DIR.key -> partitionStr.getOrElse(""),
+ RuntimeSettings.BUCKET_DIR.key -> bucketIdStr.getOrElse("")
+ )
+ NativeExpressionEvaluator.updateQueryRuntimeSettings(settings)
outputPath
}
@@ -87,4 +85,8 @@ trait MergeTreeFileCommitProtocol extends FileCommitProtocol {
)
new TaskCommitMessage(statuses)
}
+
+ override def abortTask(taskContext: TaskAttemptContext): Unit = {
+ MergeTreeCommiterHelper.resetCurrentTaskWriteInfo()
+ }
}
diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/delta/util/MergeTreePartitionUtils.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/delta/util/MergeTreePartitionUtils.scala
new file mode 100644
index 000000000000..03b45e540dc9
--- /dev/null
+++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/delta/util/MergeTreePartitionUtils.scala
@@ -0,0 +1,58 @@
+/*
+ * 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.
+ */
+package org.apache.spark.sql.delta.util
+
+import org.apache.spark.sql.catalyst.expressions.Cast
+import org.apache.spark.sql.types.StringType
+
+import org.apache.hadoop.fs.Path
+
+/**
+ * `OptimizeTableCommandOverwrites` does not use `DelayedCommitProtocol`, so we can't use
+ * `DelayedCommitProtocol.parsePartitions`. This is a copied version. TODO: Remove it.
+ */
+object MergeTreePartitionUtils {
+
+ private val timestampPartitionPattern = "yyyy-MM-dd HH:mm:ss[.S]"
+
+ def parsePartitions(dir: String): Map[String, String] = {
+ // TODO: timezones?
+ // TODO: enable validatePartitionColumns?
+ val dateFormatter = DateFormatter()
+ val timestampFormatter =
+ TimestampFormatter(timestampPartitionPattern, java.util.TimeZone.getDefault)
+ val parsedPartition =
+ PartitionUtils
+ .parsePartition(
+ new Path(dir),
+ typeInference = false,
+ Set.empty,
+ Map.empty,
+ validatePartitionColumns = false,
+ java.util.TimeZone.getDefault,
+ dateFormatter,
+ timestampFormatter)
+ ._1
+ .get
+ parsedPartition.columnNames
+ .zip(
+ parsedPartition.literals
+ .map(l => Cast(l, StringType).eval())
+ .map(Option(_).map(_.toString).orNull))
+ .toMap
+ }
+}
diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/CHColumnarToRowExec.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/CHColumnarToRowExec.scala
index 29fa0d0aba96..174493e5f4f6 100644
--- a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/CHColumnarToRowExec.scala
+++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/CHColumnarToRowExec.scala
@@ -19,7 +19,6 @@ package org.apache.spark.sql.execution
import org.apache.gluten.execution.ColumnarToRowExecBase
import org.apache.gluten.extension.ValidationResult
import org.apache.gluten.metrics.GlutenTimeMetric
-import org.apache.gluten.vectorized.CHNativeBlock
import org.apache.spark.{OneToOneDependency, Partition, SparkContext, TaskContext}
import org.apache.spark.broadcast.Broadcast
@@ -103,10 +102,7 @@ class CHColumnarToRowRDD(
logInfo(s"Skip ColumnarBatch of ${batch.numRows} rows, ${batch.numCols} cols")
Iterator.empty
} else {
- val blockAddress = GlutenTimeMetric.millis(convertTime) {
- _ => CHNativeBlock.fromColumnarBatch(batch).blockAddress()
- }
- CHExecUtil.getRowIterFromSparkRowInfo(blockAddress, batch.numCols(), batch.numRows())
+ GlutenTimeMetric.millis(convertTime)(_ => CHExecUtil.c2r(batch))
}
}
}
diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/CHColumnarWrite.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/CHColumnarWrite.scala
index 6a5c19a4f939..1342e250430e 100644
--- a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/CHColumnarWrite.scala
+++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/CHColumnarWrite.scala
@@ -16,25 +16,28 @@
*/
package org.apache.spark.sql.execution
-import org.apache.gluten.backendsapi.BackendsApiManager
+import org.apache.gluten.backendsapi.clickhouse.RuntimeSettings
+import org.apache.gluten.vectorized.NativeExpressionEvaluator
import org.apache.spark.TaskContext
import org.apache.spark.internal.Logging
import org.apache.spark.internal.io.{FileCommitProtocol, FileNameSpec, HadoopMapReduceCommitProtocol}
import org.apache.spark.internal.io.FileCommitProtocol.TaskCommitMessage
+import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.GenericInternalRow
-import org.apache.spark.sql.execution.datasources.{BasicWriteTaskStats, ExecutedWriteSummary, PartitioningUtils, WriteJobDescription, WriteTaskResult}
-import org.apache.spark.sql.vectorized.ColumnarBatch
+import org.apache.spark.sql.delta.stats.DeltaJobStatisticsTracker
+import org.apache.spark.sql.execution.datasources._
import org.apache.spark.util.Utils
import org.apache.hadoop.fs.Path
-import org.apache.hadoop.mapreduce.{JobID, OutputCommitter, TaskAttemptContext, TaskAttemptID, TaskID, TaskType}
+import org.apache.hadoop.mapreduce._
import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter
import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl
import java.lang.reflect.Field
import scala.collection.mutable
+import scala.language.implicitConversions
trait CHColumnarWrite[T <: FileCommitProtocol] {
@@ -51,7 +54,17 @@ trait CHColumnarWrite[T <: FileCommitProtocol] {
def abortTask(): Unit = {
committer.abortTask(taskAttemptContext)
}
- def commitTask(batch: ColumnarBatch): Option[WriteTaskResult]
+ def commitTask(writeResults: Seq[InternalRow]): Option[WriteTaskResult]
+
+ lazy val basicWriteJobStatsTracker: WriteTaskStatsTracker = description.statsTrackers
+ .find(_.isInstanceOf[BasicWriteJobStatsTracker])
+ .map(_.newTaskInstance())
+ .get
+
+ lazy val deltaWriteJobStatsTracker: Option[DeltaJobStatisticsTracker] =
+ description.statsTrackers
+ .find(_.isInstanceOf[DeltaJobStatisticsTracker])
+ .map(_.asInstanceOf[DeltaJobStatisticsTracker])
lazy val (taskAttemptContext: TaskAttemptContext, jobId: String) = {
// Copied from `SparkHadoopWriterUtils.createJobID` to be compatible with multi-version
@@ -84,29 +97,16 @@ trait CHColumnarWrite[T <: FileCommitProtocol] {
object CreateFileNameSpec {
def apply(taskContext: TaskAttemptContext, description: WriteJobDescription): FileNameSpec = {
val fileCounter = 0
- val suffix = f".c$fileCounter%03d" +
+ val suffix = f"-c$fileCounter%03d" +
description.outputWriterFactory.getFileExtension(taskContext)
FileNameSpec("", suffix)
}
}
-object CreateBasicWriteTaskStats {
- def apply(
- numFiles: Int,
- updatedPartitions: Set[String],
- numWrittenRows: Long): BasicWriteTaskStats = {
- val partitionsInternalRows = updatedPartitions.map {
- part =>
- val parts = new Array[Any](1)
- parts(0) = part
- new GenericInternalRow(parts)
- }.toSeq
- BasicWriteTaskStats(
- partitions = partitionsInternalRows,
- numFiles = numFiles,
- numBytes = 101,
- numRows = numWrittenRows)
- }
+// More details in local_engine::FileNameGenerator in NormalFileWriter.cpp
+object FileNamePlaceHolder {
+ val ID = "{id}"
+ val BUCKET = "{bucket}"
}
/** [[HadoopMapReduceAdapter]] for [[HadoopMapReduceCommitProtocol]]. */
@@ -139,12 +139,105 @@ case class HadoopMapReduceAdapter(sparkCommitter: HadoopMapReduceCommitProtocol)
GetFilename.invoke(sparkCommitter, taskContext, spec).asInstanceOf[String]
}
- def getTaskAttemptTempPathAndFilename(
+ def getTaskAttemptTempPathAndFilePattern(
taskContext: TaskAttemptContext,
description: WriteJobDescription): (String, String) = {
val stageDir = newTaskAttemptTempPath(description.path)
- val filename = getFilename(taskContext, CreateFileNameSpec(taskContext, description))
- (stageDir, filename)
+
+ if (isBucketWrite(description)) {
+ val filePart = getFilename(taskContext, FileNameSpec("", ""))
+ val fileSuffix = CreateFileNameSpec(taskContext, description).suffix
+ (stageDir, s"${filePart}_${FileNamePlaceHolder.BUCKET}$fileSuffix")
+ } else {
+ val filename = getFilename(taskContext, CreateFileNameSpec(taskContext, description))
+ (stageDir, filename)
+ }
+ }
+
+ private def isBucketWrite(desc: WriteJobDescription): Boolean = {
+ // In Spark 3.2, bucketSpec is not defined, instead, it uses bucketIdExpression.
+ val bucketSpecField: Field = desc.getClass.getDeclaredField("bucketSpec")
+ bucketSpecField.setAccessible(true)
+ bucketSpecField.get(desc).asInstanceOf[Option[_]].isDefined
+ }
+}
+
+/**
+ * {{{
+ * val schema =
+ * StructType(
+ * StructField("filename", StringType, false) ::
+ * StructField("partition_id", StringType, false) ::
+ * StructField("record_count", LongType, false) :: Nil)
+ * }}}
+ */
+case class NativeFileWriteResult(filename: String, partition_id: String, record_count: Long) {
+ lazy val relativePath: String = if (partition_id == "__NO_PARTITION_ID__") {
+ filename
+ } else {
+ s"$partition_id/$filename"
+ }
+}
+
+object NativeFileWriteResult {
+ implicit def apply(row: InternalRow): NativeFileWriteResult = {
+ NativeFileWriteResult(row.getString(0), row.getString(1), row.getLong(2))
+ }
+}
+
+case class NativeStatCompute(rows: Seq[InternalRow]) {
+ def apply[T](stats: Seq[T => Unit], extra: Option[InternalRow => Unit] = None)(implicit
+ creator: InternalRow => T): Unit = {
+ rows.foreach {
+ row =>
+ val stat = creator(row)
+ stats.foreach(agg => agg(stat))
+ extra.foreach(_(row))
+ }
+ }
+}
+
+case class NativeBasicWriteTaskStatsTracker(
+ description: WriteJobDescription,
+ basicWriteJobStatsTracker: WriteTaskStatsTracker)
+ extends (NativeFileWriteResult => Unit) {
+ private var numWrittenRows: Long = 0
+ override def apply(stat: NativeFileWriteResult): Unit = {
+ val absolutePath = s"${description.path}/${stat.relativePath}"
+ if (stat.partition_id != "__NO_PARTITION_ID__") {
+ basicWriteJobStatsTracker.newPartition(new GenericInternalRow(Array[Any](stat.partition_id)))
+ }
+ basicWriteJobStatsTracker.newFile(absolutePath)
+ basicWriteJobStatsTracker.closeFile(absolutePath)
+ numWrittenRows += stat.record_count
+ }
+ private def finalStats: BasicWriteTaskStats = basicWriteJobStatsTracker
+ .getFinalStats(0)
+ .asInstanceOf[BasicWriteTaskStats]
+
+ def result: BasicWriteTaskStats = finalStats.copy(numRows = numWrittenRows)
+}
+
+case class FileCommitInfo(description: WriteJobDescription)
+ extends (NativeFileWriteResult => Unit) {
+ private val partitions: mutable.Set[String] = mutable.Set[String]()
+ private val addedAbsPathFiles: mutable.Map[String, String] = mutable.Map[String, String]()
+
+ def apply(stat: NativeFileWriteResult): Unit = {
+ val tmpAbsolutePath = s"${description.path}/${stat.relativePath}"
+ if (stat.partition_id != "__NO_PARTITION_ID__") {
+ partitions += stat.partition_id
+ val customOutputPath =
+ description.customPartitionLocations.get(
+ PartitioningUtils.parsePathFragment(stat.partition_id))
+ if (customOutputPath.isDefined) {
+ addedAbsPathFiles(tmpAbsolutePath) = customOutputPath.get + "/" + stat.filename
+ }
+ }
+ }
+
+ def result: (Set[String], Map[String, String]) = {
+ (partitions.toSet, addedAbsPathFiles.toMap)
}
}
@@ -162,58 +255,40 @@ case class HadoopMapReduceCommitProtocolWrite(
* initializing the native plan and collect native write files metrics for each backend.
*/
override def doSetupNativeTask(): Unit = {
- val (writePath, writeFileName) =
- adapter.getTaskAttemptTempPathAndFilename(taskAttemptContext, description)
- logDebug(s"Native staging write path: $writePath and file name: $writeFileName")
- BackendsApiManager.getIteratorApiInstance.injectWriteFilesTempPath(writePath, writeFileName)
+ val (writePath, writeFilePattern) =
+ adapter.getTaskAttemptTempPathAndFilePattern(taskAttemptContext, description)
+ logDebug(s"Native staging write path: $writePath and file pattern: $writeFilePattern")
+
+ val settings =
+ Map(
+ RuntimeSettings.TASK_WRITE_TMP_DIR.key -> writePath,
+ RuntimeSettings.TASK_WRITE_FILENAME_PATTERN.key -> writeFilePattern)
+ NativeExpressionEvaluator.updateQueryRuntimeSettings(settings)
}
- def doCollectNativeResult(cb: ColumnarBatch): Option[WriteTaskResult] = {
- val numFiles = cb.numRows()
+ def doCollectNativeResult(stats: Seq[InternalRow]): Option[WriteTaskResult] = {
// Write an empty iterator
- if (numFiles == 0) {
+ if (stats.isEmpty) {
None
} else {
- val file_col = cb.column(0)
- val partition_col = cb.column(1)
- val count_col = cb.column(2)
-
- val outputPath = description.path
- val partitions: mutable.Set[String] = mutable.Set[String]()
- val addedAbsPathFiles: mutable.Map[String, String] = mutable.Map[String, String]()
-
- var numWrittenRows: Long = 0
- Range(0, cb.numRows()).foreach {
- i =>
- val targetFileName = file_col.getUTF8String(i).toString
- val partition = partition_col.getUTF8String(i).toString
- if (partition != "__NO_PARTITION_ID__") {
- partitions += partition
- val tmpOutputPath = outputPath + "/" + partition + "/" + targetFileName
- val customOutputPath =
- description.customPartitionLocations.get(
- PartitioningUtils.parsePathFragment(partition))
- if (customOutputPath.isDefined) {
- addedAbsPathFiles(tmpOutputPath) = customOutputPath.get + "/" + targetFileName
- }
- }
- numWrittenRows += count_col.getLong(i)
- }
-
- val updatedPartitions = partitions.toSet
- val summary =
- ExecutedWriteSummary(
- updatedPartitions = updatedPartitions,
- stats = Seq(CreateBasicWriteTaskStats(numFiles, updatedPartitions, numWrittenRows)))
+ val commitInfo = FileCommitInfo(description)
+ val basicNativeStat = NativeBasicWriteTaskStatsTracker(description, basicWriteJobStatsTracker)
+ val basicNativeStats = Seq(commitInfo, basicNativeStat)
+ NativeStatCompute(stats)(basicNativeStats)
+ val (partitions, addedAbsPathFiles) = commitInfo.result
+ val updatedPartitions = partitions
Some(
WriteTaskResult(
- new TaskCommitMessage(addedAbsPathFiles.toMap -> updatedPartitions),
- summary))
+ new TaskCommitMessage(addedAbsPathFiles -> updatedPartitions),
+ ExecutedWriteSummary(
+ updatedPartitions = updatedPartitions,
+ stats = Seq(basicNativeStat.result))
+ ))
}
}
- override def commitTask(batch: ColumnarBatch): Option[WriteTaskResult] = {
- doCollectNativeResult(batch).map(
+ override def commitTask(writeResults: Seq[InternalRow]): Option[WriteTaskResult] = {
+ doCollectNativeResult(writeResults).map(
nativeWriteTaskResult => {
val (_, taskCommitTime) = Utils.timeTakenMs {
committer.commitTask(taskAttemptContext)
diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/CHColumnarWriteFilesExec.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/CHColumnarWriteFilesExec.scala
index bf051671fbba..503fd1a90caa 100644
--- a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/CHColumnarWriteFilesExec.scala
+++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/CHColumnarWriteFilesExec.scala
@@ -16,6 +16,8 @@
*/
package org.apache.spark.sql.execution
+import org.apache.gluten.memory.CHThreadGroup
+
import org.apache.spark.{Partition, SparkException, TaskContext, TaskOutputFileAlreadyExistException}
import org.apache.spark.internal.io.{FileCommitProtocol, SparkHadoopWriterUtils}
import org.apache.spark.rdd.RDD
@@ -25,6 +27,7 @@ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec
import org.apache.spark.sql.catalyst.expressions.Attribute
import org.apache.spark.sql.connector.write.WriterCommitMessage
import org.apache.spark.sql.execution.datasources._
+import org.apache.spark.sql.execution.utils.CHExecUtil
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.vectorized.ColumnarBatch
import org.apache.spark.util.Utils
@@ -46,14 +49,16 @@ class CHColumnarWriteFilesRDD(
extends RDD[WriterCommitMessage](prev) {
private def reportTaskMetrics(writeTaskResult: WriteTaskResult): Unit = {
- val stats = writeTaskResult.summary.stats.head.asInstanceOf[BasicWriteTaskStats]
- val (numBytes, numWrittenRows) = (stats.numBytes, stats.numRows)
- // Reports bytesWritten and recordsWritten to the Spark output metrics.
- // We should update it after calling `commitTask` to overwrite the metrics.
- Option(TaskContext.get()).map(_.taskMetrics().outputMetrics).foreach {
- outputMetrics =>
- outputMetrics.setBytesWritten(numBytes)
- outputMetrics.setRecordsWritten(numWrittenRows)
+ writeTaskResult.summary.stats.find(_.isInstanceOf[BasicWriteTaskStats]).foreach {
+ s =>
+ val stats = s.asInstanceOf[BasicWriteTaskStats]
+ // Reports bytesWritten and recordsWritten to the Spark output metrics.
+ // We should update it after calling `commitTask` to overwrite the metrics.
+ Option(TaskContext.get()).map(_.taskMetrics().outputMetrics).foreach {
+ outputMetrics =>
+ outputMetrics.setBytesWritten(stats.numBytes)
+ outputMetrics.setRecordsWritten(stats.numRows)
+ }
}
}
@@ -78,6 +83,7 @@ class CHColumnarWriteFilesRDD(
}
override def compute(split: Partition, context: TaskContext): Iterator[WriterCommitMessage] = {
+ CHThreadGroup.registerNewThreadGroup()
val commitProtocol = CHColumnarWrite(jobTrackerID, description, committer)
commitProtocol.setupTask()
@@ -90,8 +96,19 @@ class CHColumnarWriteFilesRDD(
assert(iter.hasNext)
val resultColumnarBatch = iter.next()
assert(resultColumnarBatch != null)
+
+ /**
+ * we assume the number of records is less than 10,000.So the memory overhead is acceptable.
+ * otherwise, we need to access ColumnarBatch row by row, which is not efficient.
+ */
+ val writeResults = CHExecUtil.c2r(resultColumnarBatch).map(_.copy()).toSeq
+ // TODO: we need close iterator here before processing the result.
+ // TODO: task commit time
+ // TODO: get the schema from result ColumnarBatch and verify it.
+ assert(!iter.hasNext)
+
val writeTaskResult = commitProtocol
- .commitTask(resultColumnarBatch)
+ .commitTask(writeResults)
.orElse({
// If we are writing an empty iterator, then gluten backend would do nothing.
// Here we fallback to use vanilla Spark write files to generate an empty file for
diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/CHFormatWriterInjects.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/CHFormatWriterInjects.scala
index b360e286e504..7f80c134e5d6 100644
--- a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/CHFormatWriterInjects.scala
+++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/CHFormatWriterInjects.scala
@@ -78,7 +78,7 @@ trait CHFormatWriterInjects extends GlutenFormatWriterInjectsBase {
val datasourceJniWrapper =
new CHDatasourceJniWrapper(outputPath, createWriteRel(outputPath, dataSchema, context))
- new FakeRowOutputWriter(datasourceJniWrapper, outputPath)
+ new FakeRowOutputWriter(Some(datasourceJniWrapper), outputPath)
}
override def inferSchema(
diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/CHMergeTreeWriterInjects.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/CHMergeTreeWriterInjects.scala
index 76f4b467ded2..3e0e393459a6 100644
--- a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/CHMergeTreeWriterInjects.scala
+++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/CHMergeTreeWriterInjects.scala
@@ -16,6 +16,7 @@
*/
package org.apache.spark.sql.execution.datasources.v1
+import org.apache.gluten.backendsapi.clickhouse.CHConf
import org.apache.gluten.execution.ColumnarToRowExecBase
import org.apache.gluten.expression.ConverterUtils
import org.apache.gluten.substrait.`type`.ColumnTypeNode
@@ -26,6 +27,7 @@ import org.apache.gluten.substrait.rel.RelBuilder
import org.apache.gluten.utils.ConfigUtil
import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.delta.MergeTreeFileFormat
import org.apache.spark.sql.execution.SparkPlan
import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec
import org.apache.spark.sql.execution.datasources.{CHDatasourceJniWrapper, FakeRowAdaptor, OutputWriter}
@@ -75,22 +77,7 @@ class CHMergeTreeWriterInjects extends CHFormatWriterInjects {
.setFormat(formatName)
.setJobTaskAttemptId(s"$jobID/$taskAttemptID")
.build())
- .setMergetree(
- Write.MergeTreeWrite
- .newBuilder()
- .setDatabase(conf(StorageMeta.DB))
- .setTable(conf(StorageMeta.TABLE))
- .setSnapshotId(conf(StorageMeta.SNAPSHOT_ID))
- .setOrderByKey(conf(StorageMeta.ORDER_BY_KEY))
- .setLowCardKey(conf(StorageMeta.LOW_CARD_KEY))
- .setMinmaxIndexKey(conf(StorageMeta.MINMAX_INDEX_KEY))
- .setBfIndexKey(conf(StorageMeta.BF_INDEX_KEY))
- .setSetIndexKey(conf(StorageMeta.SET_INDEX_KEY))
- .setPrimaryKey(conf(StorageMeta.PRIMARY_KEY))
- .setRelativePath(StorageMeta.normalizeRelativePath(outputPath))
- .setAbsolutePath("")
- .setStoragePolicy(conf(StorageMeta.POLICY))
- .build())
+ .setMergetree(MergeTreeFileFormat.createWrite(Some(outputPath), conf))
.build()
}
@@ -100,14 +87,24 @@ class CHMergeTreeWriterInjects extends CHFormatWriterInjects {
context: TaskAttemptContext,
nativeConf: ju.Map[String, String]): OutputWriter = {
- val datasourceJniWrapper = new CHDatasourceJniWrapper(
- context.getConfiguration.get("mapreduce.task.gluten.mergetree.partPrefix"),
- context.getConfiguration.get("mapreduce.task.gluten.mergetree.partition"),
- context.getConfiguration.get("mapreduce.task.gluten.mergetree.bucketid"),
- createWriteRel(outputPath, dataSchema, context),
- ConfigUtil.serialize(nativeConf)
- )
- new FakeRowOutputWriter(datasourceJniWrapper, outputPath)
+ val wrapper = if (CHConf.get.enableOnePipelineMergeTreeWrite) {
+
+ /**
+ * In pipeline mode, CHColumnarWriteFilesRDD.writeFilesForEmptyIterator will create a JNI
+ * wrapper which is not needed in this case.
+ *
+ * TODO: We should refactor the code to avoid creating the JNI wrapper in this case.
+ */
+ None
+ } else {
+ val datasourceJniWrapper = new CHDatasourceJniWrapper(
+ createWriteRel(outputPath, dataSchema, context),
+ ConfigUtil.serialize(nativeConf)
+ )
+ Some(datasourceJniWrapper)
+ }
+
+ new FakeRowOutputWriter(wrapper, outputPath)
}
override val formatName: String = "mergetree"
diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/FakeRowOutputWriter.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/FakeRowOutputWriter.scala
index 9ebce9d3b455..574c9784257d 100644
--- a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/FakeRowOutputWriter.scala
+++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/FakeRowOutputWriter.scala
@@ -24,7 +24,7 @@ import org.apache.spark.sql.execution.datasources.{CHDatasourceJniWrapper, FakeR
import scala.collection.mutable.ArrayBuffer
-class FakeRowOutputWriter(datasourceJniWrapper: CHDatasourceJniWrapper, outputPath: String)
+class FakeRowOutputWriter(datasourceJniWrapper: Option[CHDatasourceJniWrapper], outputPath: String)
extends OutputWriter {
protected var addFiles: ArrayBuffer[AddFile] = new ArrayBuffer[AddFile]()
@@ -35,12 +35,12 @@ class FakeRowOutputWriter(datasourceJniWrapper: CHDatasourceJniWrapper, outputPa
if (nextBatch.numRows > 0) {
val col = nextBatch.column(0).asInstanceOf[CHColumnVector]
- datasourceJniWrapper.write(col.getBlockAddress)
+ datasourceJniWrapper.foreach(_.write(col.getBlockAddress))
} // else ignore this empty block
}
override def close(): Unit = {
- datasourceJniWrapper.close()
+ datasourceJniWrapper.foreach(_.close())
}
// Do NOT add override keyword for compatibility on spark 3.1.
diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/clickhouse/MergeTreeCommiterHelper.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/clickhouse/MergeTreeCommiterHelper.scala
index 1d6a488afebf..436c6fa66a29 100644
--- a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/clickhouse/MergeTreeCommiterHelper.scala
+++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/clickhouse/MergeTreeCommiterHelper.scala
@@ -48,4 +48,8 @@ object MergeTreeCommiterHelper {
require(currentTaskWriteInfo.get() == null, "currentTaskWriteInfo is not null")
currentTaskWriteInfo.set(TaskWriteInfo(s"$jobID/$taskAttemptID"))
}
+
+ def resetCurrentTaskWriteInfo(): Unit = {
+ currentTaskWriteInfo.remove()
+ }
}
diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/metadata/AddFileTags.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/metadata/AddFileTags.scala
index 8c610929a3d8..c4c971633a11 100644
--- a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/metadata/AddFileTags.scala
+++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/metadata/AddFileTags.scala
@@ -17,6 +17,7 @@
package org.apache.spark.sql.execution.datasources.v2.clickhouse.metadata
import org.apache.spark.sql.delta.actions.AddFile
+import org.apache.spark.sql.delta.util.MergeTreePartitionUtils
import org.apache.spark.sql.execution.datasources.clickhouse.WriteReturnedMetric
import com.fasterxml.jackson.core.`type`.TypeReference
@@ -26,7 +27,6 @@ import org.apache.hadoop.fs.Path
import java.util.{List => JList}
import scala.collection.JavaConverters._
-import scala.collection.mutable.ArrayBuffer
@SuppressWarnings(Array("io.github.zhztheplayer.scalawarts.InheritFromCaseClass"))
class AddMergeTreeParts(
@@ -192,46 +192,47 @@ object AddFileTags {
tableName: String,
originPathStr: String,
returnedMetrics: String,
- hostName: Seq[String]): ArrayBuffer[AddFile] = {
+ hostName: Seq[String]): Seq[AddFile] = {
+
val mapper: ObjectMapper = new ObjectMapper()
- try {
- val values: JList[WriteReturnedMetric] =
- mapper.readValue(returnedMetrics, new TypeReference[JList[WriteReturnedMetric]]() {})
- var addFiles = new ArrayBuffer[AddFile]()
- val path = new Path(originPathStr)
- val modificationTime = System.currentTimeMillis()
- addFiles.appendAll(values.asScala.map {
- value =>
- AddFileTags.partsInfoToAddFile(
- database,
- tableName,
- "MergeTree",
- path.toUri.getPath,
- hostName.map(_.trim).mkString(","),
- value.getPartName,
- "",
- value.getRowCount,
- value.getDiskSize,
- -1L,
- -1L,
- modificationTime,
- "",
- -1L,
- -1L,
- -1,
- -1L,
- value.getBucketId,
- path.toString,
- dataChange = true,
- "",
- partitionValues = value.getPartitionValues.asScala.toMap,
- marks = value.getMarkCount
- )
- })
- addFiles
- } catch {
- case e: Exception =>
- ArrayBuffer.empty[AddFile]
- }
+ val values: JList[WriteReturnedMetric] =
+ mapper.readValue(returnedMetrics, new TypeReference[JList[WriteReturnedMetric]]() {})
+ val path = new Path(originPathStr)
+ val modificationTime = System.currentTimeMillis()
+
+ values.asScala.map {
+ value =>
+ val partitionValues = if (value.getPartitionValues.isEmpty) {
+ Map.empty[String, String]
+ } else {
+ MergeTreePartitionUtils.parsePartitions(value.getPartitionValues)
+ }
+
+ AddFileTags.partsInfoToAddFile(
+ database,
+ tableName,
+ "MergeTree",
+ path.toUri.getPath,
+ hostName.map(_.trim).mkString(","),
+ value.getPartName,
+ "",
+ value.getRowCount,
+ value.getDiskSize,
+ -1L,
+ -1L,
+ modificationTime,
+ "",
+ -1L,
+ -1L,
+ -1,
+ -1L,
+ value.getBucketId,
+ path.toString,
+ dataChange = true,
+ "",
+ partitionValues = partitionValues,
+ marks = value.getMarkCount
+ )
+ }.toSeq
}
}
diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/joins/ClickHouseBuildSideRelation.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/joins/ClickHouseBuildSideRelation.scala
index 92887f16d70a..668525ba0a40 100644
--- a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/joins/ClickHouseBuildSideRelation.scala
+++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/joins/ClickHouseBuildSideRelation.scala
@@ -22,8 +22,8 @@ import org.apache.gluten.vectorized._
import org.apache.spark.internal.Logging
import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression}
-import org.apache.spark.sql.catalyst.plans.physical.BroadcastMode
+import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, UnsafeProjection}
+import org.apache.spark.sql.catalyst.plans.physical.{BroadcastMode, IdentityBroadcastMode}
import org.apache.spark.sql.execution.utils.CHExecUtil
import org.apache.spark.sql.vectorized.ColumnarBatch
import org.apache.spark.storage.CHShuffleReadStreamFactory
@@ -72,7 +72,7 @@ case class ClickHouseBuildSideRelation(
}
/**
- * Transform columnar broadcast value to Array[InternalRow] by key and distinct.
+ * Transform columnar broadcast value to Array[InternalRow] by key.
*
* @return
*/
@@ -80,10 +80,18 @@ case class ClickHouseBuildSideRelation(
// native block reader
val blockReader = new CHStreamReader(CHShuffleReadStreamFactory.create(batches, true))
val broadCastIter: Iterator[ColumnarBatch] = IteratorUtil.createBatchIterator(blockReader)
+
+ val transformProjections = mode match {
+ case HashedRelationBroadcastMode(k, _) => k
+ case IdentityBroadcastMode => output
+ }
+
// Expression compute, return block iterator
val expressionEval = new SimpleExpressionEval(
new ColumnarNativeIterator(broadCastIter.asJava),
- PlanNodesUtil.genProjectionsPlanNode(key, output))
+ PlanNodesUtil.genProjectionsPlanNode(transformProjections, output))
+
+ val proj = UnsafeProjection.create(Seq(key))
try {
// convert columnar to row
@@ -95,6 +103,7 @@ case class ClickHouseBuildSideRelation(
} else {
CHExecUtil
.getRowIterFromSparkRowInfo(block, batch.numColumns(), batch.numRows())
+ .map(proj)
.map(row => row.copy())
}
}.toArray
diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/utils/CHExecUtil.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/utils/CHExecUtil.scala
index 6d91108d22da..cc172ac4b543 100644
--- a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/utils/CHExecUtil.scala
+++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/utils/CHExecUtil.scala
@@ -113,12 +113,8 @@ object CHExecUtil extends Logging {
iter =>
iter.flatMap(
batch => {
- val blockAddress = CHNativeBlock.fromColumnarBatch(batch).blockAddress()
-
// generate rows from a columnar batch
- val rowItr: Iterator[InternalRow] =
- getRowIterFromSparkRowInfo(blockAddress, batch.numCols(), batch.numRows())
-
+ val rowItr: Iterator[InternalRow] = c2r(batch)
val projection =
UnsafeProjection.create(sortingExpressions.map(_.child), outputAttributes)
val mutablePair = new MutablePair[InternalRow, Null]()
@@ -164,6 +160,14 @@ object CHExecUtil extends Logging {
val rowInfo = CHBlockConverterJniWrapper.convertColumnarToRow(blockAddress, null)
getRowIterFromSparkRowInfo(rowInfo, columns, rows)
}
+
+ def c2r(batch: ColumnarBatch): Iterator[InternalRow] = {
+ getRowIterFromSparkRowInfo(
+ CHNativeBlock.fromColumnarBatch(batch).blockAddress(),
+ batch.numCols(),
+ batch.numRows())
+ }
+
private def buildPartitionedBlockIterator(
cbIter: Iterator[ColumnarBatch],
options: IteratorOptions,
diff --git a/backends-clickhouse/src/test-iceberg/scala/org/apache/gluten/execution/iceberg/ClickHouseIcebergSuite.scala b/backends-clickhouse/src/test-iceberg/scala/org/apache/gluten/execution/iceberg/ClickHouseIcebergSuite.scala
new file mode 100644
index 000000000000..676061235ed0
--- /dev/null
+++ b/backends-clickhouse/src/test-iceberg/scala/org/apache/gluten/execution/iceberg/ClickHouseIcebergSuite.scala
@@ -0,0 +1,640 @@
+/*
+ * 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.
+ */
+package org.apache.gluten.execution.iceberg
+
+import org.apache.gluten.execution.{GlutenClickHouseWholeStageTransformerSuite, IcebergScanTransformer}
+import org.apache.gluten.GlutenConfig
+import org.apache.spark.SparkConf
+import org.apache.spark.sql.Row
+
+class ClickHouseIcebergSuite extends GlutenClickHouseWholeStageTransformerSuite {
+
+ override protected def sparkConf: SparkConf = {
+ super.sparkConf
+ .set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager")
+ .set("spark.sql.adaptive.enabled", "true")
+ .set("spark.sql.shuffle.partitions", "2")
+ .set("spark.memory.offHeap.size", "2g")
+ .set("spark.unsafe.exceptionOnMemoryLeak", "true")
+ .set("spark.sql.autoBroadcastJoinThreshold", "-1")
+ .set(
+ "spark.sql.extensions",
+ "org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions")
+ .set("spark.sql.catalog.spark_catalog", "org.apache.iceberg.spark.SparkCatalog")
+ .set("spark.sql.catalog.spark_catalog.type", "hadoop")
+ .set("spark.sql.catalog.spark_catalog.warehouse", s"file://$basePath/tpch-data-iceberg")
+ }
+
+ test("iceberg transformer exists") {
+ withTable("iceberg_tb") {
+ spark.sql(
+ """
+ |create table iceberg_tb using iceberg as
+ |(select 1 as col1, 2 as col2, 3 as col3)
+ |""".stripMargin)
+
+ runQueryAndCompare(
+ """
+ |select * from iceberg_tb;
+ |""".stripMargin) {
+ checkGlutenOperatorMatch[IcebergScanTransformer]
+ }
+ }
+ }
+
+ testWithSpecifiedSparkVersion(
+ "iceberg bucketed join", Array("3.3", "3.5")) {
+ val leftTable = "p_str_tb"
+ val rightTable = "p_int_tb"
+ withTable(leftTable, rightTable) {
+ // Partition key of string type.
+ withSQLConf(GlutenConfig.GLUTEN_ENABLED_KEY -> "false") {
+ // Gluten does not support write iceberg table.
+ spark.sql(
+ s"""
+ |create table $leftTable(id int, name string, p string)
+ |using iceberg
+ |partitioned by (bucket(4, id));
+ |""".stripMargin)
+ spark.sql(
+ s"""
+ |insert into table $leftTable values
+ |(4, 'a5', 'p4'),
+ |(1, 'a1', 'p1'),
+ |(2, 'a3', 'p2'),
+ |(1, 'a2', 'p1'),
+ |(3, 'a4', 'p3');
+ |""".stripMargin
+ )
+ }
+
+ // Partition key of integer type.
+ withSQLConf(
+ GlutenConfig.GLUTEN_ENABLED_KEY -> "false"
+ ) {
+ // Gluten does not support write iceberg table.
+ spark.sql(
+ s"""
+ |create table $rightTable(id int, name string, p int)
+ |using iceberg
+ |partitioned by (bucket(4, id));
+ |""".stripMargin)
+ spark.sql(
+ s"""
+ |insert into table $rightTable values
+ |(3, 'b4', 23),
+ |(1, 'b2', 21),
+ |(4, 'b5', 24),
+ |(2, 'b3', 22),
+ |(1, 'b1', 21);
+ |""".stripMargin
+ )
+ }
+
+ withSQLConf(
+ "spark.sql.sources.v2.bucketing.enabled" -> "true",
+ "spark.sql.requireAllClusterKeysForCoPartition" -> "false",
+ "spark.sql.adaptive.enabled" -> "false",
+ "spark.sql.iceberg.planning.preserve-data-grouping" -> "true",
+ "spark.sql.autoBroadcastJoinThreshold" -> "-1",
+ "spark.sql.sources.v2.bucketing.pushPartValues.enabled" -> "true"
+ ) {
+ runQueryAndCompare(
+ s"""
+ |select s.id, s.name, i.name, i.p
+ | from $leftTable s inner join $rightTable i
+ | on s.id = i.id;
+ |""".stripMargin) {
+ df => {
+ assert(
+ getExecutedPlan(df).count(
+ plan => {
+ plan.isInstanceOf[IcebergScanTransformer]
+ }) == 2)
+ getExecutedPlan(df).map {
+ case plan if plan.isInstanceOf[IcebergScanTransformer] =>
+ assert(
+ plan.asInstanceOf[IcebergScanTransformer].getKeyGroupPartitioning.isDefined)
+ assert(plan.asInstanceOf[IcebergScanTransformer].getSplitInfos.length == 3)
+ case _ => // do nothing
+ }
+ checkLengthAndPlan(df, 7)
+ }
+ }
+ }
+ }
+ }
+
+ testWithSpecifiedSparkVersion(
+ "iceberg bucketed join with partition", Array("3.3", "3.5")) {
+ val leftTable = "p_str_tb"
+ val rightTable = "p_int_tb"
+ withTable(leftTable, rightTable) {
+ // Partition key of string type.
+ withSQLConf(GlutenConfig.GLUTEN_ENABLED_KEY -> "false") {
+ // Gluten does not support write iceberg table.
+ spark.sql(
+ s"""
+ |create table $leftTable(id int, name string, p int)
+ |using iceberg
+ |partitioned by (bucket(4, id), p);
+ |""".stripMargin)
+ spark.sql(
+ s"""
+ |insert into table $leftTable values
+ |(4, 'a5', 2),
+ |(1, 'a1', 1),
+ |(2, 'a3', 1),
+ |(1, 'a2', 1),
+ |(3, 'a4', 2);
+ |""".stripMargin
+ )
+ }
+
+ // Partition key of integer type.
+ withSQLConf(
+ GlutenConfig.GLUTEN_ENABLED_KEY -> "false"
+ ) {
+ // Gluten does not support write iceberg table.
+ spark.sql(
+ s"""
+ |create table $rightTable(id int, name string, p int)
+ |using iceberg
+ |partitioned by (bucket(4, id), p);
+ |""".stripMargin)
+ spark.sql(
+ s"""
+ |insert into table $rightTable values
+ |(3, 'b4', 2),
+ |(1, 'b2', 1),
+ |(4, 'b5', 2),
+ |(2, 'b3', 1),
+ |(1, 'b1', 1);
+ |""".stripMargin
+ )
+ }
+
+ withSQLConf(
+ "spark.sql.sources.v2.bucketing.enabled" -> "true",
+ "spark.sql.requireAllClusterKeysForCoPartition" -> "false",
+ "spark.sql.adaptive.enabled" -> "false",
+ "spark.sql.iceberg.planning.preserve-data-grouping" -> "true",
+ "spark.sql.autoBroadcastJoinThreshold" -> "-1",
+ "spark.sql.sources.v2.bucketing.pushPartValues.enabled" -> "true"
+ ) {
+ runQueryAndCompare(
+ s"""
+ |select s.id, s.name, i.name, i.p
+ | from $leftTable s inner join $rightTable i
+ | on s.id = i.id and s.p = i.p;
+ |""".stripMargin) {
+ df => {
+ assert(
+ getExecutedPlan(df).count(
+ plan => {
+ plan.isInstanceOf[IcebergScanTransformer]
+ }) == 2)
+ getExecutedPlan(df).map {
+ case plan if plan.isInstanceOf[IcebergScanTransformer] =>
+ assert(
+ plan.asInstanceOf[IcebergScanTransformer].getKeyGroupPartitioning.isDefined)
+ assert(plan.asInstanceOf[IcebergScanTransformer].getSplitInfos.length == 3)
+ case _ => // do nothing
+ }
+ checkLengthAndPlan(df, 7)
+ }
+ }
+ }
+ }
+ }
+
+ testWithSpecifiedSparkVersion(
+ "iceberg bucketed join with partition filter", Array("3.3", "3.5")) {
+ val leftTable = "p_str_tb"
+ val rightTable = "p_int_tb"
+ withTable(leftTable, rightTable) {
+ // Partition key of string type.
+ withSQLConf(GlutenConfig.GLUTEN_ENABLED_KEY -> "false") {
+ // Gluten does not support write iceberg table.
+ spark.sql(
+ s"""
+ |create table $leftTable(id int, name string, p int)
+ |using iceberg
+ |partitioned by (bucket(4, id), p);
+ |""".stripMargin)
+ spark.sql(
+ s"""
+ |insert into table $leftTable values
+ |(4, 'a5', 2),
+ |(1, 'a1', 1),
+ |(2, 'a3', 1),
+ |(1, 'a2', 1),
+ |(3, 'a4', 2);
+ |""".stripMargin
+ )
+ }
+
+ // Partition key of integer type.
+ withSQLConf(
+ GlutenConfig.GLUTEN_ENABLED_KEY -> "false"
+ ) {
+ // Gluten does not support write iceberg table.
+ spark.sql(
+ s"""
+ |create table $rightTable(id int, name string, p int)
+ |using iceberg
+ |partitioned by (bucket(4, id), p);
+ |""".stripMargin)
+ spark.sql(
+ s"""
+ |insert into table $rightTable values
+ |(3, 'b4', 2),
+ |(1, 'b2', 1),
+ |(4, 'b5', 2),
+ |(2, 'b3', 1),
+ |(1, 'b1', 1);
+ |""".stripMargin
+ )
+ }
+
+ withSQLConf(
+ "spark.sql.sources.v2.bucketing.enabled" -> "true",
+ "spark.sql.requireAllClusterKeysForCoPartition" -> "false",
+ "spark.sql.adaptive.enabled" -> "false",
+ "spark.sql.iceberg.planning.preserve-data-grouping" -> "true",
+ "spark.sql.autoBroadcastJoinThreshold" -> "-1",
+ "spark.sql.sources.v2.bucketing.pushPartValues.enabled" -> "true"
+ ) {
+ runQueryAndCompare(
+ s"""
+ |select s.id, s.name, i.name, i.p
+ | from $leftTable s inner join $rightTable i
+ | on s.id = i.id
+ | where s.p = 1 and i.p = 1;
+ |""".stripMargin) {
+ df => {
+ assert(
+ getExecutedPlan(df).count(
+ plan => {
+ plan.isInstanceOf[IcebergScanTransformer]
+ }) == 2)
+ getExecutedPlan(df).map {
+ case plan if plan.isInstanceOf[IcebergScanTransformer] =>
+ assert(
+ plan.asInstanceOf[IcebergScanTransformer].getKeyGroupPartitioning.isDefined)
+ assert(plan.asInstanceOf[IcebergScanTransformer].getSplitInfos.length == 1)
+ case _ => // do nothing
+ }
+ checkLengthAndPlan(df, 5)
+ }
+ }
+ }
+ }
+ }
+
+ testWithSpecifiedSparkVersion("iceberg: time travel") {
+ withTable("iceberg_tm") {
+ spark.sql(
+ s"""
+ |create table iceberg_tm (id int, name string) using iceberg
+ |""".stripMargin)
+ spark.sql(
+ s"""
+ |insert into iceberg_tm values (1, "v1"), (2, "v2")
+ |""".stripMargin)
+ spark.sql(
+ s"""
+ |insert into iceberg_tm values (3, "v3"), (4, "v4")
+ |""".stripMargin)
+
+ val df =
+ spark.sql("select snapshot_id from default.iceberg_tm.snapshots where parent_id is null")
+ val value = df.collectAsList().get(0).getAs[Long](0);
+ spark.sql(s"call system.set_current_snapshot('default.iceberg_tm',$value)");
+ val data = runQueryAndCompare("select * from iceberg_tm") { _ => }
+ checkLengthAndPlan(data, 2)
+ checkAnswer(data, Row(1, "v1") :: Row(2, "v2") :: Nil)
+ }
+ }
+
+ test("iceberg: partition filters") {
+ withTable("iceberg_pf") {
+ spark.sql(
+ s"""
+ |create table iceberg_pf (id int, name string)
+ | using iceberg partitioned by (name)
+ |""".stripMargin)
+ spark.sql(
+ s"""
+ |insert into iceberg_pf values (1, "v1"), (2, "v2"), (3, "v1"), (4, "v2")
+ |""".stripMargin)
+ val df1 = runQueryAndCompare("select * from iceberg_pf where name = 'v1'") { _ => }
+ checkLengthAndPlan(df1, 2)
+ checkAnswer(df1, Row(1, "v1") :: Row(3, "v1") :: Nil)
+ }
+ }
+
+ test("iceberg read mor table - delete and update") {
+ withTable("iceberg_mor_tb") {
+ withSQLConf(GlutenConfig.GLUTEN_ENABLED_KEY -> "false") {
+ spark.sql(
+ """
+ |create table iceberg_mor_tb (
+ | id int,
+ | name string,
+ | p string
+ |) using iceberg
+ |tblproperties (
+ | 'format-version' = '2'
+ |)
+ |partitioned by (p);
+ |""".stripMargin)
+
+ // Insert some test rows.
+ spark.sql(
+ """
+ |insert into table iceberg_mor_tb
+ |values (1, 'a1', 'p1'), (2, 'a2', 'p1'), (3, 'a3', 'p2'),
+ | (4, 'a4', 'p1'), (5, 'a5', 'p2'), (6, 'a6', 'p1');
+ |""".stripMargin)
+
+ // Delete row.
+ spark.sql(
+ """
+ |delete from iceberg_mor_tb where name = 'a1';
+ |""".stripMargin
+ )
+ // Update row.
+ spark.sql(
+ """
+ |update iceberg_mor_tb set name = 'new_a2' where id = 'a2';
+ |""".stripMargin
+ )
+ // Delete row again.
+ spark.sql(
+ """
+ |delete from iceberg_mor_tb where id = 6;
+ |""".stripMargin
+ )
+ }
+ runQueryAndCompare(
+ """
+ |select * from iceberg_mor_tb;
+ |""".stripMargin) {
+ checkGlutenOperatorMatch[IcebergScanTransformer]
+ }
+ }
+ }
+
+ // TODO: support merge-on-read mode
+ ignore("iceberg read mor table - delete and update with merge-on-read mode") {
+ withTable("iceberg_mor_tb") {
+ withSQLConf(GlutenConfig.GLUTEN_ENABLED_KEY -> "false") {
+ spark.sql(
+ """
+ |create table iceberg_mor_tb (
+ | id int,
+ | name string,
+ | p string
+ |) using iceberg
+ |tblproperties (
+ | 'format-version' = '2',
+ | 'write.delete.mode' = 'merge-on-read',
+ | 'write.update.mode' = 'merge-on-read',
+ | 'write.merge.mode' = 'merge-on-read'
+ |)
+ |partitioned by (p);
+ |""".stripMargin)
+
+ // Insert some test rows.
+ spark.sql(
+ """
+ |insert into table iceberg_mor_tb
+ |values (1, 'a1', 'p1'), (2, 'a2', 'p1'), (3, 'a3', 'p2'),
+ | (4, 'a4', 'p1'), (5, 'a5', 'p2'), (6, 'a6', 'p1');
+ |""".stripMargin)
+
+ // Delete row.
+ spark.sql(
+ """
+ |delete from iceberg_mor_tb where name = 'a1';
+ |""".stripMargin
+ )
+ // Update row.
+ spark.sql(
+ """
+ |update iceberg_mor_tb set name = 'new_a2' where id = 'a2';
+ |""".stripMargin
+ )
+ // Delete row again.
+ spark.sql(
+ """
+ |delete from iceberg_mor_tb where id = 6;
+ |""".stripMargin
+ )
+ }
+ runQueryAndCompare(
+ """
+ |select * from iceberg_mor_tb;
+ |""".stripMargin) {
+ checkGlutenOperatorMatch[IcebergScanTransformer]
+ }
+ }
+ }
+
+ test("iceberg read mor table - merge into") {
+ withTable("iceberg_mor_tb", "merge_into_source_tb") {
+ withSQLConf(GlutenConfig.GLUTEN_ENABLED_KEY -> "false") {
+ spark.sql(
+ """
+ |create table iceberg_mor_tb (
+ | id int,
+ | name string,
+ | p string
+ |) using iceberg
+ |tblproperties (
+ | 'format-version' = '2'
+ |)
+ |partitioned by (p);
+ |""".stripMargin)
+ spark.sql(
+ """
+ |create table merge_into_source_tb (
+ | id int,
+ | name string,
+ | p string
+ |) using iceberg;
+ |""".stripMargin)
+
+ // Insert some test rows.
+ spark.sql(
+ """
+ |insert into table iceberg_mor_tb
+ |values (1, 'a1', 'p1'), (2, 'a2', 'p1'), (3, 'a3', 'p2');
+ |""".stripMargin)
+ spark.sql(
+ """
+ |insert into table merge_into_source_tb
+ |values (1, 'a1_1', 'p2'), (2, 'a2_1', 'p2'), (3, 'a3_1', 'p1'),
+ | (4, 'a4', 'p2'), (5, 'a5', 'p1'), (6, 'a6', 'p2');
+ |""".stripMargin)
+
+ // Delete row.
+ spark.sql(
+ """
+ |delete from iceberg_mor_tb where name = 'a1';
+ |""".stripMargin
+ )
+ // Update row.
+ spark.sql(
+ """
+ |update iceberg_mor_tb set name = 'new_a2' where id = 'a2';
+ |""".stripMargin
+ )
+
+ // Merge into.
+ spark.sql(
+ """
+ |merge into iceberg_mor_tb t
+ |using (select * from merge_into_source_tb) s
+ |on t.id = s.id
+ |when matched then
+ | update set t.name = s.name, t.p = s.p
+ |when not matched then
+ | insert (id, name, p) values (s.id, s.name, s.p);
+ |""".stripMargin
+ )
+ }
+ runQueryAndCompare(
+ """
+ |select * from iceberg_mor_tb;
+ |""".stripMargin) {
+ checkGlutenOperatorMatch[IcebergScanTransformer]
+ }
+ }
+ }
+
+ // TODO: support merge-on-read mode
+ ignore("iceberg read mor table - merge into with merge-on-read mode") {
+ withTable("iceberg_mor_tb", "merge_into_source_tb") {
+ withSQLConf(GlutenConfig.GLUTEN_ENABLED_KEY -> "false") {
+ spark.sql(
+ """
+ |create table iceberg_mor_tb (
+ | id int,
+ | name string,
+ | p string
+ |) using iceberg
+ |tblproperties (
+ | 'format-version' = '2',
+ | 'write.delete.mode' = 'merge-on-read',
+ | 'write.update.mode' = 'merge-on-read',
+ | 'write.merge.mode' = 'merge-on-read'
+ |)
+ |partitioned by (p);
+ |""".stripMargin)
+ spark.sql(
+ """
+ |create table merge_into_source_tb (
+ | id int,
+ | name string,
+ | p string
+ |) using iceberg;
+ |""".stripMargin)
+
+ // Insert some test rows.
+ spark.sql(
+ """
+ |insert into table iceberg_mor_tb
+ |values (1, 'a1', 'p1'), (2, 'a2', 'p1'), (3, 'a3', 'p2');
+ |""".stripMargin)
+ spark.sql(
+ """
+ |insert into table merge_into_source_tb
+ |values (1, 'a1_1', 'p2'), (2, 'a2_1', 'p2'), (3, 'a3_1', 'p1'),
+ | (4, 'a4', 'p2'), (5, 'a5', 'p1'), (6, 'a6', 'p2');
+ |""".stripMargin)
+
+ // Delete row.
+ spark.sql(
+ """
+ |delete from iceberg_mor_tb where name = 'a1';
+ |""".stripMargin
+ )
+ // Update row.
+ spark.sql(
+ """
+ |update iceberg_mor_tb set name = 'new_a2' where id = 'a2';
+ |""".stripMargin
+ )
+
+ // Merge into.
+ spark.sql(
+ """
+ |merge into iceberg_mor_tb t
+ |using (select * from merge_into_source_tb) s
+ |on t.id = s.id
+ |when matched then
+ | update set t.name = s.name, t.p = s.p
+ |when not matched then
+ | insert (id, name, p) values (s.id, s.name, s.p);
+ |""".stripMargin
+ )
+ }
+ runQueryAndCompare(
+ """
+ |select * from iceberg_mor_tb;
+ |""".stripMargin) {
+ checkGlutenOperatorMatch[IcebergScanTransformer]
+ }
+ }
+ }
+
+ // Spark configuration spark.sql.iceberg.handle-timestamp-without-timezone is not supported
+ // in Spark 3.4
+ // TODO: there is a bug when using timestamp type as the partition column
+ testWithSpecifiedSparkVersion("iceberg partition type - timestamp", Array("")) {
+ Seq("true", "false").foreach {
+ flag =>
+ withSQLConf(
+ "spark.sql.iceberg.handle-timestamp-without-timezone" -> flag,
+ "spark.sql.iceberg.use-timestamp-without-timezone-in-new-tables" -> flag) {
+ withTable("part_by_timestamp") {
+ spark.sql(
+ """
+ |create table part_by_timestamp (
+ | p timestamp
+ |) using iceberg
+ |tblproperties (
+ | 'format-version' = '1'
+ |)
+ |partitioned by (p);
+ |""".stripMargin)
+
+ // Insert some test rows.
+ spark.sql(
+ """
+ |insert into table part_by_timestamp
+ |values (TIMESTAMP '2022-01-01 00:01:20');
+ |""".stripMargin)
+ val df = spark.sql("select * from part_by_timestamp")
+ checkAnswer(df, Row(java.sql.Timestamp.valueOf("2022-01-01 00:01:20")) :: Nil)
+ }
+ }
+ }
+ }
+}
diff --git a/backends-velox/src/main/scala/org/apache/spark/sql/execution/datasources/velox/VeloxOrcWriterInjects.scala b/backends-clickhouse/src/test/delta-20/org/apache/spark/gluten/delta/DeltaStatsUtils.scala
similarity index 62%
rename from backends-velox/src/main/scala/org/apache/spark/sql/execution/datasources/velox/VeloxOrcWriterInjects.scala
rename to backends-clickhouse/src/test/delta-20/org/apache/spark/gluten/delta/DeltaStatsUtils.scala
index 0999121dd8bd..5d9f761e8a74 100644
--- a/backends-velox/src/main/scala/org/apache/spark/sql/execution/datasources/velox/VeloxOrcWriterInjects.scala
+++ b/backends-clickhouse/src/test/delta-20/org/apache/spark/gluten/delta/DeltaStatsUtils.scala
@@ -14,19 +14,17 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.spark.sql.execution.datasources.velox
+package org.apache.spark.gluten.delta
-import scala.collection.JavaConverters.mapAsJavaMapConverter
-import scala.collection.mutable
+import org.apache.spark.sql.{DataFrame, SparkSession}
-class VeloxOrcWriterInjects extends VeloxFormatWriterInjects {
- override def nativeConf(
- options: Map[String, String],
- compressionCodec: String): java.util.Map[String, String] = {
- val sparkOptions = new mutable.HashMap[String, String]()
- // TODO: implement it
- sparkOptions.asJava
- }
+object DeltaStatsUtils {
- override val formatName: String = "orc"
+ def statsDF(
+ sparkSession: SparkSession,
+ deltaJson: String,
+ schema: String
+ ): DataFrame = {
+ throw new IllegalAccessException("Method not used below spark 3.5")
+ }
}
diff --git a/backends-clickhouse/src/test/delta-23/org/apache/spark/gluten/delta/DeltaStatsUtils.scala b/backends-clickhouse/src/test/delta-23/org/apache/spark/gluten/delta/DeltaStatsUtils.scala
new file mode 100644
index 000000000000..5d9f761e8a74
--- /dev/null
+++ b/backends-clickhouse/src/test/delta-23/org/apache/spark/gluten/delta/DeltaStatsUtils.scala
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+package org.apache.spark.gluten.delta
+
+import org.apache.spark.sql.{DataFrame, SparkSession}
+
+object DeltaStatsUtils {
+
+ def statsDF(
+ sparkSession: SparkSession,
+ deltaJson: String,
+ schema: String
+ ): DataFrame = {
+ throw new IllegalAccessException("Method not used below spark 3.5")
+ }
+}
diff --git a/backends-clickhouse/src/test/delta-32/org/apache/spark/gluten/delta/DeltaStatsUtils.scala b/backends-clickhouse/src/test/delta-32/org/apache/spark/gluten/delta/DeltaStatsUtils.scala
new file mode 100644
index 000000000000..08681892dc1d
--- /dev/null
+++ b/backends-clickhouse/src/test/delta-32/org/apache/spark/gluten/delta/DeltaStatsUtils.scala
@@ -0,0 +1,118 @@
+/*
+ * 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.
+ */
+package org.apache.spark.gluten.delta
+
+import org.apache.spark.sql.{wrapper, DataFrame, SparkSession}
+import org.apache.spark.sql.delta.{DeltaColumnMappingMode, DeltaLog, DeltaLogFileIndex, NoMapping}
+import org.apache.spark.sql.delta.actions.Protocol
+import org.apache.spark.sql.delta.actions.TableFeatureProtocolUtils.TABLE_FEATURES_MIN_WRITER_VERSION
+import org.apache.spark.sql.delta.stats.{DeltaStatsColumnSpec, StatisticsCollection}
+import org.apache.spark.sql.functions.{col, from_json, regexp_replace}
+import org.apache.spark.sql.types._
+
+import org.apache.hadoop.fs.{FileSystem, Path}
+
+import java.util.Locale
+
+case class Statistics(override val tableSchema: StructType) extends StatisticsCollection {
+ override val outputAttributeSchema: StructType = tableSchema
+ // [[outputTableStatsSchema]] is the candidate schema to find statistics columns.
+ override val outputTableStatsSchema: StructType = tableSchema
+ override val statsColumnSpec = DeltaStatsColumnSpec(None, Some(32))
+ override val columnMappingMode: DeltaColumnMappingMode = NoMapping
+ override val protocol: Protocol = Protocol(
+ minReaderVersion = 1,
+ minWriterVersion = TABLE_FEATURES_MIN_WRITER_VERSION,
+ readerFeatures = None,
+ writerFeatures = Some(Set()))
+
+ override def spark: SparkSession = {
+ throw new Exception("Method not used in statisticsCollectionFromMetadata")
+ }
+}
+
+object DeltaStatsUtils {
+
+ private def stringToDataType(dataType: String): DataType =
+ dataType.toLowerCase(Locale.ROOT) match {
+ case "bigint" => LongType
+ case "double" => DoubleType
+ case "string" => StringType
+ case "date" => DateType
+ case _ => throw new IllegalArgumentException(s"Unsupported data type: $dataType")
+ }
+
+ /**
+ * Parse a schema string as follows into a [[StructType]].
+ * {{{
+ * l_orderkey bigint,
+ * l_partkey bigint,
+ * l_suppkey bigint,
+ * l_linenumber bigint
+ * }}}
+ */
+ private def stringToSchema(schemaString: String): StructType = {
+ val fields = schemaString.trim.split(",\\s*").map {
+ fieldString =>
+ val parts = fieldString.trim.split("\\s+")
+ require(parts.length == 2, s"Invalid field definition: $fieldString")
+ val fieldName = parts(0).trim
+ val fieldType = stringToDataType(parts(1).trim)
+ StructField(fieldName, fieldType, nullable = true)
+ }
+ StructType(fields)
+ }
+
+ def statsDF(
+ sparkSession: SparkSession,
+ deltaJson: String,
+ schema: String
+ ): DataFrame = {
+
+ val statistics = Statistics(stringToSchema(schema))
+
+ val hadoopConf = sparkSession.sessionState.newHadoopConfWithOptions(Map.empty)
+ wrapper
+ .ofRows(
+ sparkSession,
+ DeltaLog.indexToRelation(
+ sparkSession,
+ DeltaLogFileIndex(
+ DeltaLogFileIndex.COMMIT_FILE_FORMAT,
+ FileSystem.get(hadoopConf),
+ Seq(new Path(deltaJson))),
+ Map.empty)
+ )
+ .select("add")
+ .filter("add is not null")
+ .withColumns(Map(
+ "path" -> regexp_replace(
+ col("add.path"),
+ "-[0-9A-Fa-f]{8}-[0-9A-Fa-f]{4}-[0-9A-Fa-f]{4}-[0-9A-Fa-f]{4}-[0-9A-Fa-f]{12}",
+ ""
+ ), // normalize file name
+ "stats" -> from_json(col("add.stats"), statistics.statsSchema)
+ ))
+ .select(
+ "path",
+ "stats.numRecords",
+ "stats.minValues.*",
+ "stats.maxValues.*",
+ "stats.nullCount.*")
+ }
+
+}
diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseDecimalSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseDecimalSuite.scala
index 2663a8fb81fb..9988d5372943 100644
--- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseDecimalSuite.scala
+++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseDecimalSuite.scala
@@ -404,6 +404,18 @@ class GlutenClickHouseDecimalSuite
)
}
+ test("GLUTEN-8074 Fix adjust output constant column") {
+ val sql =
+ s"""
+ |select bround(1002.5786, 3),
+ | bround(-10.8, 0),
+ | bround(13.888888888889, 5)
+ |from $decimalTable
+ |WHERE bround(cast(decimal_field as decimal(30, 2)), 1) > 0 LIMIT 2;
+ |""".stripMargin
+ runQueryAndCompare(sql)(_ => {})
+ }
+
test("fix decimal32 with negative value") {
val schema = StructType.apply(
Seq(
diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseDeltaParquetWriteSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseDeltaParquetWriteSuite.scala
index 8871aaec18f5..2f55510a7b1f 100644
--- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseDeltaParquetWriteSuite.scala
+++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseDeltaParquetWriteSuite.scala
@@ -16,7 +16,10 @@
*/
package org.apache.gluten.execution
+import org.apache.gluten.GlutenConfig
+
import org.apache.spark.SparkConf
+import org.apache.spark.gluten.delta.DeltaStatsUtils
import org.apache.spark.sql.SaveMode
import org.apache.spark.sql.delta.files.TahoeFileIndex
import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper
@@ -49,7 +52,7 @@ class GlutenClickHouseDeltaParquetWriteSuite
.set("spark.sql.autoBroadcastJoinThreshold", "10MB")
.set("spark.sql.adaptive.enabled", "true")
.set("spark.sql.files.maxPartitionBytes", "20000000")
- .set("spark.gluten.sql.native.writer.enabled", "true")
+ .set(GlutenConfig.NATIVE_WRITER_ENABLED.key, spark35.toString)
.set("spark.sql.storeAssignmentPolicy", "legacy")
.setCHSettings("mergetree.merge_after_insert", false)
.set("spark.databricks.delta.retentionDurationCheck.enabled", "false")
@@ -59,67 +62,45 @@ class GlutenClickHouseDeltaParquetWriteSuite
createNotNullTPCHTablesInParquet(tablesPath)
}
+ private val q1SchemaString: String =
+ s""" l_orderkey bigint,
+ | l_partkey bigint,
+ | l_suppkey bigint,
+ | l_linenumber bigint,
+ | l_quantity double,
+ | l_extendedprice double,
+ | l_discount double,
+ | l_tax double,
+ | l_returnflag string,
+ | l_linestatus string,
+ | l_shipdate date,
+ | l_commitdate date,
+ | l_receiptdate date,
+ | l_shipinstruct string,
+ | l_shipmode string,
+ | l_comment string""".stripMargin
+
+ private def doInsert(drop: String, create: String, insert: String): Unit = {
+ spark.sql(drop)
+ spark.sql(create)
+ spark.sql(insert)
+ }
+ private def drop(table: String): String = s"DROP TABLE IF EXISTS $table"
+ private def createLineitem(table: String): String =
+ s"""CREATE TABLE IF NOT EXISTS $table ($q1SchemaString) USING delta
+ |TBLPROPERTIES (write.format.default = 'parquet')
+ |LOCATION '$basePath/$table'
+ |""".stripMargin
+
test("test parquet table write with the delta") {
- spark.sql(s"""
- |DROP TABLE IF EXISTS lineitem_delta_parquet;
- |""".stripMargin)
- spark.sql(s"""
- |CREATE TABLE IF NOT EXISTS lineitem_delta_parquet
- |(
- | l_orderkey bigint,
- | l_partkey bigint,
- | l_suppkey bigint,
- | l_linenumber bigint,
- | l_quantity double,
- | l_extendedprice double,
- | l_discount double,
- | l_tax double,
- | l_returnflag string,
- | l_linestatus string,
- | l_shipdate date,
- | l_commitdate date,
- | l_receiptdate date,
- | l_shipinstruct string,
- | l_shipmode string,
- | l_comment string
- |)
- |USING delta
- |TBLPROPERTIES (write.format.default = 'parquet')
- |LOCATION '$basePath/lineitem_delta_parquet'
- |""".stripMargin)
+ def insert(table: String): String =
+ s"insert into table $table select /*+ REPARTITION(5) */ * from lineitem"
- spark.sql(s"""
- | insert into table lineitem_delta_parquet
- | select /*+ REPARTITION(5) */ * from lineitem
- |""".stripMargin)
+ val table = "lineitem_delta_parquet"
+ doInsert(drop(table), createLineitem(table), insert(table))
- val sqlStr =
- s"""
- |SELECT
- | l_returnflag,
- | l_linestatus,
- | sum(l_quantity) AS sum_qty,
- | sum(l_extendedprice) AS sum_base_price,
- | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price,
- | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge,
- | avg(l_quantity) AS avg_qty,
- | avg(l_extendedprice) AS avg_price,
- | avg(l_discount) AS avg_disc,
- | count(*) AS count_order
- |FROM
- | lineitem_delta_parquet
- |WHERE
- | l_shipdate <= date'1998-09-02' - interval 1 day
- |GROUP BY
- | l_returnflag,
- | l_linestatus
- |ORDER BY
- | l_returnflag,
- | l_linestatus;
- |
- |""".stripMargin
- runTPCHQueryBySQL(1, sqlStr) {
+ runTPCHQueryBySQL(1, q1(table)) {
df =>
val plans = collect(df.queryExecution.executedPlan) {
case f: FileSourceScanExecTransformer => f
@@ -128,112 +109,70 @@ class GlutenClickHouseDeltaParquetWriteSuite
assert(plans.size === 4)
val parquetScan = plans(3).asInstanceOf[FileSourceScanExecTransformer]
- assert(parquetScan.nodeName.startsWith("Scan parquet "))
+ assert(parquetScan.nodeName.startsWith("ScanTransformer parquet "))
val fileIndex = parquetScan.relation.location.asInstanceOf[TahoeFileIndex]
val addFiles = fileIndex.matchingFiles(Nil, Nil)
assert(addFiles.size === 5)
}
+
+ if (spark35) {
+ val vanillaTable = "lineitem_delta_parquet_vanilla"
+ withSQLConf((GlutenConfig.NATIVE_WRITER_ENABLED.key, "false")) {
+ doInsert(drop(vanillaTable), createLineitem(vanillaTable), insert(vanillaTable))
+ }
+ val expected = DeltaStatsUtils
+ .statsDF(
+ spark,
+ s"$basePath/$vanillaTable/_delta_log/00000000000000000001.json",
+ q1SchemaString)
+ .collect()
+
+ checkAnswer(
+ DeltaStatsUtils.statsDF(
+ spark,
+ s"$basePath/$table/_delta_log/00000000000000000001.json",
+ q1SchemaString),
+ expected
+ )
+ }
}
test("test parquet insert overwrite with the delta") {
- spark.sql(s"""
- |DROP TABLE IF EXISTS lineitem_delta_parquet_insertoverwrite;
- |""".stripMargin)
-
- spark.sql(s"""
- |CREATE TABLE IF NOT EXISTS lineitem_delta_parquet_insertoverwrite
- |(
- | l_orderkey bigint,
- | l_partkey bigint,
- | l_suppkey bigint,
- | l_linenumber bigint,
- | l_quantity double,
- | l_extendedprice double,
- | l_discount double,
- | l_tax double,
- | l_returnflag string,
- | l_linestatus string,
- | l_shipdate date,
- | l_commitdate date,
- | l_receiptdate date,
- | l_shipinstruct string,
- | l_shipmode string,
- | l_comment string
- |)
- |USING delta
- |LOCATION '$basePath/lineitem_delta_parquet_insertoverwrite'
- |""".stripMargin)
-
- spark.sql(s"""
- | insert into table lineitem_delta_parquet_insertoverwrite
- | select * from lineitem
- |""".stripMargin)
+ def insert(table: String): String =
+ s"insert into table $table select * from lineitem"
+ val table = "lineitem_delta_parquet_insertoverwrite"
+ doInsert(drop(table), createLineitem(table), insert(table))
spark.sql(s"""
- | insert overwrite table lineitem_delta_parquet_insertoverwrite
+ | insert overwrite table $table
| select * from lineitem where mod(l_orderkey,2) = 1
|""".stripMargin)
- val sql2 =
- s"""
- | select count(*) from lineitem_delta_parquet_insertoverwrite
- |""".stripMargin
- assert(
- // total rows should remain unchanged
- spark.sql(sql2).collect().apply(0).get(0) === 300001
- )
+
+ // total rows should remain unchanged
+ assert(spark.sql(s"select count(*) from $table").collect().apply(0).get(0) === 300001)
}
test("test parquet insert overwrite partitioned table with small table, static with delta") {
- spark.sql(s"""
- |DROP TABLE IF EXISTS lineitem_delta_parquet_insertoverwrite2;
- |""".stripMargin)
-
- spark.sql(s"""
- |CREATE TABLE IF NOT EXISTS lineitem_delta_parquet_insertoverwrite2
- |(
- | l_orderkey bigint,
- | l_partkey bigint,
- | l_suppkey bigint,
- | l_linenumber bigint,
- | l_quantity double,
- | l_extendedprice double,
- | l_discount double,
- | l_tax double,
- | l_returnflag string,
- | l_linestatus string,
- | l_shipdate date,
- | l_commitdate date,
- | l_receiptdate date,
- | l_shipinstruct string,
- | l_shipmode string,
- | l_comment string
- |)
- |USING delta
- |PARTITIONED BY (l_shipdate)
- |LOCATION '$basePath/lineitem_delta_parquet_insertoverwrite2'
- |""".stripMargin)
-
- spark.sql(s"""
- | insert into table lineitem_delta_parquet_insertoverwrite2
- | select * from lineitem
- | where l_shipdate BETWEEN date'1993-01-01' AND date'1993-03-31'
- |""".stripMargin)
-
- spark.sql(
+ val table = "lineitem_delta_parquet_insertoverwrite2"
+ doInsert(
+ drop(table),
s"""
- | insert overwrite table lineitem_delta_parquet_insertoverwrite2
- | select * from lineitem where l_shipdate BETWEEN date'1993-02-01' AND date'1993-02-10'
- |""".stripMargin)
- val sql2 =
+ |CREATE TABLE IF NOT EXISTS $table ($q1SchemaString) USING delta
+ |PARTITIONED BY (l_shipdate)
+ |LOCATION '$basePath/$table'
+ |""".stripMargin,
s"""
- | select count(*) from lineitem_delta_parquet_insertoverwrite2
- |
+ | insert into table $table select * from lineitem
+ | where l_shipdate BETWEEN date'1993-01-01' AND date'1993-03-31'
|""".stripMargin
- assert(
- // total rows should remain unchanged
- spark.sql(sql2).collect().apply(0).get(0) === 2418
)
+ spark.sql(s"""
+ | insert overwrite table $table select * from lineitem
+ | where l_shipdate BETWEEN date'1993-02-01' AND date'1993-02-10'
+ |""".stripMargin)
+ // total rows should remain unchanged
+ assert(spark.sql(s"select count(*) from $table").collect().apply(0).get(0) === 2418)
}
test("test parquet insert overwrite partitioned table with small table, dynamic with delta") {
@@ -244,24 +183,7 @@ class GlutenClickHouseDeltaParquetWriteSuite
spark.sql(s"""
|CREATE TABLE IF NOT EXISTS lineitem_delta_parquet_insertoverwrite3
- |(
- | l_orderkey bigint,
- | l_partkey bigint,
- | l_suppkey bigint,
- | l_linenumber bigint,
- | l_quantity double,
- | l_extendedprice double,
- | l_discount double,
- | l_tax double,
- | l_returnflag string,
- | l_linestatus string,
- | l_shipdate date,
- | l_commitdate date,
- | l_receiptdate date,
- | l_shipinstruct string,
- | l_shipmode string,
- | l_comment string
- |)
+ |($q1SchemaString)
|USING delta
|PARTITIONED BY (l_shipdate)
|LOCATION '$basePath/lineitem_delta_parquet_insertoverwrite3'
@@ -297,24 +219,7 @@ class GlutenClickHouseDeltaParquetWriteSuite
spark.sql(s"""
|CREATE TABLE IF NOT EXISTS lineitem_delta_parquet_update
- |(
- | l_orderkey bigint,
- | l_partkey bigint,
- | l_suppkey bigint,
- | l_linenumber bigint,
- | l_quantity double,
- | l_extendedprice double,
- | l_discount double,
- | l_tax double,
- | l_returnflag string,
- | l_linestatus string,
- | l_shipdate date,
- | l_commitdate date,
- | l_receiptdate date,
- | l_shipinstruct string,
- | l_shipmode string,
- | l_comment string
- |)
+ |($q1SchemaString)
|USING delta
|LOCATION '$basePath/lineitem_delta_parquet_update'
|""".stripMargin)
@@ -348,7 +253,7 @@ class GlutenClickHouseDeltaParquetWriteSuite
assert(scanExec.size === 1)
val parquetScan = scanExec.head
- assert(parquetScan.nodeName.startsWith("Scan parquet"))
+ assert(parquetScan.nodeName.startsWith("ScanTransformer parquet"))
val fileIndex = parquetScan.relation.location.asInstanceOf[TahoeFileIndex]
val addFiles = fileIndex.matchingFiles(Nil, Nil)
@@ -360,10 +265,9 @@ class GlutenClickHouseDeltaParquetWriteSuite
| select count(*) from lineitem_delta_parquet_update
|
|""".stripMargin
- assert(
- // total rows should remain unchanged
- spark.sql(sql2).collect().apply(0).get(0) === 600572
- )
+
+ // total rows should remain unchanged
+ assert(spark.sql(sql2).collect().apply(0).get(0) === 600572)
}
test("test parquet table delete with the delta") {
@@ -373,24 +277,7 @@ class GlutenClickHouseDeltaParquetWriteSuite
spark.sql(s"""
|CREATE TABLE IF NOT EXISTS lineitem_delta_parquet_delete
- |(
- | l_orderkey bigint,
- | l_partkey bigint,
- | l_suppkey bigint,
- | l_linenumber bigint,
- | l_quantity double,
- | l_extendedprice double,
- | l_discount double,
- | l_tax double,
- | l_returnflag string,
- | l_linestatus string,
- | l_shipdate date,
- | l_commitdate date,
- | l_receiptdate date,
- | l_shipinstruct string,
- | l_shipmode string,
- | l_comment string
- |)
+ |($q1SchemaString)
|USING delta
|LOCATION '$basePath/lineitem_delta_parquet_delete'
|""".stripMargin)
@@ -440,24 +327,7 @@ class GlutenClickHouseDeltaParquetWriteSuite
spark.sql(s"""
|CREATE TABLE IF NOT EXISTS lineitem_delta_parquet_upsert
- |(
- | l_orderkey bigint,
- | l_partkey bigint,
- | l_suppkey bigint,
- | l_linenumber bigint,
- | l_quantity double,
- | l_extendedprice double,
- | l_discount double,
- | l_tax double,
- | l_returnflag string,
- | l_linestatus string,
- | l_shipdate date,
- | l_commitdate date,
- | l_receiptdate date,
- | l_shipinstruct string,
- | l_shipmode string,
- | l_comment string
- |)
+ |($q1SchemaString)
|USING delta
|LOCATION '$basePath/lineitem_delta_parquet_upsert'
|""".stripMargin)
@@ -542,24 +412,7 @@ class GlutenClickHouseDeltaParquetWriteSuite
spark.sql(s"""
|CREATE TABLE IF NOT EXISTS lineitem_delta_parquet_partition
- |(
- | l_orderkey bigint,
- | l_partkey bigint,
- | l_suppkey bigint,
- | l_linenumber bigint,
- | l_quantity double,
- | l_extendedprice double,
- | l_discount double,
- | l_tax double,
- | l_returnflag string,
- | l_linestatus string,
- | l_shipdate date,
- | l_commitdate date,
- | l_receiptdate date,
- | l_shipinstruct string,
- | l_shipmode string,
- | l_comment string
- |)
+ |($q1SchemaString)
|USING delta
|PARTITIONED BY (l_shipdate, l_returnflag)
|LOCATION '$basePath/lineitem_delta_parquet_partition'
@@ -635,32 +488,7 @@ class GlutenClickHouseDeltaParquetWriteSuite
| l_comment from lineitem
| where l_shipdate BETWEEN date'1993-02-01' AND date'1993-02-10'
|""".stripMargin)
- val sqlStr =
- s"""
- |SELECT
- | l_returnflag,
- | l_linestatus,
- | sum(l_quantity) AS sum_qty,
- | sum(l_extendedprice) AS sum_base_price,
- | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price,
- | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge,
- | avg(l_quantity) AS avg_qty,
- | avg(l_extendedprice) AS avg_price,
- | avg(l_discount) AS avg_disc,
- | count(*) AS count_order
- |FROM
- | lineitem_delta_parquet_partition
- |WHERE
- | l_shipdate <= date'1998-09-02' - interval 1 day
- |GROUP BY
- | l_returnflag,
- | l_linestatus
- |ORDER BY
- | l_returnflag,
- | l_linestatus;
- |
- |""".stripMargin
- runTPCHQueryBySQL(1, sqlStr, compareResult = false) {
+ runTPCHQueryBySQL(1, q1("lineitem_delta_parquet_partition"), compareResult = false) {
df =>
val result = df.collect()
assert(result.length === 2)
@@ -678,7 +506,7 @@ class GlutenClickHouseDeltaParquetWriteSuite
assert(scanExec.size === 1)
val parquetScan = scanExec.head
- assert(parquetScan.nodeName.startsWith("Scan parquet"))
+ assert(parquetScan.nodeName.startsWith("ScanTransformer parquet"))
assert(parquetScan.metrics("numFiles").value === 201)
val fileIndex = parquetScan.relation.location.asInstanceOf[TahoeFileIndex]
@@ -703,32 +531,7 @@ class GlutenClickHouseDeltaParquetWriteSuite
| as select * from lineitem
|""".stripMargin)
- val sqlStr =
- s"""
- |SELECT
- | l_returnflag,
- | l_linestatus,
- | sum(l_quantity) AS sum_qty,
- | sum(l_extendedprice) AS sum_base_price,
- | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price,
- | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge,
- | avg(l_quantity) AS avg_qty,
- | avg(l_extendedprice) AS avg_price,
- | avg(l_discount) AS avg_disc,
- | count(*) AS count_order
- |FROM
- | lineitem_delta_parquet_ctas1
- |WHERE
- | l_shipdate <= date'1998-09-02' - interval 1 day
- |GROUP BY
- | l_returnflag,
- | l_linestatus
- |ORDER BY
- | l_returnflag,
- | l_linestatus;
- |
- |""".stripMargin
- runTPCHQueryBySQL(1, sqlStr) {
+ runTPCHQueryBySQL(1, q1("lineitem_delta_parquet_ctas1")) {
df =>
val scanExec = collect(df.queryExecution.executedPlan) {
case f: FileSourceScanExecTransformer => f
@@ -736,7 +539,7 @@ class GlutenClickHouseDeltaParquetWriteSuite
assert(scanExec.size === 1)
val parquetScan = scanExec.head
- assert(parquetScan.nodeName.startsWith("Scan parquet"))
+ assert(parquetScan.nodeName.startsWith("ScanTransformer parquet"))
val fileIndex = parquetScan.relation.location.asInstanceOf[TahoeFileIndex]
val addFiles = fileIndex.matchingFiles(Nil, Nil)
@@ -756,65 +559,14 @@ class GlutenClickHouseDeltaParquetWriteSuite
|LOCATION '$basePath/lineitem_mergetree_ctas2'
| as select * from lineitem
|""".stripMargin)
-
- val sqlStr =
- s"""
- |SELECT
- | l_returnflag,
- | l_linestatus,
- | sum(l_quantity) AS sum_qty,
- | sum(l_extendedprice) AS sum_base_price,
- | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price,
- | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge,
- | avg(l_quantity) AS avg_qty,
- | avg(l_extendedprice) AS avg_price,
- | avg(l_discount) AS avg_disc,
- | count(*) AS count_order
- |FROM
- | lineitem_delta_parquet_ctas2
- |WHERE
- | l_shipdate <= date'1998-09-02' - interval 1 day
- |GROUP BY
- | l_returnflag,
- | l_linestatus
- |ORDER BY
- | l_returnflag,
- | l_linestatus;
- |
- |""".stripMargin
- runTPCHQueryBySQL(1, sqlStr) { _ => {} }
-
+ runTPCHQueryBySQL(1, q1("lineitem_delta_parquet_ctas2")) { _ => {} }
}
test("test path based parquet write with the delta") {
val dataPath = s"$basePath/lineitem_delta_parquet_filebased"
clearDataPath(dataPath)
-
- val sourceDF = spark.sql(s"""
- |select * from lineitem
- |""".stripMargin)
-
- spark.sql(s"""
- |CREATE TABLE delta.`$dataPath` (
- | l_orderkey bigint,
- | l_partkey bigint,
- | l_suppkey bigint,
- | l_linenumber bigint,
- | l_quantity double,
- | l_extendedprice double,
- | l_discount double,
- | l_tax double,
- | l_returnflag string,
- | l_linestatus string,
- | l_shipdate date,
- | l_commitdate date,
- | l_receiptdate date,
- | l_shipinstruct string,
- | l_shipmode string,
- | l_comment string
- |)
- |USING delta
- |""".stripMargin)
+ val sourceDF = spark.sql("select * from lineitem")
+ spark.sql(s"CREATE TABLE delta.`$dataPath` ($q1SchemaString) USING delta")
sourceDF.write
.format("delta")
@@ -829,32 +581,7 @@ class GlutenClickHouseDeltaParquetWriteSuite
.mode(SaveMode.Overwrite)
.save(dataPath)
- val sqlStr =
- s"""
- |SELECT
- | l_returnflag,
- | l_linestatus,
- | sum(l_quantity) AS sum_qty,
- | sum(l_extendedprice) AS sum_base_price,
- | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price,
- | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge,
- | avg(l_quantity) AS avg_qty,
- | avg(l_extendedprice) AS avg_price,
- | avg(l_discount) AS avg_disc,
- | count(*) AS count_order
- |FROM
- | delta.`$dataPath`
- |WHERE
- | l_shipdate <= date'1998-09-02' - interval 1 day
- |GROUP BY
- | l_returnflag,
- | l_linestatus
- |ORDER BY
- | l_returnflag,
- | l_linestatus;
- |
- |""".stripMargin
- runTPCHQueryBySQL(1, sqlStr) {
+ runTPCHQueryBySQL(1, q1(s"delta.`$dataPath`")) {
df =>
val plans = collect(df.queryExecution.executedPlan) {
case f: FileSourceScanExecTransformer => f
@@ -863,7 +590,7 @@ class GlutenClickHouseDeltaParquetWriteSuite
assert(plans.size === 4)
val parquetScan = plans(3).asInstanceOf[FileSourceScanExecTransformer]
- assert(parquetScan.nodeName.startsWith("Scan parquet"))
+ assert(parquetScan.nodeName.startsWith("ScanTransformer parquet"))
val fileIndex = parquetScan.relation.location.asInstanceOf[TahoeFileIndex]
val addFiles = fileIndex.matchingFiles(Nil, Nil)
@@ -975,7 +702,7 @@ class GlutenClickHouseDeltaParquetWriteSuite
assert(scanExec.size === 1)
val parquetScan = scanExec.head
- assert(parquetScan.nodeName.startsWith("Scan parquet"))
+ assert(parquetScan.nodeName.startsWith("ScanTransformer parquet"))
val fileIndex = parquetScan.relation.location.asInstanceOf[TahoeFileIndex]
val addFiles = fileIndex.matchingFiles(Nil, Nil)
@@ -997,7 +724,7 @@ class GlutenClickHouseDeltaParquetWriteSuite
assert(scanExec.size === 1)
val parquetScan = scanExec.head
- assert(parquetScan.nodeName.startsWith("Scan parquet"))
+ assert(parquetScan.nodeName.startsWith("ScanTransformer parquet"))
val fileIndex = parquetScan.relation.location.asInstanceOf[TahoeFileIndex]
val addFiles = fileIndex.matchingFiles(Nil, Nil)
@@ -1149,32 +876,7 @@ class GlutenClickHouseDeltaParquetWriteSuite
.mode(SaveMode.Append)
.save(dataPath)
- val sqlStr =
- s"""
- |SELECT
- | l_returnflag,
- | l_linestatus,
- | sum(l_quantity) AS sum_qty,
- | sum(l_extendedprice) AS sum_base_price,
- | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price,
- | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge,
- | avg(l_quantity) AS avg_qty,
- | avg(l_extendedprice) AS avg_price,
- | avg(l_discount) AS avg_disc,
- | count(*) AS count_order
- |FROM
- | delta.`$dataPath`
- |WHERE
- | l_shipdate <= date'1998-09-02' - interval 1 day
- |GROUP BY
- | l_returnflag,
- | l_linestatus
- |ORDER BY
- | l_returnflag,
- | l_linestatus;
- |
- |""".stripMargin
- runTPCHQueryBySQL(1, sqlStr, compareResult = false) {
+ runTPCHQueryBySQL(1, q1(s"delta.`$dataPath`"), compareResult = false) {
df =>
val result = df.collect()
assert(result.length === 2)
@@ -1192,7 +894,7 @@ class GlutenClickHouseDeltaParquetWriteSuite
assert(scanExec.size === 1)
val parquetScan = scanExec.head
- assert(parquetScan.nodeName.startsWith("Scan parquet"))
+ assert(parquetScan.nodeName.startsWith("ScanTransformer parquet"))
assert(parquetScan.metrics("numFiles").value === 200)
val fileIndex = parquetScan.relation.location.asInstanceOf[TahoeFileIndex]
@@ -1215,35 +917,10 @@ class GlutenClickHouseDeltaParquetWriteSuite
| as select * from lineitem
|""".stripMargin)
- val sqlStr =
- s"""
- |SELECT
- | l_returnflag,
- | l_linestatus,
- | sum(l_quantity) AS sum_qty,
- | sum(l_extendedprice) AS sum_base_price,
- | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price,
- | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge,
- | avg(l_quantity) AS avg_qty,
- | avg(l_extendedprice) AS avg_price,
- | avg(l_discount) AS avg_disc,
- | count(*) AS count_order
- |FROM
- | delta.`$dataPath`
- |WHERE
- | l_shipdate <= date'1998-09-02' - interval 1 day
- |GROUP BY
- | l_returnflag,
- | l_linestatus
- |ORDER BY
- | l_returnflag,
- | l_linestatus;
- |
- |""".stripMargin
- runTPCHQueryBySQL(1, sqlStr) { _ => {} }
+ runTPCHQueryBySQL(1, q1(s"delta.`$dataPath`")) { _ => {} }
}
- testSparkVersionLE33("test parquet optimize basic") {
+ test("test parquet optimize basic") {
withSQLConf("spark.databricks.delta.optimize.maxFileSize" -> "20000000") {
spark.sql(s"""
|DROP TABLE IF EXISTS lineitem_delta_parquet_optimize;
@@ -1278,7 +955,7 @@ class GlutenClickHouseDeltaParquetWriteSuite
}
}
- testSparkVersionLE33("test parquet optimize partitioned by one low card column") {
+ test("test parquet optimize partitioned by one low card column") {
spark.sql(s"""
|DROP TABLE IF EXISTS lineitem_delta_parquet_optimize_p2;
|""".stripMargin)
@@ -1317,7 +994,7 @@ class GlutenClickHouseDeltaParquetWriteSuite
assert(ret2.apply(0).get(0) === 600572)
}
- testSparkVersionLE33("test parquet optimize parallel delete") {
+ test("test parquet optimize parallel delete") {
withSQLConf("spark.databricks.delta.vacuum.parallelDelete.enabled" -> "true") {
spark.sql(s"""
|DROP TABLE IF EXISTS lineitem_delta_parquet_optimize_p4;
diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseFileFormatSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseExcelFormatSuite.scala
similarity index 99%
rename from backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseFileFormatSuite.scala
rename to backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseExcelFormatSuite.scala
index 88a34a786a8c..7c444d572b69 100644
--- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseFileFormatSuite.scala
+++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseExcelFormatSuite.scala
@@ -16,6 +16,7 @@
*/
package org.apache.gluten.execution
+import org.apache.gluten.GlutenConfig
import org.apache.gluten.backendsapi.clickhouse.CHConf
import org.apache.spark.SparkConf
@@ -46,7 +47,7 @@ case class AllDataTypesWithNonPrimitiveType(
// data: (Seq[Int], (Int, String))
)
-class GlutenClickHouseFileFormatSuite
+class GlutenClickHouseExcelFormatSuite
extends GlutenClickHouseTPCHAbstractSuite
with AdaptiveSparkPlanHelper {
import testImplicits._
@@ -76,7 +77,7 @@ class GlutenClickHouseFileFormatSuite
// in this case, FakeRowAdaptor does R2C
test("parquet native writer writing a in memory DF") {
- withSQLConf(("spark.gluten.sql.native.writer.enabled", "true")) {
+ withSQLConf((GlutenConfig.NATIVE_WRITER_ENABLED.key, "true")) {
val filePath = basePath + "/native_parquet_test"
val format = "parquet"
@@ -99,7 +100,7 @@ class GlutenClickHouseFileFormatSuite
// in this case, FakeRowAdaptor only wrap&transfer
test("parquet native writer writing a DF from file") {
- withSQLConf(("spark.gluten.sql.native.writer.enabled", "true")) {
+ withSQLConf((GlutenConfig.NATIVE_WRITER_ENABLED.key, "true")) {
val filePath = basePath + "/native_parquet_test"
val format = "parquet"
@@ -123,7 +124,7 @@ class GlutenClickHouseFileFormatSuite
// in this case, FakeRowAdaptor only wrap&transfer
test("parquet native writer writing a DF from an aggregate") {
- withSQLConf(("spark.gluten.sql.native.writer.enabled", "true")) {
+ withSQLConf((GlutenConfig.NATIVE_WRITER_ENABLED.key, "true")) {
val filePath = basePath + "/native_parquet_test_agg"
val format = "parquet"
@@ -273,7 +274,8 @@ class GlutenClickHouseFileFormatSuite
StructField.apply("boolean_field", BooleanType, nullable = true),
StructField.apply("decimal_field", DecimalType.apply(10, 2), nullable = true),
StructField.apply("date_field", DateType, nullable = true),
- StructField.apply("timestamp_field", TimestampType, nullable = true)
+ StructField.apply("timestamp_field", TimestampType, nullable = true),
+ StructField.apply("boolean_field2", BooleanType, nullable = true)
))
val options = new util.HashMap[String, String]()
@@ -1479,7 +1481,7 @@ class GlutenClickHouseFileFormatSuite
| from $format.`$tablePath`
| where long_field > 30
|""".stripMargin
- withSQLConf(("spark.gluten.sql.native.writer.enabled", "true")) {
+ withSQLConf((GlutenConfig.NATIVE_WRITER_ENABLED.key, "true")) {
testFileFormatBase(tablePath, format, sql, df => {})
}
}
diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHAbstractSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHAbstractSuite.scala
index 0205bae2d612..fbaebdecaaf6 100644
--- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHAbstractSuite.scala
+++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHAbstractSuite.scala
@@ -621,4 +621,42 @@ abstract class GlutenClickHouseTPCHAbstractSuite
checkDataFrame(noFallBack, customCheck, df)
}
+ def q1(tableName: String): String =
+ s"""
+ |SELECT
+ | l_returnflag,
+ | l_linestatus,
+ | sum(l_quantity) AS sum_qty,
+ | sum(l_extendedprice) AS sum_base_price,
+ | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price,
+ | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge,
+ | avg(l_quantity) AS avg_qty,
+ | avg(l_extendedprice) AS avg_price,
+ | avg(l_discount) AS avg_disc,
+ | count(*) AS count_order
+ |FROM
+ | $tableName
+ |WHERE
+ | l_shipdate <= date'1998-09-02' - interval 1 day
+ |GROUP BY
+ | l_returnflag,
+ | l_linestatus
+ |ORDER BY
+ | l_returnflag,
+ | l_linestatus;
+ |
+ |""".stripMargin
+
+ def q6(tableName: String): String =
+ s"""
+ |SELECT
+ | sum(l_extendedprice * l_discount) AS revenue
+ |FROM
+ | $tableName
+ |WHERE
+ | l_shipdate >= date'1994-01-01'
+ | AND l_shipdate < date'1994-01-01' + interval 1 year
+ | AND l_discount BETWEEN 0.06 - 0.01 AND 0.06 + 0.01
+ | AND l_quantity < 24
+ |""".stripMargin
}
diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHBucketSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHBucketSuite.scala
index 6a09bf494217..90e09e75f1ff 100644
--- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHBucketSuite.scala
+++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHBucketSuite.scala
@@ -40,6 +40,7 @@ class GlutenClickHouseTPCHBucketSuite
override protected val queriesResults: String = rootPath + "bucket-queries-output"
override protected def sparkConf: SparkConf = {
+ import org.apache.gluten.backendsapi.clickhouse.CHConf._
super.sparkConf
.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager")
.set("spark.io.compression.codec", "LZ4")
@@ -47,6 +48,7 @@ class GlutenClickHouseTPCHBucketSuite
.set("spark.sql.autoBroadcastJoinThreshold", "-1") // for test bucket join
.set("spark.sql.adaptive.enabled", "true")
.set("spark.gluten.sql.columnar.backend.ch.shuffle.hash.algorithm", "sparkMurmurHash3_32")
+ .setCHConfig("enable_grace_aggregate_spill_test", "true")
}
override protected val createNullableTables = true
diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHSuite.scala
index a56f45d1ba3d..65a01dea3073 100644
--- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHSuite.scala
+++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHSuite.scala
@@ -47,7 +47,7 @@ class GlutenClickHouseTPCHSuite extends GlutenClickHouseTPCHAbstractSuite {
}
assert(scanExec.size == 1)
- assert(scanExec(0).nodeName.startsWith("Scan mergetree"))
+ assert(scanExec(0).nodeName.startsWith("ScanTransformer mergetree"))
val sortExec = df.queryExecution.executedPlan.collect {
case sortExec: SortExecTransformer => sortExec
@@ -563,5 +563,32 @@ class GlutenClickHouseTPCHSuite extends GlutenClickHouseTPCHAbstractSuite {
compareResultsAgainstVanillaSpark(sql, true, { _ => })
spark.sql("drop table t1")
}
+
+ test("GLUTEN-7780 fix split diff") {
+ val sql = "select split(concat('a|b|c', cast(id as string)), '\\|')" +
+ ", split(concat('a|b|c', cast(id as string)), '\\\\|')" +
+ ", split(concat('a|b|c', cast(id as string)), '|') from range(10)"
+ compareResultsAgainstVanillaSpark(sql, true, { _ => })
+ }
+ test("GLUTEN-8142 duplicated columns in group by") {
+ sql("create table test_8142 (day string, rtime int, uid string, owner string) using parquet")
+ sql("insert into test_8142 values ('2024-09-01', 123, 'user1', 'owner1')")
+ sql("insert into test_8142 values ('2024-09-01', 123, 'user1', 'owner1')")
+ sql("insert into test_8142 values ('2024-09-02', 567, 'user2', 'owner2')")
+ compareResultsAgainstVanillaSpark(
+ """
+ |select days, rtime, uid, owner, day1
+ |from (
+ | select day1 as days, rtime, uid, owner, day1
+ | from (
+ | select distinct coalesce(day, "today") as day1, rtime, uid, owner
+ | from test_8142 where day = '2024-09-01'
+ | )) group by days, rtime, uid, owner, day1
+ |""".stripMargin,
+ true,
+ { _ => }
+ )
+ sql("drop table test_8142")
+ }
}
// scalastyle:off line.size.limit
diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseWholeStageTransformerSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseWholeStageTransformerSuite.scala
index 0bd19dd97172..d7b277edfac5 100644
--- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseWholeStageTransformerSuite.scala
+++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseWholeStageTransformerSuite.scala
@@ -17,6 +17,7 @@
package org.apache.gluten.execution
import org.apache.gluten.GlutenConfig
+import org.apache.gluten.backendsapi.clickhouse.RuntimeConfig
import org.apache.gluten.utils.UTSystemParameters
import org.apache.spark.{SPARK_VERSION_SHORT, SparkConf}
@@ -41,20 +42,20 @@ class GlutenClickHouseWholeStageTransformerSuite extends WholeStageTransformerSu
}
val SPARK_DIR_NAME: String = sparkVersion.replace(".", "-")
- val S3_METADATA_PATH = s"/tmp/metadata/s3/$SPARK_DIR_NAME/"
+ val S3_METADATA_PATH = s"/tmp/metadata/s3/$SPARK_DIR_NAME"
val S3_CACHE_PATH = s"/tmp/s3_cache/$SPARK_DIR_NAME/"
val S3_ENDPOINT = "s3://127.0.0.1:9000/"
val MINIO_ENDPOINT: String = S3_ENDPOINT.replace("s3", "http")
val BUCKET_NAME: String = SPARK_DIR_NAME
val WHOLE_PATH: String = MINIO_ENDPOINT + BUCKET_NAME + "/"
- val HDFS_METADATA_PATH = s"/tmp/metadata/hdfs/$SPARK_DIR_NAME/"
+ val HDFS_METADATA_PATH = s"/tmp/metadata/hdfs/$SPARK_DIR_NAME"
val HDFS_CACHE_PATH = s"/tmp/hdfs_cache/$SPARK_DIR_NAME/"
val HDFS_URL_ENDPOINT = "hdfs://127.0.0.1:8020"
val HDFS_URL = s"$HDFS_URL_ENDPOINT/$SPARK_DIR_NAME"
- val S3_ACCESS_KEY = "BypTYzcXOlfr03FFIvt4"
- val S3_SECRET_KEY = "K9MDaGItPSaphorZM8t4hXf30gHF9dBWi6L2dK5E"
+ val S3_ACCESS_KEY = "minioadmin"
+ val S3_SECRET_KEY = "minioadmin"
val CH_DEFAULT_STORAGE_DIR = "/data"
@@ -82,7 +83,8 @@ class GlutenClickHouseWholeStageTransformerSuite extends WholeStageTransformerSu
.set("spark.gluten.sql.enable.native.validation", "false")
.set("spark.sql.warehouse.dir", warehouse)
.setCHConfig("user_defined_path", "/tmp/user_defined")
- .setCHConfig("path", UTSystemParameters.diskOutputDataPath)
+ .set(RuntimeConfig.PATH.key, UTSystemParameters.diskOutputDataPath)
+ .set(RuntimeConfig.TMP_PATH.key, s"/tmp/libch/$SPARK_DIR_NAME")
if (UTSystemParameters.testMergeTreeOnObjectStorage) {
conf
.set("spark.hadoop.fs.s3a.access.key", S3_ACCESS_KEY)
diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenFunctionValidateSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenFunctionValidateSuite.scala
index 08342e588730..dbe8852290aa 100644
--- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenFunctionValidateSuite.scala
+++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenFunctionValidateSuite.scala
@@ -855,4 +855,9 @@ class GlutenFunctionValidateSuite extends GlutenClickHouseWholeStageTransformerS
compareResultsAgainstVanillaSpark(sql, true, { _ => })
}
}
+
+ test("GLUTEN-7796 cast bool to string") {
+ val sql = "select cast(id % 2 = 1 as string) from range(10)"
+ compareResultsAgainstVanillaSpark(sql, true, { _ => })
+ }
}
diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/compatibility/GlutenClickhouseFunctionSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/compatibility/GlutenClickhouseFunctionSuite.scala
index 2cfe935ef7e0..2437ffd03550 100644
--- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/compatibility/GlutenClickhouseFunctionSuite.scala
+++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/compatibility/GlutenClickhouseFunctionSuite.scala
@@ -17,7 +17,7 @@
package org.apache.gluten.execution.compatibility
import org.apache.gluten.GlutenConfig
-import org.apache.gluten.execution.GlutenClickHouseTPCHAbstractSuite
+import org.apache.gluten.execution.{GlutenClickHouseTPCHAbstractSuite, ProjectExecTransformer}
import org.apache.gluten.utils.UTSystemParameters
import org.apache.spark.SparkConf
@@ -64,7 +64,7 @@ class GlutenClickhouseFunctionSuite extends GlutenClickHouseTPCHAbstractSuite {
test("test uuid - write and read") {
withSQLConf(
- ("spark.gluten.sql.native.writer.enabled", "true"),
+ (GlutenConfig.NATIVE_WRITER_ENABLED.key, "true"),
(GlutenConfig.GLUTEN_ENABLED.key, "true")) {
withTable("uuid_test") {
spark.sql("create table if not exists uuid_test (id string) using parquet")
@@ -269,6 +269,22 @@ class GlutenClickhouseFunctionSuite extends GlutenClickHouseTPCHAbstractSuite {
}
}
+ test("GLUTEN-7594: cast const map to string") {
+ withSQLConf(
+ (
+ "spark.sql.optimizer.excludedRules",
+ "org.apache.spark.sql.catalyst.optimizer.ConstantFolding," +
+ "org.apache.spark.sql.catalyst.optimizer.NullPropagation")) {
+ runQueryAndCompare(
+ """
+ |select cast(map(1,'2') as string)
+ |""".stripMargin,
+ true,
+ false
+ )(checkGlutenOperatorMatch[ProjectExecTransformer])
+ }
+ }
+
test("GLUTEN-7550 get_json_object in IN") {
withTable("test_7550") {
sql("create table test_7550(a string) using parquet")
@@ -385,4 +401,18 @@ class GlutenClickhouseFunctionSuite extends GlutenClickHouseTPCHAbstractSuite {
}
}
+ test("GLUTEN-8148: Fix corr with NaN") {
+ withTable("corr_nan") {
+ sql("create table if not exists corr_nan (x double, y double) using parquet")
+ sql("insert into corr_nan values(0,1)")
+ compareResultsAgainstVanillaSpark(
+ """
+ |select corr(x,y), corr(y,x) from corr_nan
+ """.stripMargin,
+ true,
+ { _ => }
+ )
+ }
+ }
+
}
diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseHiveTableSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseHiveTableSuite.scala
index 4e2b5ad63e0a..14d6ff53649f 100644
--- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseHiveTableSuite.scala
+++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseHiveTableSuite.scala
@@ -17,7 +17,7 @@
package org.apache.gluten.execution.hive
import org.apache.gluten.GlutenConfig
-import org.apache.gluten.execution.{GlutenClickHouseWholeStageTransformerSuite, ProjectExecTransformer, TransformSupport}
+import org.apache.gluten.execution.{FileSourceScanExecTransformer, GlutenClickHouseWholeStageTransformerSuite, ProjectExecTransformer, TransformSupport}
import org.apache.gluten.test.AllDataTypesWithComplexType
import org.apache.gluten.utils.UTSystemParameters
@@ -28,6 +28,7 @@ import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper
import org.apache.spark.sql.execution.datasources.v2.clickhouse.ClickHouseConfig
import org.apache.spark.sql.hive.HiveTableScanExecTransformer
import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{ArrayType, StructType}
import org.apache.hadoop.fs.Path
@@ -542,6 +543,37 @@ class GlutenClickHouseHiveTableSuite
)
}
+ test("GLUTEN-7700: test hive table with partition values contain space") {
+ val tbl = "test_7700"
+ val create_table_sql =
+ s"""
+ |create table if not exists $tbl (
+ | id int
+ |) partitioned by (itime string)
+ |stored as orc;
+ |""".stripMargin
+ val insert_sql =
+ s"""
+ |insert overwrite table $tbl partition (itime = '2024-10-24 10:02:04')
+ |select id from range(3)
+ |""".stripMargin
+ val select_sql =
+ s"""
+ |select * from $tbl
+ |""".stripMargin
+ val drop_sql = s"drop table if exists $tbl"
+
+ spark.sql(create_table_sql)
+ spark.sql(insert_sql)
+
+ compareResultsAgainstVanillaSpark(
+ select_sql,
+ compareResult = true,
+ df => assert(df.count() == 3)
+ )
+ spark.sql(drop_sql)
+ }
+
test("test hive compressed txt table") {
withSQLConf(SQLConf.FILES_MAX_PARTITION_BYTES.key -> "11") {
Seq("DefaultCodec", "BZip2Codec").foreach {
@@ -1419,4 +1451,191 @@ class GlutenClickHouseHiveTableSuite
spark.sql("DROP TABLE test_tbl_7054")
}
+ test("Nested column pruning for Project(Filter(Generate))") {
+ spark.sql("drop table if exists aj")
+ spark.sql(
+ """
+ |CREATE TABLE if not exists aj (
+ | country STRING,
+ | event STRUCT, event_id:STRING, event_info:MAP>
+ |)
+ |USING orc
+ """.stripMargin)
+
+ spark.sql("""
+ |INSERT INTO aj VALUES
+ | ('USA', named_struct('time', 1622547800, 'lng', -122, 'lat', 37, 'net',
+ | 'wifi', 'log_extra', map('key1', 'value1'), 'event_id', 'event1',
+ | 'event_info', map('tab_type', '5', 'action', '13'))),
+ | ('Canada', named_struct('time', 1622547801, 'lng', -79, 'lat', 43, 'net',
+ | '4g', 'log_extra', map('key2', 'value2'), 'event_id', 'event2',
+ | 'event_info', map('tab_type', '4', 'action', '12')))
+ """.stripMargin)
+
+ val sql = """
+ | SELECT * FROM (
+ | SELECT
+ | game_name,
+ | CASE WHEN
+ | event.event_info['tab_type'] IN (5) THEN '1' ELSE '0' END AS entrance
+ | FROM aj
+ | LATERAL VIEW explode(split(nvl(event.event_info['game_name'],'0'),','))
+ | as game_name
+ | WHERE event.event_info['action'] IN (13)
+ |) WHERE game_name = 'xxx'
+ """.stripMargin
+
+ compareResultsAgainstVanillaSpark(
+ sql,
+ compareResult = true,
+ df => {
+ val scan = df.queryExecution.executedPlan.collect {
+ case scan: FileSourceScanExecTransformer => scan
+ }.head
+ val fieldType = scan.schema.fields.head.dataType.asInstanceOf[StructType]
+ assert(fieldType.size == 1)
+ }
+ )
+
+ spark.sql("drop table if exists aj")
+ }
+
+ test("Nested column pruning for Project(Filter(Generate)) on generator") {
+ def assertFieldSizeAfterPruning(sql: String, expectSize: Int): Unit = {
+ compareResultsAgainstVanillaSpark(
+ sql,
+ compareResult = true,
+ df => {
+ val scan = df.queryExecution.executedPlan.collect {
+ case scan: FileSourceScanExecTransformer => scan
+ }.head
+
+ val fieldType =
+ scan.schema.fields.head.dataType
+ .asInstanceOf[ArrayType]
+ .elementType
+ .asInstanceOf[StructType]
+ assert(fieldType.size == expectSize)
+ }
+ )
+ }
+
+ spark.sql("drop table if exists ajog")
+ spark.sql(
+ """
+ |CREATE TABLE if not exists ajog (
+ | country STRING,
+ | events ARRAY, event_id:STRING, event_info:MAP>>
+ |)
+ |USING orc
+ """.stripMargin)
+
+ spark.sql("""
+ |INSERT INTO ajog VALUES
+ | ('USA', array(named_struct('time', 1622547800, 'lng', -122, 'lat', 37, 'net',
+ | 'wifi', 'log_extra', map('key1', 'value1'), 'event_id', 'event1',
+ | 'event_info', map('tab_type', '5', 'action', '13')))),
+ | ('Canada', array(named_struct('time', 1622547801, 'lng', -79, 'lat', 43, 'net',
+ | '4g', 'log_extra', map('key2', 'value2'), 'event_id', 'event2',
+ | 'event_info', map('tab_type', '4', 'action', '12'))))
+ """.stripMargin)
+
+ // Test nested column pruning on generator with single field extracted
+ val sql1 = """
+ |select
+ |case when event.event_info['tab_type'] in (5) then '1' else '0' end as entrance
+ |from ajog
+ |lateral view explode(events) as event
+ |where event.event_info['action'] in (13)
+ """.stripMargin
+ assertFieldSizeAfterPruning(sql1, 1)
+
+ // Test nested column pruning on generator with multiple field extracted,
+ // which resolves SPARK-34956
+ val sql2 = """
+ |select event.event_id,
+ |case when event.event_info['tab_type'] in (5) then '1' else '0' end as entrance
+ |from ajog
+ |lateral view explode(events) as event
+ |where event.event_info['action'] in (13)
+ """.stripMargin
+ assertFieldSizeAfterPruning(sql2, 2)
+
+ // Test nested column pruning with two adjacent generate operator
+ val sql3 = """
+ |SELECT
+ |abflag,
+ |event.event_info,
+ |event.log_extra
+ |FROM
+ |ajog
+ |LATERAL VIEW EXPLODE(events) AS event
+ |LATERAL VIEW EXPLODE(split(event.log_extra['key1'], ',')) AS abflag
+ |WHERE
+ |event.event_id = 'event1'
+ |AND event.event_info['tab_type'] IS NOT NULL
+ |AND event.event_info['tab_type'] != ''
+ |AND event.log_extra['key1'] = 'value1'
+ |LIMIT 100;
+ """.stripMargin
+ assertFieldSizeAfterPruning(sql3, 3)
+
+ spark.sql("drop table if exists ajog")
+ }
+
+ test("test hive table scan nested column pruning") {
+ val json_table_name = "test_tbl_7267_json"
+ val pq_table_name = "test_tbl_7267_pq"
+ val create_table_sql =
+ s"""
+ | create table if not exists %s(
+ | id bigint,
+ | d1 STRUCT>>,
+ | d2 STRUCT>>,
+ | day string,
+ | hour string
+ | ) partitioned by(day, hour)
+ |""".stripMargin
+ val create_table_json = create_table_sql.format(json_table_name) +
+ s"""
+ | ROW FORMAT SERDE 'org.apache.hive.hcatalog.data.JsonSerDe'
+ | STORED AS INPUTFORMAT 'org.apache.hadoop.mapred.TextInputFormat'
+ | OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat'
+ |""".stripMargin
+ val create_table_pq = create_table_sql.format(pq_table_name) + " Stored as PARQUET"
+ val insert_sql =
+ """
+ | insert into %s values(1,
+ | named_struct('c', 'c123', 'd', array(named_struct('x', 'x123', 'y', 'y123'))),
+ | named_struct('c', 'c124', 'd', map('m124', named_struct('x', 'x124', 'y', 'y124'))),
+ | '2024-09-26', '12'
+ | )
+ |""".stripMargin
+ val select_sql =
+ "select id, d1.c, d1.d[0].x, d2.d['m124'].y from %s where day = '2024-09-26' and hour = '12'"
+ val table_names = Array.apply(json_table_name, pq_table_name)
+ val create_table_sqls = Array.apply(create_table_json, create_table_pq)
+ for (i <- table_names.indices) {
+ val table_name = table_names(i)
+ val create_table = create_table_sqls(i)
+ spark.sql(create_table)
+ spark.sql(insert_sql.format(table_name))
+ withSQLConf(("spark.sql.hive.convertMetastoreParquet" -> "false")) {
+ compareResultsAgainstVanillaSpark(
+ select_sql.format(table_name),
+ compareResult = true,
+ df => {
+ val scan = collect(df.queryExecution.executedPlan) {
+ case l: HiveTableScanExecTransformer => l
+ }
+ assert(scan.size == 1)
+ }
+ )
+ }
+ spark.sql("drop table if exists %s".format(table_name))
+ }
+ }
+
}
diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseNativeWriteTableSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseNativeWriteTableSuite.scala
index 53aef16d143e..16ed302a02f4 100644
--- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseNativeWriteTableSuite.scala
+++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseNativeWriteTableSuite.scala
@@ -553,7 +553,7 @@ class GlutenClickHouseNativeWriteTableSuite
// spark write does not support bucketed table
// https://issues.apache.org/jira/browse/SPARK-19256
val table_name = table_name_template.format(format)
- writeAndCheckRead(origin_table, table_name, fields_.keys.toSeq, isSparkVersionLE("3.3")) {
+ writeAndCheckRead(origin_table, table_name, fields_.keys.toSeq) {
fields =>
spark
.table("origin_table")
@@ -589,8 +589,9 @@ class GlutenClickHouseNativeWriteTableSuite
("byte_field", "byte"),
("boolean_field", "boolean"),
("decimal_field", "decimal(23,12)"),
- ("date_field", "date"),
- ("timestamp_field", "timestamp")
+ ("date_field", "date")
+ // ("timestamp_field", "timestamp")
+ // FIXME https://github.com/apache/incubator-gluten/issues/8053
)
val origin_table = "origin_table"
withSource(genTestData(), origin_table) {
@@ -598,7 +599,7 @@ class GlutenClickHouseNativeWriteTableSuite
format =>
val table_name = table_name_template.format(format)
val testFields = fields.keys.toSeq
- writeAndCheckRead(origin_table, table_name, testFields, isSparkVersionLE("3.3")) {
+ writeAndCheckRead(origin_table, table_name, testFields) {
fields =>
spark
.table(origin_table)
@@ -610,7 +611,7 @@ class GlutenClickHouseNativeWriteTableSuite
}
val table_name_vanilla = table_name_vanilla_template.format(format)
spark.sql(s"drop table IF EXISTS $table_name_vanilla")
- withSQLConf(("spark.gluten.sql.native.writer.enabled", "false")) {
+ withSQLConf((GlutenConfig.NATIVE_WRITER_ENABLED.key, "false")) {
withNativeWriteCheck(checkNative = false) {
spark
.table("origin_table")
@@ -658,7 +659,7 @@ class GlutenClickHouseNativeWriteTableSuite
nativeWrite {
format =>
val table_name = table_name_template.format(format)
- writeAndCheckRead(origin_table, table_name, fields.keys.toSeq, isSparkVersionLE("3.3")) {
+ writeAndCheckRead(origin_table, table_name, fields.keys.toSeq) {
fields =>
spark
.table("origin_table")
@@ -672,7 +673,7 @@ class GlutenClickHouseNativeWriteTableSuite
val table_name_vanilla = table_name_vanilla_template.format(format)
spark.sql(s"drop table IF EXISTS $table_name_vanilla")
- withSQLConf(("spark.gluten.sql.native.writer.enabled", "false")) {
+ withSQLConf((GlutenConfig.NATIVE_WRITER_ENABLED.key, "false")) {
withNativeWriteCheck(checkNative = false) {
spark
.table("origin_table")
@@ -762,7 +763,7 @@ class GlutenClickHouseNativeWriteTableSuite
format =>
val table_name = table_name_template.format(format)
spark.sql(s"drop table IF EXISTS $table_name")
- withNativeWriteCheck(checkNative = isSparkVersionLE("3.3")) {
+ withNativeWriteCheck(checkNative = true) {
spark
.range(10000000)
.selectExpr("id", "cast('2020-01-01' as date) as p")
@@ -798,7 +799,7 @@ class GlutenClickHouseNativeWriteTableSuite
format =>
val table_name = table_name_template.format(format)
spark.sql(s"drop table IF EXISTS $table_name")
- withNativeWriteCheck(checkNative = isSparkVersionLE("3.3")) {
+ withNativeWriteCheck(checkNative = true) {
spark
.range(30000)
.selectExpr("id", "cast(null as string) as p")
@@ -903,7 +904,7 @@ class GlutenClickHouseNativeWriteTableSuite
}
test("GLUTEN-2584: fix native write and read mismatch about complex types") {
- def table(format: String): String = s"t_$format"
+ def table(format: String): String = s"t_2584_$format"
def create(format: String, table_name: Option[String] = None): String =
s"""CREATE TABLE ${table_name.getOrElse(table(format))}(
| id INT,
@@ -934,4 +935,63 @@ class GlutenClickHouseNativeWriteTableSuite
}
)
}
+
+ test(
+ "GLUTEN-8021/8022/8032: fix orc read/write mismatch and parquet" +
+ "read exception when written complex column contains null") {
+ def table(format: String): String = s"t_8021_$format"
+ def create(format: String, table_name: Option[String] = None): String =
+ s"""CREATE TABLE ${table_name.getOrElse(table(format))}(
+ |id int,
+ |x int,
+ |y int,
+ |mp map,
+ |arr array,
+ |tup struct,
+ |arr_mp array
-
- org.apache.gluten
- gluten-substrait
- ${project.version}
- compile
-
org.apache.gluten
gluten-arrow
diff --git a/backends-velox/src/main/java/org/apache/gluten/columnarbatch/VeloxColumnarBatches.java b/backends-velox/src/main/java/org/apache/gluten/columnarbatch/VeloxColumnarBatches.java
index e2035455fd74..db2d08e31435 100644
--- a/backends-velox/src/main/java/org/apache/gluten/columnarbatch/VeloxColumnarBatches.java
+++ b/backends-velox/src/main/java/org/apache/gluten/columnarbatch/VeloxColumnarBatches.java
@@ -16,6 +16,7 @@
*/
package org.apache.gluten.columnarbatch;
+import org.apache.gluten.backendsapi.BackendsApiManager;
import org.apache.gluten.runtime.Runtime;
import org.apache.gluten.runtime.Runtimes;
@@ -59,8 +60,10 @@ public static ColumnarBatch toVeloxBatch(ColumnarBatch input) {
return input;
}
Preconditions.checkArgument(!isVeloxBatch(input));
- final Runtime runtime = Runtimes.contextInstance("VeloxColumnarBatches#toVeloxBatch");
- final long handle = ColumnarBatches.getNativeHandle(input);
+ final Runtime runtime =
+ Runtimes.contextInstance(
+ BackendsApiManager.getBackendName(), "VeloxColumnarBatches#toVeloxBatch");
+ final long handle = ColumnarBatches.getNativeHandle(BackendsApiManager.getBackendName(), input);
final long outHandle = VeloxColumnarBatchJniWrapper.create(runtime).from(handle);
final ColumnarBatch output = ColumnarBatches.create(outHandle);
@@ -88,9 +91,13 @@ public static ColumnarBatch toVeloxBatch(ColumnarBatch input) {
* Otherwise {@link UnsupportedOperationException} will be thrown.
*/
public static ColumnarBatch compose(ColumnarBatch... batches) {
- final Runtime runtime = Runtimes.contextInstance("VeloxColumnarBatches#compose");
+ final Runtime runtime =
+ Runtimes.contextInstance(
+ BackendsApiManager.getBackendName(), "VeloxColumnarBatches#compose");
final long[] handles =
- Arrays.stream(batches).mapToLong(ColumnarBatches::getNativeHandle).toArray();
+ Arrays.stream(batches)
+ .mapToLong(b -> ColumnarBatches.getNativeHandle(BackendsApiManager.getBackendName(), b))
+ .toArray();
final long handle = VeloxColumnarBatchJniWrapper.create(runtime).compose(handles);
return ColumnarBatches.create(handle);
}
diff --git a/backends-velox/src/main/java/org/apache/gluten/metrics/IteratorMetricsJniWrapper.java b/backends-velox/src/main/java/org/apache/gluten/metrics/IteratorMetricsJniWrapper.java
index 7250fde0fe40..d03f8816adfc 100644
--- a/backends-velox/src/main/java/org/apache/gluten/metrics/IteratorMetricsJniWrapper.java
+++ b/backends-velox/src/main/java/org/apache/gluten/metrics/IteratorMetricsJniWrapper.java
@@ -16,6 +16,7 @@
*/
package org.apache.gluten.metrics;
+import org.apache.gluten.backendsapi.BackendsApiManager;
import org.apache.gluten.runtime.Runtime;
import org.apache.gluten.runtime.RuntimeAware;
import org.apache.gluten.runtime.Runtimes;
@@ -29,7 +30,8 @@ private IteratorMetricsJniWrapper(Runtime runtime) {
}
public static IteratorMetricsJniWrapper create() {
- final Runtime runtime = Runtimes.contextInstance("IteratorMetrics");
+ final Runtime runtime =
+ Runtimes.contextInstance(BackendsApiManager.getBackendName(), "IteratorMetrics");
return new IteratorMetricsJniWrapper(runtime);
}
diff --git a/backends-velox/src/main/java/org/apache/gluten/utils/GlutenURLDecoder.java b/backends-velox/src/main/java/org/apache/gluten/utils/GlutenURLDecoder.java
index 9228a2f860ae..856ddf159730 100644
--- a/backends-velox/src/main/java/org/apache/gluten/utils/GlutenURLDecoder.java
+++ b/backends-velox/src/main/java/org/apache/gluten/utils/GlutenURLDecoder.java
@@ -31,7 +31,7 @@ public class GlutenURLDecoder {
* Note: The World Wide Web Consortium
* Recommendation states that UTF-8 should be used. Not doing so may introduce
- * incompatibilites.
+ * incompatibilities.
*
* @param s the String
to decode
* @param enc The name of a supported character
diff --git a/backends-velox/src/main/java/org/apache/gluten/utils/VeloxBatchResizer.java b/backends-velox/src/main/java/org/apache/gluten/utils/VeloxBatchResizer.java
index 2a6dcb43a052..73bfec08cf48 100644
--- a/backends-velox/src/main/java/org/apache/gluten/utils/VeloxBatchResizer.java
+++ b/backends-velox/src/main/java/org/apache/gluten/utils/VeloxBatchResizer.java
@@ -16,6 +16,7 @@
*/
package org.apache.gluten.utils;
+import org.apache.gluten.backendsapi.BackendsApiManager;
import org.apache.gluten.runtime.Runtime;
import org.apache.gluten.runtime.Runtimes;
import org.apache.gluten.vectorized.ColumnarBatchInIterator;
@@ -28,10 +29,14 @@
public final class VeloxBatchResizer {
public static ColumnarBatchOutIterator create(
int minOutputBatchSize, int maxOutputBatchSize, Iterator in) {
- final Runtime runtime = Runtimes.contextInstance("VeloxBatchResizer");
+ final Runtime runtime =
+ Runtimes.contextInstance(BackendsApiManager.getBackendName(), "VeloxBatchResizer");
long outHandle =
VeloxBatchResizerJniWrapper.create(runtime)
- .create(minOutputBatchSize, maxOutputBatchSize, new ColumnarBatchInIterator(in));
+ .create(
+ minOutputBatchSize,
+ maxOutputBatchSize,
+ new ColumnarBatchInIterator(BackendsApiManager.getBackendName(), in));
return new ColumnarBatchOutIterator(runtime, outHandle);
}
}
diff --git a/backends-velox/src/main/java/org/apache/gluten/utils/VeloxBloomFilter.java b/backends-velox/src/main/java/org/apache/gluten/utils/VeloxBloomFilter.java
index 10179d63edb3..cdf2b195cd0c 100644
--- a/backends-velox/src/main/java/org/apache/gluten/utils/VeloxBloomFilter.java
+++ b/backends-velox/src/main/java/org/apache/gluten/utils/VeloxBloomFilter.java
@@ -16,6 +16,7 @@
*/
package org.apache.gluten.utils;
+import org.apache.gluten.backendsapi.BackendsApiManager;
import org.apache.gluten.runtime.Runtimes;
import org.apache.commons.io.IOUtils;
@@ -30,7 +31,8 @@
public class VeloxBloomFilter extends BloomFilter {
private final VeloxBloomFilterJniWrapper jni =
- VeloxBloomFilterJniWrapper.create(Runtimes.contextInstance("VeloxBloomFilter"));
+ VeloxBloomFilterJniWrapper.create(
+ Runtimes.contextInstance(BackendsApiManager.getBackendName(), "VeloxBloomFilter"));
private final long handle;
private VeloxBloomFilter(byte[] data) {
diff --git a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxBackend.scala b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxBackend.scala
index 251b93cc7cec..6bf4b6a4e256 100644
--- a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxBackend.scala
+++ b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxBackend.scala
@@ -18,7 +18,7 @@ package org.apache.gluten.backendsapi.velox
import org.apache.gluten.GlutenBuildInfo._
import org.apache.gluten.GlutenConfig
-import org.apache.gluten.backend.Backend
+import org.apache.gluten.backend.Component.BuildInfo
import org.apache.gluten.backendsapi._
import org.apache.gluten.columnarbatch.VeloxBatch
import org.apache.gluten.exception.GlutenNotSupportException
@@ -35,12 +35,13 @@ import org.apache.spark.sql.catalyst.catalog.BucketSpec
import org.apache.spark.sql.catalyst.expressions.{Alias, CumeDist, DenseRank, Descending, Expression, Lag, Lead, NamedExpression, NthValue, NTile, PercentRank, RangeFrame, Rank, RowNumber, SortOrder, SpecialFrameBoundary, SpecifiedWindowFrame}
import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, ApproximatePercentile, Percentile}
import org.apache.spark.sql.catalyst.plans.{JoinType, LeftOuter, RightOuter}
-import org.apache.spark.sql.catalyst.util.CharVarcharUtils
+import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, CharVarcharUtils}
import org.apache.spark.sql.execution.{ColumnarCachedBatchSerializer, SparkPlan}
+import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec
import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec
import org.apache.spark.sql.execution.command.CreateDataSourceTableAsSelectCommand
import org.apache.spark.sql.execution.datasources.{FileFormat, InsertIntoHadoopFsRelationCommand}
-import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat
+import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, ParquetOptions}
import org.apache.spark.sql.hive.execution.HiveFileFormat
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types._
@@ -51,11 +52,11 @@ import scala.util.control.Breaks.breakable
class VeloxBackend extends SubstraitBackend {
import VeloxBackend._
+
override def name(): String = VeloxBackend.BACKEND_NAME
- override def defaultBatchType: Convention.BatchType = VeloxBatch
+ override def buildInfo(): BuildInfo =
+ BuildInfo("Velox", VELOX_BRANCH, VELOX_REVISION, VELOX_REVISION_TIME)
override def convFuncOverride(): ConventionFunc.Override = new ConvFunc()
- override def buildInfo(): Backend.BuildInfo =
- Backend.BuildInfo("Velox", VELOX_BRANCH, VELOX_REVISION, VELOX_REVISION_TIME)
override def iteratorApi(): IteratorApi = new VeloxIteratorApi
override def sparkPlanExecApi(): SparkPlanExecApi = new VeloxSparkPlanExecApi
override def transformerApi(): TransformerApi = new VeloxTransformerApi
@@ -72,6 +73,8 @@ object VeloxBackend {
private class ConvFunc() extends ConventionFunc.Override {
override def batchTypeOf: PartialFunction[SparkPlan, Convention.BatchType] = {
+ case a: AdaptiveSparkPlanExec if a.supportsColumnar =>
+ VeloxBatch
case i: InMemoryTableScanExec
if i.supportsColumnar && i.relation.cacheBuilder.serializer
.isInstanceOf[ColumnarCachedBatchSerializer] =>
@@ -81,85 +84,103 @@ object VeloxBackend {
}
object VeloxBackendSettings extends BackendSettingsApi {
-
val SHUFFLE_SUPPORTED_CODEC = Set("lz4", "zstd")
val GLUTEN_VELOX_UDF_LIB_PATHS = VeloxBackend.CONF_PREFIX + ".udfLibraryPaths"
val GLUTEN_VELOX_DRIVER_UDF_LIB_PATHS = VeloxBackend.CONF_PREFIX + ".driver.udfLibraryPaths"
val GLUTEN_VELOX_INTERNAL_UDF_LIB_PATHS = VeloxBackend.CONF_PREFIX + ".internal.udfLibraryPaths"
val GLUTEN_VELOX_UDF_ALLOW_TYPE_CONVERSION = VeloxBackend.CONF_PREFIX + ".udfAllowTypeConversion"
- val MAXIMUM_BATCH_SIZE: Int = 32768
+ /** The columnar-batch type this backend is by default using. */
+ override def primaryBatchType: Convention.BatchType = VeloxBatch
- override def validateScan(
+ override def validateScanExec(
format: ReadFileFormat,
fields: Array[StructField],
- rootPaths: Seq[String]): ValidationResult = {
- val filteredRootPaths = distinctRootPaths(rootPaths)
- if (
- filteredRootPaths.nonEmpty && !VeloxFileSystemValidationJniWrapper
- .allSupportedByRegisteredFileSystems(filteredRootPaths.toArray)
- ) {
- return ValidationResult.failed(
- s"Scheme of [$filteredRootPaths] is not supported by registered file systems.")
- }
- // Validate if all types are supported.
- def validateTypes(validatorFunc: PartialFunction[StructField, String]): ValidationResult = {
- // Collect unsupported types.
- val unsupportedDataTypeReason = fields.collect(validatorFunc)
- if (unsupportedDataTypeReason.isEmpty) {
- ValidationResult.succeeded
+ rootPaths: Seq[String],
+ properties: Map[String, String]): ValidationResult = {
+
+ def validateScheme(): Option[String] = {
+ val filteredRootPaths = distinctRootPaths(rootPaths)
+ if (
+ filteredRootPaths.nonEmpty && !VeloxFileSystemValidationJniWrapper
+ .allSupportedByRegisteredFileSystems(filteredRootPaths.toArray)
+ ) {
+ Some(s"Scheme of [$filteredRootPaths] is not supported by registered file systems.")
} else {
- ValidationResult.failed(
- s"Found unsupported data type in $format: ${unsupportedDataTypeReason.mkString(", ")}.")
+ None
}
}
- format match {
- case ParquetReadFormat =>
- val typeValidator: PartialFunction[StructField, String] = {
- // Parquet timestamp is not fully supported yet
- case StructField(_, TimestampType, _, _)
- if GlutenConfig.getConf.forceParquetTimestampTypeScanFallbackEnabled =>
- "TimestampType"
- }
- validateTypes(typeValidator)
- case DwrfReadFormat => ValidationResult.succeeded
- case OrcReadFormat =>
- if (!GlutenConfig.getConf.veloxOrcScanEnabled) {
- ValidationResult.failed(s"Velox ORC scan is turned off.")
+ def validateFormat(): Option[String] = {
+ def validateTypes(validatorFunc: PartialFunction[StructField, String]): Option[String] = {
+ // Collect unsupported types.
+ val unsupportedDataTypeReason = fields.collect(validatorFunc)
+ if (unsupportedDataTypeReason.nonEmpty) {
+ Some(
+ s"Found unsupported data type in $format: ${unsupportedDataTypeReason.mkString(", ")}.")
} else {
+ None
+ }
+ }
+
+ def isCharType(stringType: StringType, metadata: Metadata): Boolean = {
+ val charTypePattern = "char\\((\\d+)\\)".r
+ GlutenConfig.getConf.forceOrcCharTypeScanFallbackEnabled && charTypePattern
+ .findFirstIn(
+ CharVarcharUtils
+ .getRawTypeString(metadata)
+ .getOrElse(stringType.catalogString))
+ .isDefined
+ }
+
+ format match {
+ case ParquetReadFormat =>
val typeValidator: PartialFunction[StructField, String] = {
- case StructField(_, arrayType: ArrayType, _, _)
- if arrayType.elementType.isInstanceOf[StructType] =>
- "StructType as element in ArrayType"
- case StructField(_, arrayType: ArrayType, _, _)
- if arrayType.elementType.isInstanceOf[ArrayType] =>
- "ArrayType as element in ArrayType"
- case StructField(_, mapType: MapType, _, _)
- if mapType.keyType.isInstanceOf[StructType] =>
- "StructType as Key in MapType"
- case StructField(_, mapType: MapType, _, _)
- if mapType.valueType.isInstanceOf[ArrayType] =>
- "ArrayType as Value in MapType"
- case StructField(_, stringType: StringType, _, metadata)
- if isCharType(stringType, metadata) =>
- CharVarcharUtils.getRawTypeString(metadata) + " not support"
- case StructField(_, TimestampType, _, _) => "TimestampType not support"
+ // Parquet timestamp is not fully supported yet
+ case StructField(_, TimestampType, _, _)
+ if GlutenConfig.getConf.forceParquetTimestampTypeScanFallbackEnabled =>
+ "TimestampType(force fallback)"
}
- validateTypes(typeValidator)
- }
- case _ => ValidationResult.failed(s"Unsupported file format for $format.")
+ val parquetOptions = new ParquetOptions(CaseInsensitiveMap(properties), SQLConf.get)
+ if (parquetOptions.mergeSchema) {
+ // https://github.com/apache/incubator-gluten/issues/7174
+ Some(s"not support when merge schema is true")
+ } else {
+ validateTypes(typeValidator)
+ }
+ case DwrfReadFormat => None
+ case OrcReadFormat =>
+ if (!GlutenConfig.getConf.veloxOrcScanEnabled) {
+ Some(s"Velox ORC scan is turned off, ${GlutenConfig.VELOX_ORC_SCAN_ENABLED.key}")
+ } else {
+ val typeValidator: PartialFunction[StructField, String] = {
+ case StructField(_, arrayType: ArrayType, _, _)
+ if arrayType.elementType.isInstanceOf[StructType] =>
+ "StructType as element in ArrayType"
+ case StructField(_, arrayType: ArrayType, _, _)
+ if arrayType.elementType.isInstanceOf[ArrayType] =>
+ "ArrayType as element in ArrayType"
+ case StructField(_, mapType: MapType, _, _)
+ if mapType.keyType.isInstanceOf[StructType] =>
+ "StructType as Key in MapType"
+ case StructField(_, mapType: MapType, _, _)
+ if mapType.valueType.isInstanceOf[ArrayType] =>
+ "ArrayType as Value in MapType"
+ case StructField(_, stringType: StringType, _, metadata)
+ if isCharType(stringType, metadata) =>
+ CharVarcharUtils.getRawTypeString(metadata) + "(force fallback)"
+ case StructField(_, TimestampType, _, _) => "TimestampType"
+ }
+ validateTypes(typeValidator)
+ }
+ case _ => Some(s"Unsupported file format for $format.")
+ }
}
- }
- def isCharType(stringType: StringType, metadata: Metadata): Boolean = {
- val charTypePattern = "char\\((\\d+)\\)".r
- GlutenConfig.getConf.forceOrcCharTypeScanFallbackEnabled && charTypePattern
- .findFirstIn(
- CharVarcharUtils
- .getRawTypeString(metadata)
- .getOrElse(stringType.catalogString))
- .isDefined
+ validateScheme().orElse(validateFormat()) match {
+ case Some(reason) => ValidationResult.failed(reason)
+ case _ => ValidationResult.succeeded
+ }
}
def distinctRootPaths(paths: Seq[String]): Seq[String] = {
@@ -216,15 +237,26 @@ object VeloxBackendSettings extends BackendSettingsApi {
// Validate if all types are supported.
def validateDataTypes(): Option[String] = {
- val unsupportedTypes = fields.flatMap {
- field =>
- field.dataType match {
- case _: StructType => Some("StructType")
- case _: ArrayType => Some("ArrayType")
- case _: MapType => Some("MapType")
- case _: YearMonthIntervalType => Some("YearMonthIntervalType")
+ val unsupportedTypes = format match {
+ case _: ParquetFileFormat =>
+ fields.flatMap {
+ case StructField(_, _: YearMonthIntervalType, _, _) =>
+ Some("YearMonthIntervalType")
+ case StructField(_, _: StructType, _, _) =>
+ Some("StructType")
case _ => None
}
+ case _ =>
+ fields.flatMap {
+ field =>
+ field.dataType match {
+ case _: StructType => Some("StructType")
+ case _: ArrayType => Some("ArrayType")
+ case _: MapType => Some("MapType")
+ case _: YearMonthIntervalType => Some("YearMonthIntervalType")
+ case _ => None
+ }
+ }
}
if (unsupportedTypes.nonEmpty) {
Some(unsupportedTypes.mkString("Found unsupported type:", ",", ""))
@@ -368,8 +400,10 @@ object VeloxBackendSettings extends BackendSettingsApi {
case _ =>
}
windowExpression.windowFunction match {
- case _: RowNumber | _: Rank | _: CumeDist | _: DenseRank | _: PercentRank |
- _: NthValue | _: NTile | _: Lag | _: Lead =>
+ case _: RowNumber | _: Rank | _: CumeDist | _: DenseRank | _: PercentRank | _: NTile =>
+ case nv: NthValue if !nv.input.foldable =>
+ case l: Lag if !l.input.foldable =>
+ case l: Lead if !l.input.foldable =>
case aggrExpr: AggregateExpression
if !aggrExpr.aggregateFunction.isInstanceOf[ApproximatePercentile]
&& !aggrExpr.aggregateFunction.isInstanceOf[Percentile] =>
diff --git a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxIteratorApi.scala b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxIteratorApi.scala
index 320d1f366c23..26bc108c15d9 100644
--- a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxIteratorApi.scala
+++ b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxIteratorApi.scala
@@ -39,9 +39,7 @@ import org.apache.spark.sql.execution.metric.SQLMetric
import org.apache.spark.sql.types._
import org.apache.spark.sql.utils.SparkInputMetricsUtil.InputMetricsWrapper
import org.apache.spark.sql.vectorized.ColumnarBatch
-import org.apache.spark.util.{ExecutorManager, SerializableConfiguration, SparkDirectoryUtil}
-
-import org.apache.hadoop.fs.{FileSystem, Path}
+import org.apache.spark.util.{ExecutorManager, SparkDirectoryUtil}
import java.lang.{Long => JLong}
import java.nio.charset.StandardCharsets
@@ -57,8 +55,7 @@ class VeloxIteratorApi extends IteratorApi with Logging {
partitionSchema: StructType,
fileFormat: ReadFileFormat,
metadataColumnNames: Seq[String],
- properties: Map[String, String],
- serializableHadoopConf: SerializableConfiguration): SplitInfo = {
+ properties: Map[String, String]): SplitInfo = {
partition match {
case f: FilePartition =>
val (
@@ -69,7 +66,7 @@ class VeloxIteratorApi extends IteratorApi with Logging {
modificationTimes,
partitionColumns,
metadataColumns) =
- constructSplitInfo(partitionSchema, f.files, metadataColumnNames, serializableHadoopConf)
+ constructSplitInfo(partitionSchema, f.files, metadataColumnNames)
val preferredLocations =
SoftAffinity.getFilePartitionLocations(f)
LocalFilesBuilder.makeLocalFiles(
@@ -112,8 +109,7 @@ class VeloxIteratorApi extends IteratorApi with Logging {
private def constructSplitInfo(
schema: StructType,
files: Array[PartitionedFile],
- metadataColumnNames: Seq[String],
- serializableHadoopConf: SerializableConfiguration) = {
+ metadataColumnNames: Seq[String]) = {
val paths = new JArrayList[String]()
val starts = new JArrayList[JLong]
val lengths = new JArrayList[JLong]()
@@ -125,15 +121,9 @@ class VeloxIteratorApi extends IteratorApi with Logging {
file =>
// The "file.filePath" in PartitionedFile is not the original encoded path, so the decoded
// path is incorrect in some cases and here fix the case of ' ' by using GlutenURLDecoder
- var filePath = file.filePath.toString
- if (filePath.startsWith("viewfs")) {
- val viewPath = new Path(filePath)
- val viewFileSystem = FileSystem.get(viewPath.toUri, serializableHadoopConf.value)
- filePath = viewFileSystem.resolvePath(viewPath).toString
- }
paths.add(
GlutenURLDecoder
- .decode(filePath, StandardCharsets.UTF_8.name()))
+ .decode(file.filePath.toString, StandardCharsets.UTF_8.name()))
starts.add(JLong.valueOf(file.start))
lengths.add(JLong.valueOf(file.length))
val (fileSize, modificationTime) =
@@ -193,9 +183,9 @@ class VeloxIteratorApi extends IteratorApi with Logging {
val columnarNativeIterators =
new JArrayList[ColumnarBatchInIterator](inputIterators.map {
- iter => new ColumnarBatchInIterator(iter.asJava)
+ iter => new ColumnarBatchInIterator(BackendsApiManager.getBackendName, iter.asJava)
}.asJava)
- val transKernel = NativePlanEvaluator.create()
+ val transKernel = NativePlanEvaluator.create(BackendsApiManager.getBackendName)
val splitInfoByteArray = inputPartition
.asInstanceOf[GlutenPartition]
@@ -245,10 +235,10 @@ class VeloxIteratorApi extends IteratorApi with Logging {
ExecutorManager.tryTaskSet(numaBindingInfo)
- val transKernel = NativePlanEvaluator.create()
+ val transKernel = NativePlanEvaluator.create(BackendsApiManager.getBackendName)
val columnarNativeIterator =
new JArrayList[ColumnarBatchInIterator](inputIterators.map {
- iter => new ColumnarBatchInIterator(iter.asJava)
+ iter => new ColumnarBatchInIterator(BackendsApiManager.getBackendName, iter.asJava)
}.asJava)
val spillDirPath = SparkDirectoryUtil
.get()
diff --git a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxListenerApi.scala b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxListenerApi.scala
index 850509db3e91..175e34177a5d 100644
--- a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxListenerApi.scala
+++ b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxListenerApi.scala
@@ -31,8 +31,9 @@ import org.apache.spark.{HdfsConfGenerator, SparkConf, SparkContext}
import org.apache.spark.api.plugin.PluginContext
import org.apache.spark.internal.Logging
import org.apache.spark.network.util.ByteUnit
+import org.apache.spark.sql.execution.ColumnarCachedBatchSerializer
import org.apache.spark.sql.execution.datasources.GlutenWriterColumnarRules
-import org.apache.spark.sql.execution.datasources.velox.{VeloxOrcWriterInjects, VeloxParquetWriterInjects, VeloxRowSplitter}
+import org.apache.spark.sql.execution.datasources.velox.{VeloxParquetWriterInjects, VeloxRowSplitter}
import org.apache.spark.sql.expression.UDFResolver
import org.apache.spark.sql.internal.{GlutenConfigUtil, StaticSQLConf}
import org.apache.spark.util.{SparkDirectoryUtil, SparkResourceUtil}
@@ -75,7 +76,7 @@ class VeloxListenerApi extends ListenerApi with Logging {
if (conf.getBoolean(GlutenConfig.COLUMNAR_TABLE_CACHE_ENABLED.key, defaultValue = false)) {
conf.set(
StaticSQLConf.SPARK_CACHE_SERIALIZER.key,
- "org.apache.spark.sql.execution.ColumnarCachedBatchSerializer")
+ classOf[ColumnarCachedBatchSerializer].getName)
}
// Static initializers for driver.
@@ -89,7 +90,7 @@ class VeloxListenerApi extends ListenerApi with Logging {
SparkDirectoryUtil.init(conf)
UDFResolver.resolveUdfConf(conf, isDriver = true)
- initialize(conf)
+ initialize(conf, isDriver = true)
UdfJniWrapper.registerFunctionSignatures()
}
@@ -116,16 +117,16 @@ class VeloxListenerApi extends ListenerApi with Logging {
SparkDirectoryUtil.init(conf)
UDFResolver.resolveUdfConf(conf, isDriver = false)
- initialize(conf)
+ initialize(conf, isDriver = false)
}
override def onExecutorShutdown(): Unit = shutdown()
- private def initialize(conf: SparkConf): Unit = {
+ private def initialize(conf: SparkConf, isDriver: Boolean): Unit = {
// Force batch type initializations.
- VeloxBatch.getClass
- ArrowJavaBatch.getClass
- ArrowNativeBatch.getClass
+ VeloxBatch.ensureRegistered()
+ ArrowJavaBatch.ensureRegistered()
+ ArrowNativeBatch.ensureRegistered()
// Sets this configuration only once, since not undoable.
if (conf.getBoolean(GlutenConfig.GLUTEN_DEBUG_KEEP_JNI_WORKSPACE, defaultValue = false)) {
@@ -157,11 +158,16 @@ class VeloxListenerApi extends ListenerApi with Logging {
}
// Initial native backend with configurations.
- val parsed = GlutenConfigUtil.parseConfig(conf.getAll.toMap)
- NativeBackendInitializer.initializeBackend(parsed)
+ var parsed = GlutenConfigUtil.parseConfig(conf.getAll.toMap)
+
+ // Workaround for https://github.com/apache/incubator-gluten/issues/7837
+ if (isDriver && !inLocalMode(conf)) {
+ parsed += (GlutenConfig.COLUMNAR_VELOX_CACHE_ENABLED.key -> "false")
+ }
+ NativeBackendInitializer.forBackend(VeloxBackend.BACKEND_NAME).initialize(parsed)
// Inject backend-specific implementations to override spark classes.
- GlutenFormatFactory.register(new VeloxParquetWriterInjects, new VeloxOrcWriterInjects)
+ GlutenFormatFactory.register(new VeloxParquetWriterInjects)
GlutenFormatFactory.injectPostRuleFactory(
session => GlutenWriterColumnarRules.NativeWritePostRule(session))
GlutenFormatFactory.register(new VeloxRowSplitter())
diff --git a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxMetricsApi.scala b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxMetricsApi.scala
index 10b0c493c10a..934b680382ea 100644
--- a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxMetricsApi.scala
+++ b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxMetricsApi.scala
@@ -22,7 +22,8 @@ import org.apache.gluten.substrait.{AggregationParams, JoinParams}
import org.apache.spark.SparkContext
import org.apache.spark.internal.Logging
-import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.{ColumnarInputAdapter, SparkPlan}
+import org.apache.spark.sql.execution.adaptive.QueryStageExec
import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics}
import java.lang.{Long => JLong}
@@ -38,18 +39,37 @@ class VeloxMetricsApi extends MetricsApi with Logging {
}
override def genInputIteratorTransformerMetrics(
- sparkContext: SparkContext): Map[String, SQLMetric] = {
+ child: SparkPlan,
+ sparkContext: SparkContext,
+ forBroadcast: Boolean): Map[String, SQLMetric] = {
+ def metricsPlan(plan: SparkPlan): SparkPlan = {
+ plan match {
+ case ColumnarInputAdapter(child) => metricsPlan(child)
+ case q: QueryStageExec => metricsPlan(q.plan)
+ case _ => plan
+ }
+ }
+
+ val outputMetrics = if (forBroadcast) {
+ metricsPlan(child).metrics
+ .filterKeys(key => key.equals("numOutputRows") || key.equals("outputVectors"))
+ } else {
+ Map(
+ "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"),
+ "outputVectors" -> SQLMetrics.createMetric(sparkContext, "number of output vectors")
+ )
+ }
+
Map(
"cpuCount" -> SQLMetrics.createMetric(sparkContext, "cpu wall time count"),
- "wallNanos" -> SQLMetrics.createNanoTimingMetric(sparkContext, "time of input iterator"),
- "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"),
- "outputVectors" -> SQLMetrics.createMetric(sparkContext, "number of output vectors")
- )
+ "wallNanos" -> SQLMetrics.createNanoTimingMetric(sparkContext, "time of input iterator")
+ ) ++ outputMetrics
}
override def genInputIteratorTransformerMetricsUpdater(
- metrics: Map[String, SQLMetric]): MetricsUpdater = {
- InputIteratorMetricsUpdater(metrics)
+ metrics: Map[String, SQLMetric],
+ forBroadcast: Boolean): MetricsUpdater = {
+ InputIteratorMetricsUpdater(metrics, forBroadcast)
}
override def genBatchScanTransformerMetrics(sparkContext: SparkContext): Map[String, SQLMetric] =
@@ -562,4 +582,15 @@ class VeloxMetricsApi extends MetricsApi with Logging {
override def genSampleTransformerMetricsUpdater(metrics: Map[String, SQLMetric]): MetricsUpdater =
new SampleMetricsUpdater(metrics)
+
+ override def genUnionTransformerMetrics(sparkContext: SparkContext): Map[String, SQLMetric] = Map(
+ "numInputRows" -> SQLMetrics.createMetric(sparkContext, "number of input rows"),
+ "inputVectors" -> SQLMetrics.createMetric(sparkContext, "number of input vectors"),
+ "inputBytes" -> SQLMetrics.createSizeMetric(sparkContext, "number of input bytes"),
+ "wallNanos" -> SQLMetrics.createNanoTimingMetric(sparkContext, "time of union"),
+ "cpuCount" -> SQLMetrics.createMetric(sparkContext, "cpu wall time count")
+ )
+
+ override def genUnionTransformerMetricsUpdater(metrics: Map[String, SQLMetric]): MetricsUpdater =
+ new UnionMetricsUpdater(metrics)
}
diff --git a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxRuleApi.scala b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxRuleApi.scala
index a838c463c390..7337be573710 100644
--- a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxRuleApi.scala
+++ b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxRuleApi.scala
@@ -16,60 +16,93 @@
*/
package org.apache.gluten.backendsapi.velox
+import org.apache.gluten.GlutenConfig
import org.apache.gluten.backendsapi.RuleApi
-import org.apache.gluten.datasource.ArrowConvertorRule
+import org.apache.gluten.columnarbatch.VeloxBatch
import org.apache.gluten.extension._
import org.apache.gluten.extension.columnar._
-import org.apache.gluten.extension.columnar.MiscColumnarRules.{RemoveGlutenTableCacheColumnarToRow, RemoveTopmostColumnarToRow, RewriteSubqueryBroadcast, TransformPreOverrides}
-import org.apache.gluten.extension.columnar.enumerated.EnumeratedTransform
-import org.apache.gluten.extension.columnar.rewrite.RewriteSparkPlanRulesManager
+import org.apache.gluten.extension.columnar.MiscColumnarRules.{RemoveGlutenTableCacheColumnarToRow, RemoveTopmostColumnarToRow, RewriteSubqueryBroadcast}
+import org.apache.gluten.extension.columnar.enumerated.{RasOffload, RemoveSort}
+import org.apache.gluten.extension.columnar.enumerated.planner.cost.{LegacyCoster, RoughCoster, RoughCoster2}
+import org.apache.gluten.extension.columnar.heuristic.{ExpandFallbackPolicy, HeuristicTransform}
+import org.apache.gluten.extension.columnar.offload.{OffloadExchange, OffloadJoin, OffloadOthers}
+import org.apache.gluten.extension.columnar.rewrite._
import org.apache.gluten.extension.columnar.transition.{InsertTransitions, RemoveTransitions}
-import org.apache.gluten.extension.injector.{RuleInjector, SparkInjector}
+import org.apache.gluten.extension.columnar.validator.Validator
+import org.apache.gluten.extension.columnar.validator.Validators.ValidatorBuilderImplicits
+import org.apache.gluten.extension.injector.{Injector, SparkInjector}
import org.apache.gluten.extension.injector.GlutenInjector.{LegacyInjector, RasInjector}
import org.apache.gluten.sql.shims.SparkShimLoader
-import org.apache.spark.sql.execution.{ColumnarCollapseTransformStages, GlutenFallbackReporter}
+import org.apache.spark.sql.execution._
+import org.apache.spark.sql.execution.aggregate.{HashAggregateExec, ObjectHashAggregateExec, SortAggregateExec}
+import org.apache.spark.sql.execution.datasources.WriteFilesExec
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanExecBase
+import org.apache.spark.sql.execution.exchange.Exchange
+import org.apache.spark.sql.execution.joins.BaseJoinExec
+import org.apache.spark.sql.execution.python.EvalPythonExec
+import org.apache.spark.sql.execution.window.WindowExec
+import org.apache.spark.sql.hive.HiveTableScanExecTransformer
class VeloxRuleApi extends RuleApi {
import VeloxRuleApi._
- override def injectRules(injector: RuleInjector): Unit = {
+ override def injectRules(injector: Injector): Unit = {
injectSpark(injector.spark)
injectLegacy(injector.gluten.legacy)
injectRas(injector.gluten.ras)
}
}
-private object VeloxRuleApi {
- def injectSpark(injector: SparkInjector): Unit = {
+object VeloxRuleApi {
+ private def injectSpark(injector: SparkInjector): Unit = {
// Inject the regular Spark rules directly.
injector.injectOptimizerRule(CollectRewriteRule.apply)
injector.injectOptimizerRule(HLLRewriteRule.apply)
injector.injectPostHocResolutionRule(ArrowConvertorRule.apply)
}
- def injectLegacy(injector: LegacyInjector): Unit = {
- // Gluten columnar: Transform rules.
- injector.injectTransform(_ => RemoveTransitions)
- injector.injectTransform(_ => PushDownInputFileExpression.PreOffload)
- injector.injectTransform(c => FallbackOnANSIMode.apply(c.session))
- injector.injectTransform(c => FallbackMultiCodegens.apply(c.session))
- injector.injectTransform(_ => RewriteSubqueryBroadcast())
- injector.injectTransform(c => BloomFilterMightContainJointRewriteRule.apply(c.session))
- injector.injectTransform(c => ArrowScanReplaceRule.apply(c.session))
- injector.injectTransform(_ => RewriteSparkPlanRulesManager())
- injector.injectTransform(_ => AddFallbackTagRule())
- injector.injectTransform(_ => TransformPreOverrides())
- injector.injectTransform(c => PartialProjectRule.apply(c.session))
- injector.injectTransform(_ => RemoveNativeWriteFilesSortAndProject())
- injector.injectTransform(c => RewriteTransformer.apply(c.session))
- injector.injectTransform(_ => PushDownFilterToScan)
- injector.injectTransform(_ => PushDownInputFileExpression.PostOffload)
- injector.injectTransform(_ => EnsureLocalSortRequirements)
- injector.injectTransform(_ => EliminateLocalSort)
- injector.injectTransform(_ => CollapseProjectExecTransformer)
- injector.injectTransform(c => FlushableHashAggregateRule.apply(c.session))
- injector.injectTransform(c => InsertTransitions(c.outputsColumnar))
+ private def injectLegacy(injector: LegacyInjector): Unit = {
+ // Legacy: Pre-transform rules.
+ injector.injectPreTransform(_ => RemoveTransitions)
+ injector.injectPreTransform(_ => PushDownInputFileExpression.PreOffload)
+ injector.injectPreTransform(c => FallbackOnANSIMode.apply(c.session))
+ injector.injectPreTransform(c => FallbackMultiCodegens.apply(c.session))
+ injector.injectPreTransform(c => MergeTwoPhasesHashBaseAggregate(c.session))
+ injector.injectPreTransform(_ => RewriteSubqueryBroadcast())
+ injector.injectPreTransform(c => BloomFilterMightContainJointRewriteRule.apply(c.session))
+ injector.injectPreTransform(c => ArrowScanReplaceRule.apply(c.session))
+
+ // Legacy: The legacy transform rule.
+ val validatorBuilder: GlutenConfig => Validator = conf =>
+ Validator
+ .builder()
+ .fallbackByHint()
+ .fallbackIfScanOnlyWithFilterPushed(conf.enableScanOnly)
+ .fallbackComplexExpressions()
+ .fallbackByBackendSettings()
+ .fallbackByUserOptions()
+ .fallbackByTestInjects()
+ .fallbackByNativeValidation()
+ .build()
+ val rewrites =
+ Seq(RewriteIn, RewriteMultiChildrenCount, RewriteJoin, PullOutPreProject, PullOutPostProject)
+ val offloads = Seq(OffloadOthers(), OffloadExchange(), OffloadJoin())
+ injector.injectTransform(
+ c => HeuristicTransform.Single(validatorBuilder(c.glutenConf), rewrites, offloads))
+
+ // Legacy: Post-transform rules.
+ injector.injectPostTransform(_ => UnionTransformerRule())
+ injector.injectPostTransform(c => PartialProjectRule.apply(c.session))
+ injector.injectPostTransform(_ => RemoveNativeWriteFilesSortAndProject())
+ injector.injectPostTransform(c => RewriteTransformer.apply(c.session))
+ injector.injectPostTransform(_ => PushDownFilterToScan)
+ injector.injectPostTransform(_ => PushDownInputFileExpression.PostOffload)
+ injector.injectPostTransform(_ => EnsureLocalSortRequirements)
+ injector.injectPostTransform(_ => EliminateLocalSort)
+ injector.injectPostTransform(_ => CollapseProjectExecTransformer)
+ injector.injectPostTransform(c => FlushableHashAggregateRule.apply(c.session))
+ injector.injectPostTransform(c => InsertTransitions.create(c.outputsColumnar, VeloxBatch))
// Gluten columnar: Fallback policies.
injector.injectFallbackPolicy(
@@ -80,45 +113,91 @@ private object VeloxRuleApi {
SparkShimLoader.getSparkShims
.getExtendedColumnarPostRules()
.foreach(each => injector.injectPost(c => each(c.session)))
- injector.injectPost(c => ColumnarCollapseTransformStages(c.conf))
+ injector.injectPost(c => ColumnarCollapseTransformStages(c.glutenConf))
// Gluten columnar: Final rules.
injector.injectFinal(c => RemoveGlutenTableCacheColumnarToRow(c.session))
- injector.injectFinal(c => GlutenFallbackReporter(c.conf, c.session))
+ injector.injectFinal(c => GlutenFallbackReporter(c.glutenConf, c.session))
injector.injectFinal(_ => RemoveFallbackTagRule())
}
- def injectRas(injector: RasInjector): Unit = {
+ private def injectRas(injector: RasInjector): Unit = {
// Gluten RAS: Pre rules.
- injector.inject(_ => RemoveTransitions)
- injector.inject(_ => PushDownInputFileExpression.PreOffload)
- injector.inject(c => FallbackOnANSIMode.apply(c.session))
- injector.inject(_ => RewriteSubqueryBroadcast())
- injector.inject(c => BloomFilterMightContainJointRewriteRule.apply(c.session))
- injector.inject(c => ArrowScanReplaceRule.apply(c.session))
+ injector.injectPreTransform(_ => RemoveTransitions)
+ injector.injectPreTransform(_ => PushDownInputFileExpression.PreOffload)
+ injector.injectPreTransform(c => FallbackOnANSIMode.apply(c.session))
+ injector.injectPreTransform(c => MergeTwoPhasesHashBaseAggregate(c.session))
+ injector.injectPreTransform(_ => RewriteSubqueryBroadcast())
+ injector.injectPreTransform(c => BloomFilterMightContainJointRewriteRule.apply(c.session))
+ injector.injectPreTransform(c => ArrowScanReplaceRule.apply(c.session))
// Gluten RAS: The RAS rule.
- injector.inject(c => EnumeratedTransform(c.session, c.outputsColumnar))
+ val validatorBuilder: GlutenConfig => Validator = conf =>
+ Validator
+ .builder()
+ .fallbackByHint()
+ .fallbackIfScanOnlyWithFilterPushed(conf.enableScanOnly)
+ .fallbackComplexExpressions()
+ .fallbackByBackendSettings()
+ .fallbackByUserOptions()
+ .fallbackByTestInjects()
+ .build()
+ val rewrites =
+ Seq(RewriteIn, RewriteMultiChildrenCount, RewriteJoin, PullOutPreProject, PullOutPostProject)
+ injector.injectCoster(_ => LegacyCoster)
+ injector.injectCoster(_ => RoughCoster)
+ injector.injectCoster(_ => RoughCoster2)
+ injector.injectRasRule(_ => RemoveSort)
+ val offloads: Seq[RasOffload] = Seq(
+ RasOffload.from[Exchange](OffloadExchange()),
+ RasOffload.from[BaseJoinExec](OffloadJoin()),
+ RasOffload.from[FilterExec](OffloadOthers()),
+ RasOffload.from[ProjectExec](OffloadOthers()),
+ RasOffload.from[DataSourceV2ScanExecBase](OffloadOthers()),
+ RasOffload.from[DataSourceScanExec](OffloadOthers()),
+ RasOffload.from(HiveTableScanExecTransformer.isHiveTableScan(_))(OffloadOthers()),
+ RasOffload.from[CoalesceExec](OffloadOthers()),
+ RasOffload.from[HashAggregateExec](OffloadOthers()),
+ RasOffload.from[SortAggregateExec](OffloadOthers()),
+ RasOffload.from[ObjectHashAggregateExec](OffloadOthers()),
+ RasOffload.from[UnionExec](OffloadOthers()),
+ RasOffload.from[ExpandExec](OffloadOthers()),
+ RasOffload.from[WriteFilesExec](OffloadOthers()),
+ RasOffload.from[SortExec](OffloadOthers()),
+ RasOffload.from[TakeOrderedAndProjectExec](OffloadOthers()),
+ RasOffload.from[WindowExec](OffloadOthers()),
+ RasOffload.from(SparkShimLoader.getSparkShims.isWindowGroupLimitExec(_))(OffloadOthers()),
+ RasOffload.from[LimitExec](OffloadOthers()),
+ RasOffload.from[GenerateExec](OffloadOthers()),
+ RasOffload.from[EvalPythonExec](OffloadOthers()),
+ RasOffload.from[SampleExec](OffloadOthers())
+ )
+ offloads.foreach(
+ offload =>
+ injector.injectRasRule(
+ c => RasOffload.Rule(offload, validatorBuilder(c.glutenConf), rewrites)))
// Gluten RAS: Post rules.
- injector.inject(_ => RemoveTransitions)
- injector.inject(c => PartialProjectRule.apply(c.session))
- injector.inject(_ => RemoveNativeWriteFilesSortAndProject())
- injector.inject(c => RewriteTransformer.apply(c.session))
- injector.inject(_ => PushDownFilterToScan)
- injector.inject(_ => PushDownInputFileExpression.PostOffload)
- injector.inject(_ => EnsureLocalSortRequirements)
- injector.inject(_ => EliminateLocalSort)
- injector.inject(_ => CollapseProjectExecTransformer)
- injector.inject(c => FlushableHashAggregateRule.apply(c.session))
- injector.inject(c => InsertTransitions(c.outputsColumnar))
- injector.inject(c => RemoveTopmostColumnarToRow(c.session, c.ac.isAdaptiveContext()))
+ injector.injectPostTransform(_ => RemoveTransitions)
+ injector.injectPostTransform(_ => UnionTransformerRule())
+ injector.injectPostTransform(c => PartialProjectRule.apply(c.session))
+ injector.injectPostTransform(_ => RemoveNativeWriteFilesSortAndProject())
+ injector.injectPostTransform(c => RewriteTransformer.apply(c.session))
+ injector.injectPostTransform(_ => PushDownFilterToScan)
+ injector.injectPostTransform(_ => PushDownInputFileExpression.PostOffload)
+ injector.injectPostTransform(_ => EnsureLocalSortRequirements)
+ injector.injectPostTransform(_ => EliminateLocalSort)
+ injector.injectPostTransform(_ => CollapseProjectExecTransformer)
+ injector.injectPostTransform(c => FlushableHashAggregateRule.apply(c.session))
+ injector.injectPostTransform(c => InsertTransitions.create(c.outputsColumnar, VeloxBatch))
+ injector.injectPostTransform(
+ c => RemoveTopmostColumnarToRow(c.session, c.ac.isAdaptiveContext()))
SparkShimLoader.getSparkShims
.getExtendedColumnarPostRules()
- .foreach(each => injector.inject(c => each(c.session)))
- injector.inject(c => ColumnarCollapseTransformStages(c.conf))
- injector.inject(c => RemoveGlutenTableCacheColumnarToRow(c.session))
- injector.inject(c => GlutenFallbackReporter(c.conf, c.session))
- injector.inject(_ => RemoveFallbackTagRule())
+ .foreach(each => injector.injectPostTransform(c => each(c.session)))
+ injector.injectPostTransform(c => ColumnarCollapseTransformStages(c.glutenConf))
+ injector.injectPostTransform(c => RemoveGlutenTableCacheColumnarToRow(c.session))
+ injector.injectPostTransform(c => GlutenFallbackReporter(c.glutenConf, c.session))
+ injector.injectPostTransform(_ => RemoveFallbackTagRule())
}
}
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 81564a440117..d837ac423407 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
@@ -153,7 +153,8 @@ class VeloxSparkPlanExecApi extends SparkPlanExecApi {
left.dataType.isInstanceOf[DecimalType] && right.dataType
.isInstanceOf[DecimalType] && !SQLConf.get.decimalOperationsAllowPrecisionLoss
) {
- val newName = "not_allow_precision_loss_"
+ // https://github.com/facebookincubator/velox/pull/10383
+ val newName = substraitExprName + "_deny_precision_loss"
GenericExpressionTransformer(newName, Seq(left, right), original)
} else {
GenericExpressionTransformer(substraitExprName, Seq(left, right), original)
@@ -631,7 +632,7 @@ class VeloxSparkPlanExecApi extends SparkPlanExecApi {
}
numOutputRows += serialized.map(_.getNumRows).sum
dataSize += rawSize
- ColumnarBuildSideRelation(child.output, serialized.map(_.getSerialized))
+ ColumnarBuildSideRelation(child.output, serialized.map(_.getSerialized), mode)
}
override def doCanonicalizeForBroadcastMode(mode: BroadcastMode): BroadcastMode = {
@@ -723,30 +724,29 @@ class VeloxSparkPlanExecApi extends SparkPlanExecApi {
val trimParaSepStr = "\u2029"
// Needs to be trimmed for casting to float/double/decimal
val trimSpaceStr = ('\u0000' to '\u0020').toList.mkString
+ // ISOControl characters, refer java.lang.Character.isISOControl(int)
+ val isoControlStr = (('\u0000' to '\u001F') ++ ('\u007F' to '\u009F')).toList.mkString
// scalastyle:on nonascii
- c.dataType match {
- case BinaryType | _: ArrayType | _: MapType | _: StructType | _: UserDefinedType[_] =>
- c
- case FloatType | DoubleType | _: DecimalType =>
- c.child.dataType match {
- case StringType if GlutenConfig.getConf.castFromVarcharAddTrimNode =>
- val trimNode = StringTrim(c.child, Some(Literal(trimSpaceStr)))
- c.withNewChildren(Seq(trimNode)).asInstanceOf[Cast]
- case _ =>
- c
- }
- case _ =>
- c.child.dataType match {
- case StringType if GlutenConfig.getConf.castFromVarcharAddTrimNode =>
- val trimNode = StringTrim(
- c.child,
- Some(
- Literal(trimWhitespaceStr +
- trimSpaceSepStr + trimLineSepStr + trimParaSepStr)))
- c.withNewChildren(Seq(trimNode)).asInstanceOf[Cast]
- case _ =>
- c
+ if (GlutenConfig.getConf.castFromVarcharAddTrimNode && c.child.dataType == StringType) {
+ val trimStr = c.dataType match {
+ case BinaryType | _: ArrayType | _: MapType | _: StructType | _: UserDefinedType[_] =>
+ None
+ case FloatType | DoubleType | _: DecimalType =>
+ Some(trimSpaceStr)
+ case _ =>
+ Some(
+ (trimWhitespaceStr + trimSpaceSepStr + trimLineSepStr
+ + trimParaSepStr + isoControlStr).toSet.mkString
+ )
+ }
+ trimStr
+ .map {
+ trim =>
+ c.withNewChildren(Seq(StringTrim(c.child, Some(Literal(trim))))).asInstanceOf[Cast]
}
+ .getOrElse(c)
+ } else {
+ c
}
}
@@ -807,7 +807,7 @@ class VeloxSparkPlanExecApi extends SparkPlanExecApi {
override def maybeCollapseTakeOrderedAndProject(plan: SparkPlan): SparkPlan = {
// This to-top-n optimization assumes exchange operators were already placed in input plan.
plan.transformUp {
- case p @ LimitTransformer(SortExecTransformer(sortOrder, _, child, _), 0, count) =>
+ case p @ LimitExecTransformer(SortExecTransformer(sortOrder, _, child, _), 0, count) =>
val global = child.outputPartitioning.satisfies(AllTuples)
val topN = TopNTransformer(count, sortOrder, global, child)
if (topN.doValidate().ok()) {
diff --git a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxTransformerApi.scala b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxTransformerApi.scala
index 1687f24ce35e..c6d2bc065879 100644
--- a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxTransformerApi.scala
+++ b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxTransformerApi.scala
@@ -16,7 +16,7 @@
*/
package org.apache.gluten.backendsapi.velox
-import org.apache.gluten.backendsapi.TransformerApi
+import org.apache.gluten.backendsapi.{BackendsApiManager, TransformerApi}
import org.apache.gluten.execution.WriteFilesExecTransformer
import org.apache.gluten.expression.ConverterUtils
import org.apache.gluten.runtime.Runtimes
@@ -87,7 +87,9 @@ class VeloxTransformerApi extends TransformerApi with Logging {
override def getNativePlanString(substraitPlan: Array[Byte], details: Boolean): String = {
TaskResources.runUnsafe {
val jniWrapper = PlanEvaluatorJniWrapper.create(
- Runtimes.contextInstance("VeloxTransformerApi#getNativePlanString"))
+ Runtimes.contextInstance(
+ BackendsApiManager.getBackendName,
+ "VeloxTransformerApi#getNativePlanString"))
jniWrapper.nativePlanString(substraitPlan, details)
}
}
diff --git a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxValidatorApi.scala b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxValidatorApi.scala
index ddf77e5fa3d4..9b3001366198 100644
--- a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxValidatorApi.scala
+++ b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxValidatorApi.scala
@@ -16,7 +16,7 @@
*/
package org.apache.gluten.backendsapi.velox
-import org.apache.gluten.backendsapi.ValidatorApi
+import org.apache.gluten.backendsapi.{BackendsApiManager, ValidatorApi}
import org.apache.gluten.extension.ValidationResult
import org.apache.gluten.substrait.plan.PlanNode
import org.apache.gluten.validate.NativePlanValidationInfo
@@ -38,7 +38,7 @@ class VeloxValidatorApi extends ValidatorApi {
override def doNativeValidateWithFailureReason(plan: PlanNode): ValidationResult = {
TaskResources.runUnsafe {
- val validator = NativePlanEvaluator.create()
+ val validator = NativePlanEvaluator.create(BackendsApiManager.getBackendName)
asValidationResult(validator.doNativeValidateWithFailureReason(plan.toProtobuf.toByteArray))
}
}
diff --git a/backends-velox/src/main/scala/org/apache/gluten/columnarbatch/VeloxBatch.scala b/backends-velox/src/main/scala/org/apache/gluten/columnarbatch/VeloxBatch.scala
index 0c7600c856b1..5d9a78d31874 100644
--- a/backends-velox/src/main/scala/org/apache/gluten/columnarbatch/VeloxBatch.scala
+++ b/backends-velox/src/main/scala/org/apache/gluten/columnarbatch/VeloxBatch.scala
@@ -20,8 +20,10 @@ import org.apache.gluten.execution.{ArrowColumnarToVeloxColumnarExec, RowToVelox
import org.apache.gluten.extension.columnar.transition.{Convention, Transition}
object VeloxBatch extends Convention.BatchType {
- fromRow(RowToVeloxColumnarExec.apply)
- toRow(VeloxColumnarToRowExec.apply)
- fromBatch(ArrowBatches.ArrowNativeBatch, ArrowColumnarToVeloxColumnarExec.apply)
- toBatch(ArrowBatches.ArrowNativeBatch, Transition.empty)
+ override protected def registerTransitions(): Unit = {
+ fromRow(RowToVeloxColumnarExec.apply)
+ toRow(VeloxColumnarToRowExec.apply)
+ fromBatch(ArrowBatches.ArrowNativeBatch, ArrowColumnarToVeloxColumnarExec.apply)
+ toBatch(ArrowBatches.ArrowNativeBatch, Transition.empty)
+ }
}
diff --git a/backends-velox/src/main/scala/org/apache/gluten/datasource/VeloxDataSourceUtil.scala b/backends-velox/src/main/scala/org/apache/gluten/datasource/VeloxDataSourceUtil.scala
index fe0d0eb0f8f2..7ea67feb1f8d 100644
--- a/backends-velox/src/main/scala/org/apache/gluten/datasource/VeloxDataSourceUtil.scala
+++ b/backends-velox/src/main/scala/org/apache/gluten/datasource/VeloxDataSourceUtil.scala
@@ -16,6 +16,7 @@
*/
package org.apache.gluten.datasource
+import org.apache.gluten.backendsapi.BackendsApiManager
import org.apache.gluten.memory.arrow.alloc.ArrowBufferAllocators
import org.apache.gluten.runtime.Runtimes
import org.apache.gluten.utils.ArrowAbiUtil
@@ -38,7 +39,7 @@ object VeloxDataSourceUtil {
def readSchema(file: FileStatus): Option[StructType] = {
val allocator = ArrowBufferAllocators.contextInstance()
- val runtime = Runtimes.contextInstance("VeloxWriter")
+ val runtime = Runtimes.contextInstance(BackendsApiManager.getBackendName, "VeloxWriter")
val datasourceJniWrapper = VeloxDataSourceJniWrapper.create(runtime)
val dsHandle =
datasourceJniWrapper.init(file.getPath.toString, -1, new util.HashMap[String, String]())
diff --git a/backends-velox/src/main/scala/org/apache/gluten/execution/ColumnarPartialProjectExec.scala b/backends-velox/src/main/scala/org/apache/gluten/execution/ColumnarPartialProjectExec.scala
index d993e399dbf4..576f3a2cb205 100644
--- a/backends-velox/src/main/scala/org/apache/gluten/execution/ColumnarPartialProjectExec.scala
+++ b/backends-velox/src/main/scala/org/apache/gluten/execution/ColumnarPartialProjectExec.scala
@@ -17,9 +17,11 @@
package org.apache.gluten.execution
import org.apache.gluten.GlutenConfig
+import org.apache.gluten.backendsapi.BackendsApiManager
import org.apache.gluten.columnarbatch.{ColumnarBatches, VeloxColumnarBatches}
import org.apache.gluten.expression.{ArrowProjection, ExpressionUtils}
-import org.apache.gluten.extension.{GlutenPlan, ValidationResult}
+import org.apache.gluten.extension.ValidationResult
+import org.apache.gluten.extension.columnar.transition.Convention
import org.apache.gluten.iterator.Iterators
import org.apache.gluten.memory.arrow.alloc.ArrowBufferAllocators
import org.apache.gluten.sql.shims.SparkShimLoader
@@ -31,7 +33,7 @@ import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeRef
import org.apache.spark.sql.execution.{ExplainUtils, ProjectExec, SparkPlan, UnaryExecNode}
import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics}
import org.apache.spark.sql.hive.HiveUdfUtil
-import org.apache.spark.sql.types.{BinaryType, BooleanType, ByteType, DataType, DateType, DecimalType, DoubleType, FloatType, IntegerType, LongType, NullType, ShortType, StringType, TimestampType, YearMonthIntervalType}
+import org.apache.spark.sql.types._
import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector}
import scala.collection.mutable.ListBuffer
@@ -51,7 +53,7 @@ import scala.collection.mutable.ListBuffer
case class ColumnarPartialProjectExec(original: ProjectExec, child: SparkPlan)(
replacedAliasUdf: Seq[Alias])
extends UnaryExecNode
- with GlutenPlan {
+ with ValidatablePlan {
private val projectAttributes: ListBuffer[Attribute] = ListBuffer()
private val projectIndexInChild: ListBuffer[Int] = ListBuffer()
@@ -73,6 +75,10 @@ case class ColumnarPartialProjectExec(original: ProjectExec, child: SparkPlan)(
override def output: Seq[Attribute] = child.output ++ replacedAliasUdf.map(_.toAttribute)
+ override def batchType(): Convention.BatchType = BackendsApiManager.getSettings.primaryBatchType
+
+ override def rowType0(): Convention.RowType = Convention.RowType.None
+
final override def doExecute(): RDD[InternalRow] = {
throw new UnsupportedOperationException(
s"${this.getClass.getSimpleName} doesn't support doExecute")
@@ -82,8 +88,6 @@ case class ColumnarPartialProjectExec(original: ProjectExec, child: SparkPlan)(
replacedAliasUdf :: Nil
}
- final override val supportsColumnar: Boolean = true
-
private def validateExpression(expr: Expression): Boolean = {
expr.deterministic && !expr.isInstanceOf[LambdaFunction] && expr.children
.forall(validateExpression)
@@ -184,7 +188,8 @@ case class ColumnarPartialProjectExec(original: ProjectExec, child: SparkPlan)(
Iterator.empty
} else {
val start = System.currentTimeMillis()
- val childData = ColumnarBatches.select(batch, projectIndexInChild.toArray)
+ val childData = ColumnarBatches
+ .select(BackendsApiManager.getBackendName, batch, projectIndexInChild.toArray)
val projectedBatch = getProjectedBatchArrow(childData, c2a, a2c)
val batchIterator = projectedBatch.map {
diff --git a/backends-velox/src/main/scala/org/apache/gluten/execution/HashAggregateExecTransformer.scala b/backends-velox/src/main/scala/org/apache/gluten/execution/HashAggregateExecTransformer.scala
index fe5e0d92d6d5..9a6ba3a220f1 100644
--- a/backends-velox/src/main/scala/org/apache/gluten/execution/HashAggregateExecTransformer.scala
+++ b/backends-velox/src/main/scala/org/apache/gluten/execution/HashAggregateExecTransformer.scala
@@ -72,7 +72,7 @@ abstract class HashAggregateExecTransformer(
val aggParams = new AggregationParams
val operatorId = context.nextOperatorId(this.nodeName)
val relNode = getAggRel(context, operatorId, aggParams, childCtx.root)
- TransformContext(childCtx.outputAttributes, output, relNode)
+ TransformContext(output, relNode)
}
// Return whether the outputs partial aggregation should be combined for Velox computing.
@@ -214,7 +214,7 @@ abstract class HashAggregateExecTransformer(
VeloxIntermediateData.getIntermediateTypeNode(aggregateFunction)
)
aggregateNodeList.add(aggFunctionNode)
- case Final =>
+ case Final | Complete =>
val aggFunctionNode = ExpressionBuilder.makeAggregateFunction(
VeloxAggregateFunctionsBuilder.create(args, aggregateFunction, aggregateMode),
childrenNodeList,
@@ -242,7 +242,7 @@ abstract class HashAggregateExecTransformer(
aggregateFunction.inputAggBufferAttributes.head.nullable)
)
aggregateNodeList.add(partialNode)
- case Final =>
+ case Final | Complete =>
val aggFunctionNode = ExpressionBuilder.makeAggregateFunction(
VeloxAggregateFunctionsBuilder.create(args, aggregateFunction, aggregateMode),
childrenNodeList,
@@ -275,7 +275,7 @@ abstract class HashAggregateExecTransformer(
expression.mode match {
case Partial | PartialMerge =>
typeNodeList.add(VeloxIntermediateData.getIntermediateTypeNode(aggregateFunction))
- case Final =>
+ case Final | Complete =>
typeNodeList.add(
ConverterUtils
.getTypeNode(aggregateFunction.dataType, aggregateFunction.nullable))
@@ -356,7 +356,7 @@ abstract class HashAggregateExecTransformer(
// The process of handling the inconsistency in column types and order between
// Spark and Velox is exactly the opposite of applyExtractStruct.
aggregateExpression.mode match {
- case PartialMerge | Final =>
+ case PartialMerge | Final | Complete =>
val newInputAttributes = new ArrayBuffer[Attribute]()
val childNodes = new JArrayList[ExpressionNode]()
val (sparkOrders, sparkTypes) =
@@ -467,7 +467,7 @@ abstract class HashAggregateExecTransformer(
// by previous projection.
childrenNodes.add(ExpressionBuilder.makeSelection(colIdx))
colIdx += 1
- case Partial =>
+ case Partial | Complete =>
aggFunc.children.foreach {
_ =>
childrenNodes.add(ExpressionBuilder.makeSelection(colIdx))
@@ -600,7 +600,7 @@ abstract class HashAggregateExecTransformer(
}
val aggregateFunc = aggExpr.aggregateFunction
val childrenNodes = aggExpr.mode match {
- case Partial =>
+ case Partial | Complete =>
aggregateFunc.children.toList.map(
expr => {
ExpressionConverter
@@ -784,7 +784,7 @@ case class HashAggregateExecPullOutHelper(
expr.mode match {
case Partial | PartialMerge =>
expr.aggregateFunction.aggBufferAttributes
- case Final =>
+ case Final | Complete =>
Seq(aggregateAttributes(index))
case other =>
throw new GlutenNotSupportException(s"Unsupported aggregate mode: $other.")
diff --git a/backends-velox/src/main/scala/org/apache/gluten/execution/RowToVeloxColumnarExec.scala b/backends-velox/src/main/scala/org/apache/gluten/execution/RowToVeloxColumnarExec.scala
index a853778484b1..9cdcf854db8b 100644
--- a/backends-velox/src/main/scala/org/apache/gluten/execution/RowToVeloxColumnarExec.scala
+++ b/backends-velox/src/main/scala/org/apache/gluten/execution/RowToVeloxColumnarExec.scala
@@ -17,6 +17,7 @@
package org.apache.gluten.execution
import org.apache.gluten.GlutenConfig
+import org.apache.gluten.backendsapi.BackendsApiManager
import org.apache.gluten.columnarbatch.ColumnarBatches
import org.apache.gluten.iterator.Iterators
import org.apache.gluten.memory.arrow.alloc.ArrowBufferAllocators
@@ -43,7 +44,6 @@ import org.apache.arrow.memory.ArrowBuf
import scala.collection.mutable.ListBuffer
case class RowToVeloxColumnarExec(child: SparkPlan) extends RowToColumnarExecBase(child = child) {
-
override def doExecuteColumnarInternal(): RDD[ColumnarBatch] = {
val numInputRows = longMetric("numInputRows")
val numOutputBatches = longMetric("numOutputBatches")
@@ -122,7 +122,7 @@ object RowToVeloxColumnarExec {
val arrowSchema =
SparkArrowUtil.toArrowSchema(schema, SQLConf.get.sessionLocalTimeZone)
- val runtime = Runtimes.contextInstance("RowToColumnar")
+ val runtime = Runtimes.contextInstance(BackendsApiManager.getBackendName, "RowToColumnar")
val jniWrapper = NativeRowToColumnarJniWrapper.create(runtime)
val arrowAllocator = ArrowBufferAllocators.contextInstance()
val cSchema = ArrowSchema.allocateNew(arrowAllocator)
diff --git a/backends-velox/src/main/scala/org/apache/gluten/execution/TopNTransformer.scala b/backends-velox/src/main/scala/org/apache/gluten/execution/TopNTransformer.scala
index 01c89bee217b..f3adbe351aa4 100644
--- a/backends-velox/src/main/scala/org/apache/gluten/execution/TopNTransformer.scala
+++ b/backends-velox/src/main/scala/org/apache/gluten/execution/TopNTransformer.scala
@@ -79,7 +79,7 @@ case class TopNTransformer(
child.output,
childCtx.root,
validation = false)
- TransformContext(child.output, child.output, relNode)
+ TransformContext(child.output, relNode)
}
private def getRelNode(
diff --git a/backends-velox/src/main/scala/org/apache/gluten/execution/VeloxColumnarToRowExec.scala b/backends-velox/src/main/scala/org/apache/gluten/execution/VeloxColumnarToRowExec.scala
index 8aedeb87cb28..4c0f79538c0f 100644
--- a/backends-velox/src/main/scala/org/apache/gluten/execution/VeloxColumnarToRowExec.scala
+++ b/backends-velox/src/main/scala/org/apache/gluten/execution/VeloxColumnarToRowExec.scala
@@ -16,6 +16,7 @@
*/
package org.apache.gluten.execution
+import org.apache.gluten.backendsapi.BackendsApiManager
import org.apache.gluten.columnarbatch.{ColumnarBatches, VeloxColumnarBatches}
import org.apache.gluten.exception.GlutenNotSupportException
import org.apache.gluten.extension.ValidationResult
@@ -122,7 +123,7 @@ object VeloxColumnarToRowExec {
return Iterator.empty
}
- val runtime = Runtimes.contextInstance("ColumnarToRow")
+ val runtime = Runtimes.contextInstance(BackendsApiManager.getBackendName, "ColumnarToRow")
// TODO: Pass the jni jniWrapper and arrowSchema and serializeSchema method by broadcast.
val jniWrapper = NativeColumnarToRowJniWrapper.create(runtime)
val c2rId = jniWrapper.nativeColumnarToRowInit()
@@ -156,7 +157,7 @@ object VeloxColumnarToRowExec {
val cols = batch.numCols()
val rows = batch.numRows()
val beforeConvert = System.currentTimeMillis()
- val batchHandle = ColumnarBatches.getNativeHandle(batch)
+ val batchHandle = ColumnarBatches.getNativeHandle(BackendsApiManager.getBackendName, batch)
var info =
jniWrapper.nativeColumnarToRowConvert(c2rId, batchHandle, 0)
diff --git a/backends-velox/src/main/scala/org/apache/gluten/execution/VeloxResizeBatchesExec.scala b/backends-velox/src/main/scala/org/apache/gluten/execution/VeloxResizeBatchesExec.scala
index 995582024ba5..9ed687d33703 100644
--- a/backends-velox/src/main/scala/org/apache/gluten/execution/VeloxResizeBatchesExec.scala
+++ b/backends-velox/src/main/scala/org/apache/gluten/execution/VeloxResizeBatchesExec.scala
@@ -16,7 +16,8 @@
*/
package org.apache.gluten.execution
-import org.apache.gluten.extension.GlutenPlan
+import org.apache.gluten.backendsapi.BackendsApiManager
+import org.apache.gluten.extension.columnar.transition.Convention
import org.apache.gluten.iterator.Iterators
import org.apache.gluten.utils.VeloxBatchResizer
@@ -53,7 +54,10 @@ case class VeloxResizeBatchesExec(
"selfTime" -> SQLMetrics.createTimingMetric(sparkContext, "time to append / split batches")
)
- override def supportsColumnar: Boolean = true
+ override def batchType(): Convention.BatchType = BackendsApiManager.getSettings.primaryBatchType
+
+ override def rowType0(): Convention.RowType = Convention.RowType.None
+
override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException()
override protected def doExecuteColumnar(): RDD[ColumnarBatch] = {
diff --git a/backends-velox/src/main/scala/org/apache/gluten/datasource/ArrowConvertorRule.scala b/backends-velox/src/main/scala/org/apache/gluten/extension/ArrowConvertorRule.scala
similarity index 97%
rename from backends-velox/src/main/scala/org/apache/gluten/datasource/ArrowConvertorRule.scala
rename to backends-velox/src/main/scala/org/apache/gluten/extension/ArrowConvertorRule.scala
index b1b0b813f6c8..5e02cf54b0e5 100644
--- a/backends-velox/src/main/scala/org/apache/gluten/datasource/ArrowConvertorRule.scala
+++ b/backends-velox/src/main/scala/org/apache/gluten/extension/ArrowConvertorRule.scala
@@ -14,9 +14,10 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.gluten.datasource
+package org.apache.gluten.extension
import org.apache.gluten.backendsapi.BackendsApiManager
+import org.apache.gluten.datasource.ArrowCSVFileFormat
import org.apache.gluten.datasource.v2.ArrowCSVTable
import org.apache.gluten.sql.shims.SparkShimLoader
diff --git a/backends-velox/src/main/scala/org/apache/gluten/metrics/InputIteratorMetricsUpdater.scala b/backends-velox/src/main/scala/org/apache/gluten/metrics/InputIteratorMetricsUpdater.scala
index a9067d069e03..8002a44ae92b 100644
--- a/backends-velox/src/main/scala/org/apache/gluten/metrics/InputIteratorMetricsUpdater.scala
+++ b/backends-velox/src/main/scala/org/apache/gluten/metrics/InputIteratorMetricsUpdater.scala
@@ -17,20 +17,23 @@
package org.apache.gluten.metrics
import org.apache.spark.sql.execution.metric.SQLMetric
-case class InputIteratorMetricsUpdater(metrics: Map[String, SQLMetric]) extends MetricsUpdater {
+case class InputIteratorMetricsUpdater(metrics: Map[String, SQLMetric], forBroadcast: Boolean)
+ extends MetricsUpdater {
override def updateNativeMetrics(opMetrics: IOperatorMetrics): Unit = {
if (opMetrics != null) {
val operatorMetrics = opMetrics.asInstanceOf[OperatorMetrics]
metrics("cpuCount") += operatorMetrics.cpuCount
metrics("wallNanos") += operatorMetrics.wallNanos
- if (operatorMetrics.outputRows == 0 && operatorMetrics.outputVectors == 0) {
- // Sometimes, velox does not update metrics for intermediate operator,
- // here we try to use the input metrics
- metrics("numOutputRows") += operatorMetrics.inputRows
- metrics("outputVectors") += operatorMetrics.inputVectors
- } else {
- metrics("numOutputRows") += operatorMetrics.outputRows
- metrics("outputVectors") += operatorMetrics.outputVectors
+ if (!forBroadcast) {
+ if (operatorMetrics.outputRows == 0 && operatorMetrics.outputVectors == 0) {
+ // Sometimes, velox does not update metrics for intermediate operator,
+ // here we try to use the input metrics
+ metrics("numOutputRows") += operatorMetrics.inputRows
+ metrics("outputVectors") += operatorMetrics.inputVectors
+ } else {
+ metrics("numOutputRows") += operatorMetrics.outputRows
+ metrics("outputVectors") += operatorMetrics.outputVectors
+ }
}
}
}
diff --git a/backends-velox/src/main/scala/org/apache/gluten/metrics/MetricsUtil.scala b/backends-velox/src/main/scala/org/apache/gluten/metrics/MetricsUtil.scala
index cd50d0b8e20c..b8ef1620f905 100644
--- a/backends-velox/src/main/scala/org/apache/gluten/metrics/MetricsUtil.scala
+++ b/backends-velox/src/main/scala/org/apache/gluten/metrics/MetricsUtil.scala
@@ -58,7 +58,8 @@ object MetricsUtil extends Logging {
assert(t.children.size == 1, "MetricsUpdater.None can only be used on unary operator")
treeifyMetricsUpdaters(t.children.head)
case t: TransformSupport =>
- MetricsUpdaterTree(t.metricsUpdater(), t.children.map(treeifyMetricsUpdaters))
+ // Reversed children order to match the traversal code.
+ MetricsUpdaterTree(t.metricsUpdater(), t.children.reverse.map(treeifyMetricsUpdaters))
case _ =>
MetricsUpdaterTree(MetricsUpdater.Terminate, Seq())
}
@@ -233,6 +234,12 @@ object MetricsUtil extends Logging {
operatorMetrics,
metrics.getSingleMetrics,
joinParamsMap.get(operatorIdx))
+ case u: UnionMetricsUpdater =>
+ // JoinRel outputs two suites of metrics respectively for hash build and hash probe.
+ // Therefore, fetch one more suite of metrics here.
+ operatorMetrics.add(metrics.getOperatorMetrics(curMetricsIdx))
+ curMetricsIdx -= 1
+ u.updateUnionMetrics(operatorMetrics)
case hau: HashAggregateMetricsUpdater =>
hau.updateAggregationMetrics(operatorMetrics, aggParamsMap.get(operatorIdx))
case lu: LimitMetricsUpdater =>
diff --git a/backends-velox/src/main/scala/org/apache/gluten/metrics/UnionMetricsUpdater.scala b/backends-velox/src/main/scala/org/apache/gluten/metrics/UnionMetricsUpdater.scala
new file mode 100644
index 000000000000..9e91cf368c0a
--- /dev/null
+++ b/backends-velox/src/main/scala/org/apache/gluten/metrics/UnionMetricsUpdater.scala
@@ -0,0 +1,35 @@
+/*
+ * 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.
+ */
+package org.apache.gluten.metrics
+
+import org.apache.spark.sql.execution.metric.SQLMetric
+
+class UnionMetricsUpdater(val metrics: Map[String, SQLMetric]) extends MetricsUpdater {
+ override def updateNativeMetrics(opMetrics: IOperatorMetrics): Unit = {
+ throw new UnsupportedOperationException()
+ }
+
+ def updateUnionMetrics(unionMetrics: java.util.ArrayList[OperatorMetrics]): Unit = {
+ // Union was interpreted to LocalExchange + LocalPartition. Use metrics from LocalExchange.
+ val localExchangeMetrics = unionMetrics.get(0)
+ metrics("numInputRows") += localExchangeMetrics.inputRows
+ metrics("inputVectors") += localExchangeMetrics.inputVectors
+ metrics("inputBytes") += localExchangeMetrics.inputBytes
+ metrics("cpuCount") += localExchangeMetrics.cpuCount
+ metrics("wallNanos") += localExchangeMetrics.wallNanos
+ }
+}
diff --git a/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderCentos7.scala b/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderCentos7.scala
index 16cd18e41a05..b9d90d589c02 100755
--- a/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderCentos7.scala
+++ b/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderCentos7.scala
@@ -37,7 +37,6 @@ class SharedLibraryLoaderCentos7 extends SharedLibraryLoader {
loader.loadAndCreateLink("libntlm.so.0", "libntlm.so", false)
loader.loadAndCreateLink("libgsasl.so.7", "libgsasl.so", false)
loader.loadAndCreateLink("libprotobuf.so.32", "libprotobuf.so", false)
- loader.loadAndCreateLink("libhdfs.so.0.0.0", "libhdfs.so", false)
loader.loadAndCreateLink("libre2.so.10", "libre2.so", false)
loader.loadAndCreateLink("libzstd.so.1", "libzstd.so", false)
loader.loadAndCreateLink("liblz4.so.1", "liblz4.so", false)
diff --git a/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderCentos8.scala b/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderCentos8.scala
index 0a75c30c22ff..dbb7d59f889d 100755
--- a/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderCentos8.scala
+++ b/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderCentos8.scala
@@ -42,7 +42,6 @@ class SharedLibraryLoaderCentos8 extends SharedLibraryLoader {
loader.loadAndCreateLink("libntlm.so.0", "libntlm.so", false)
loader.loadAndCreateLink("libgsasl.so.7", "libgsasl.so", false)
loader.loadAndCreateLink("libprotobuf.so.32", "libprotobuf.so", false)
- loader.loadAndCreateLink("libhdfs.so.0.0.0", "libhdfs.so", false)
loader.loadAndCreateLink("libre2.so.0", "libre2.so", false)
loader.loadAndCreateLink("libsodium.so.23", "libsodium.so", false)
}
diff --git a/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderCentos9.scala b/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderCentos9.scala
index 50f9fe4aaadc..f633a79f8fcd 100755
--- a/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderCentos9.scala
+++ b/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderCentos9.scala
@@ -42,7 +42,6 @@ class SharedLibraryLoaderCentos9 extends SharedLibraryLoader {
loader.loadAndCreateLink("libntlm.so.0", "libntlm.so", false)
loader.loadAndCreateLink("libgsasl.so.7", "libgsasl.so", false)
loader.loadAndCreateLink("libprotobuf.so.32", "libprotobuf.so", false)
- loader.loadAndCreateLink("libhdfs.so.0.0.0", "libhdfs.so", false)
loader.loadAndCreateLink("libre2.so.9", "libre2.so", false)
loader.loadAndCreateLink("libsodium.so.23", "libsodium.so", false)
}
diff --git a/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderDebian11.scala b/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderDebian11.scala
index 06c065ba2883..4f9b6fdd7250 100644
--- a/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderDebian11.scala
+++ b/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderDebian11.scala
@@ -44,6 +44,5 @@ class SharedLibraryLoaderDebian11 extends SharedLibraryLoader {
loader.loadAndCreateLink("libsnappy.so.1", "libsnappy.so", false)
loader.loadAndCreateLink("libcurl.so.4", "libcurl.so", false)
loader.loadAndCreateLink("libprotobuf.so.32", "libprotobuf.so", false)
- loader.loadAndCreateLink("libhdfs.so.0.0.0", "libhdfs.so", false)
}
}
diff --git a/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderDebian12.scala b/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderDebian12.scala
index 8018995328fe..e3967eea3267 100644
--- a/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderDebian12.scala
+++ b/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderDebian12.scala
@@ -50,6 +50,5 @@ class SharedLibraryLoaderDebian12 extends SharedLibraryLoader {
loader.loadAndCreateLink("libevent-2.1.so.7", "libevent-2.1.so", false)
loader.loadAndCreateLink("libcurl.so.4", "libcurl.so", false)
loader.loadAndCreateLink("libprotobuf.so.32", "libprotobuf.so", false)
- loader.loadAndCreateLink("libhdfs.so.0.0.0", "libhdfs.so", false)
}
}
diff --git a/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderUbuntu2004.scala b/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderUbuntu2004.scala
index d1f21a0013fb..6d9271e9e3d9 100755
--- a/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderUbuntu2004.scala
+++ b/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderUbuntu2004.scala
@@ -57,7 +57,6 @@ class SharedLibraryLoaderUbuntu2004 extends SharedLibraryLoader {
loader.loadAndCreateLink("libicudata.so.66", "libicudata.so", false)
loader.loadAndCreateLink("libicuuc.so.66", "libicuuc.so", false)
loader.loadAndCreateLink("libxml2.so.2", "libxml2.so", false)
- loader.loadAndCreateLink("libhdfs.so.0.0.0", "libhdfs.so", false)
loader.loadAndCreateLink("libre2.so.5", "libre2.so", false)
loader.loadAndCreateLink("libsnappy.so.1", "libsnappy.so", false)
loader.loadAndCreateLink("libthrift-0.13.0.so", "libthrift.so", false)
diff --git a/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderUbuntu2204.scala b/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderUbuntu2204.scala
index 3cf4d30237ac..95f7db5655b2 100755
--- a/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderUbuntu2204.scala
+++ b/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderUbuntu2204.scala
@@ -42,7 +42,6 @@ class SharedLibraryLoaderUbuntu2204 extends SharedLibraryLoader {
loader.loadAndCreateLink("libgsasl.so.7", "libgsasl.so", false)
loader.loadAndCreateLink("libprotobuf.so.32", "libprotobuf.so", false)
loader.loadAndCreateLink("libxml2.so.2", "libxml2.so", false)
- loader.loadAndCreateLink("libhdfs.so.0.0.0", "libhdfs.so", false)
loader.loadAndCreateLink("libre2.so.9", "libre2.so", false)
loader.loadAndCreateLink("libsnappy.so.1", "libsnappy.so", false)
loader.loadAndCreateLink("libthrift-0.16.0.so", "libthrift.so", false)
diff --git a/backends-velox/src/main/scala/org/apache/gluten/vectorized/ColumnarBatchSerializer.scala b/backends-velox/src/main/scala/org/apache/gluten/vectorized/ColumnarBatchSerializer.scala
index fa4d92652ca3..cd035e3202d2 100644
--- a/backends-velox/src/main/scala/org/apache/gluten/vectorized/ColumnarBatchSerializer.scala
+++ b/backends-velox/src/main/scala/org/apache/gluten/vectorized/ColumnarBatchSerializer.scala
@@ -17,6 +17,7 @@
package org.apache.gluten.vectorized
import org.apache.gluten.GlutenConfig
+import org.apache.gluten.backendsapi.BackendsApiManager
import org.apache.gluten.iterator.ClosableIterator
import org.apache.gluten.memory.arrow.alloc.ArrowBufferAllocators
import org.apache.gluten.runtime.Runtimes
@@ -98,13 +99,15 @@ private class ColumnarBatchSerializerInstance(
val compressionCodecBackend =
GlutenConfig.getConf.columnarShuffleCodecBackend.orNull
val batchSize = GlutenConfig.getConf.maxBatchSize
- val runtime = Runtimes.contextInstance("ShuffleReader")
+ val bufferSize = GlutenConfig.getConf.columnarShuffleReaderBufferSize
+ val runtime = Runtimes.contextInstance(BackendsApiManager.getBackendName, "ShuffleReader")
val jniWrapper = ShuffleReaderJniWrapper.create(runtime)
val shuffleReaderHandle = jniWrapper.make(
cSchema.memoryAddress(),
compressionCodec,
compressionCodecBackend,
batchSize,
+ bufferSize,
shuffleWriterType)
// Close shuffle reader instance as lately as the end of task processing,
// since the native reader could hold a reference to memory pool that
@@ -133,7 +136,8 @@ private class ColumnarBatchSerializerInstance(
extends DeserializationStream
with TaskResource {
private val byteIn: JniByteInputStream = JniByteInputStreams.create(in)
- private val runtime = Runtimes.contextInstance("ShuffleReader")
+ private val runtime =
+ Runtimes.contextInstance(BackendsApiManager.getBackendName, "ShuffleReader")
private val wrappedOut: ClosableIterator = new ColumnarBatchOutIterator(
runtime,
ShuffleReaderJniWrapper
diff --git a/backends-velox/src/main/scala/org/apache/spark/api/python/ColumnarArrowEvalPythonExec.scala b/backends-velox/src/main/scala/org/apache/spark/api/python/ColumnarArrowEvalPythonExec.scala
index f1f5eb906238..4a5f43b7fdfa 100644
--- a/backends-velox/src/main/scala/org/apache/spark/api/python/ColumnarArrowEvalPythonExec.scala
+++ b/backends-velox/src/main/scala/org/apache/spark/api/python/ColumnarArrowEvalPythonExec.scala
@@ -19,9 +19,8 @@ package org.apache.spark.api.python
import org.apache.gluten.columnarbatch.ArrowBatches.ArrowJavaBatch
import org.apache.gluten.columnarbatch.ColumnarBatches
import org.apache.gluten.exception.GlutenException
-import org.apache.gluten.extension.GlutenPlan
+import org.apache.gluten.execution.GlutenPlan
import org.apache.gluten.extension.columnar.transition.{Convention, ConventionReq}
-import org.apache.gluten.extension.columnar.transition.ConventionReq.KnownChildrenConventions
import org.apache.gluten.iterator.Iterators
import org.apache.gluten.memory.arrow.alloc.ArrowBufferAllocators
import org.apache.gluten.utils.PullOutProjectHelper
@@ -212,14 +211,14 @@ case class ColumnarArrowEvalPythonExec(
child: SparkPlan,
evalType: Int)
extends EvalPythonExec
- with GlutenPlan
- with KnownChildrenConventions {
- override def supportsColumnar: Boolean = true
+ with GlutenPlan {
- override protected def batchType0(): Convention.BatchType = ArrowJavaBatch
+ override def batchType(): Convention.BatchType = ArrowJavaBatch
- override def requiredChildrenConventions(): Seq[ConventionReq] = List(
- ConventionReq.of(ConventionReq.RowType.Any, ConventionReq.BatchType.Is(ArrowJavaBatch)))
+ override def rowType0(): Convention.RowType = Convention.RowType.None
+
+ override def requiredChildConvention(): Seq[ConventionReq] = List(
+ ConventionReq.ofBatch(ConventionReq.BatchType.Is(ArrowJavaBatch)))
override lazy val metrics = Map(
"numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"),
diff --git a/backends-velox/src/main/scala/org/apache/spark/shuffle/ColumnarShuffleWriter.scala b/backends-velox/src/main/scala/org/apache/spark/shuffle/ColumnarShuffleWriter.scala
index eaf9d99a9ecc..e9f821512e16 100644
--- a/backends-velox/src/main/scala/org/apache/spark/shuffle/ColumnarShuffleWriter.scala
+++ b/backends-velox/src/main/scala/org/apache/spark/shuffle/ColumnarShuffleWriter.scala
@@ -17,6 +17,7 @@
package org.apache.spark.shuffle
import org.apache.gluten.GlutenConfig
+import org.apache.gluten.backendsapi.BackendsApiManager
import org.apache.gluten.columnarbatch.ColumnarBatches
import org.apache.gluten.memory.memtarget.{MemoryTarget, Spiller, Spillers}
import org.apache.gluten.runtime.Runtimes
@@ -99,7 +100,7 @@ class ColumnarShuffleWriter[K, V](
private val reallocThreshold = GlutenConfig.getConf.columnarShuffleReallocThreshold
- private val runtime = Runtimes.contextInstance("ShuffleWriter")
+ private val runtime = Runtimes.contextInstance(BackendsApiManager.getBackendName, "ShuffleWriter")
private val jniWrapper = ShuffleWriterJniWrapper.create(runtime)
@@ -135,7 +136,7 @@ class ColumnarShuffleWriter[K, V](
logInfo(s"Skip ColumnarBatch of ${cb.numRows} rows, ${cb.numCols} cols")
} else {
val rows = cb.numRows()
- val handle = ColumnarBatches.getNativeHandle(cb)
+ val handle = ColumnarBatches.getNativeHandle(BackendsApiManager.getBackendName, cb)
if (nativeShuffleWriter == -1L) {
nativeShuffleWriter = jniWrapper.make(
dep.nativePartitioning.getShortName,
diff --git a/backends-velox/src/main/scala/org/apache/spark/sql/execution/ArrowFileSourceScanExec.scala b/backends-velox/src/main/scala/org/apache/spark/sql/execution/ArrowFileSourceScanExec.scala
index 85bc68223487..16b8fb0e9f6f 100644
--- a/backends-velox/src/main/scala/org/apache/spark/sql/execution/ArrowFileSourceScanExec.scala
+++ b/backends-velox/src/main/scala/org/apache/spark/sql/execution/ArrowFileSourceScanExec.scala
@@ -33,8 +33,6 @@ case class ArrowFileSourceScanExec(original: FileSourceScanExec)
override def output: Seq[Attribute] = original.output
- override def supportsColumnar: Boolean = original.supportsColumnar
-
override def doCanonicalize(): FileSourceScanExec = original.doCanonicalize()
override protected def doExecuteColumnar(): RDD[ColumnarBatch] = {
diff --git a/backends-velox/src/main/scala/org/apache/spark/sql/execution/BaseArrowScanExec.scala b/backends-velox/src/main/scala/org/apache/spark/sql/execution/BaseArrowScanExec.scala
index 38a6d1803d0b..1aacc1b95416 100644
--- a/backends-velox/src/main/scala/org/apache/spark/sql/execution/BaseArrowScanExec.scala
+++ b/backends-velox/src/main/scala/org/apache/spark/sql/execution/BaseArrowScanExec.scala
@@ -17,12 +17,13 @@
package org.apache.spark.sql.execution
import org.apache.gluten.columnarbatch.ArrowBatches
-import org.apache.gluten.extension.GlutenPlan
+import org.apache.gluten.execution.GlutenPlan
import org.apache.gluten.extension.columnar.transition.Convention
trait BaseArrowScanExec extends GlutenPlan {
-
- final override protected def batchType0(): Convention.BatchType = {
+ final override def batchType(): Convention.BatchType = {
ArrowBatches.ArrowJavaBatch
}
+
+ final override def rowType0(): Convention.RowType = Convention.RowType.None
}
diff --git a/backends-velox/src/main/scala/org/apache/spark/sql/execution/BroadcastUtils.scala b/backends-velox/src/main/scala/org/apache/spark/sql/execution/BroadcastUtils.scala
index 5bc6b7c56da5..c5323d4f8d50 100644
--- a/backends-velox/src/main/scala/org/apache/spark/sql/execution/BroadcastUtils.scala
+++ b/backends-velox/src/main/scala/org/apache/spark/sql/execution/BroadcastUtils.scala
@@ -16,6 +16,7 @@
*/
package org.apache.spark.sql.execution
+import org.apache.gluten.backendsapi.BackendsApiManager
import org.apache.gluten.columnarbatch.ColumnarBatches
import org.apache.gluten.runtime.Runtimes
import org.apache.gluten.sql.shims.SparkShimLoader
@@ -105,7 +106,8 @@ object BroadcastUtils {
}
ColumnarBuildSideRelation(
SparkShimLoader.getSparkShims.attributesFromStruct(schema),
- serialized)
+ serialized,
+ mode)
}
// Rebroadcast Velox relation.
context.broadcast(toRelation).asInstanceOf[Broadcast[T]]
@@ -123,7 +125,8 @@ object BroadcastUtils {
}
ColumnarBuildSideRelation(
SparkShimLoader.getSparkShims.attributesFromStruct(schema),
- serialized)
+ serialized,
+ mode)
}
// Rebroadcast Velox relation.
context.broadcast(toRelation).asInstanceOf[Broadcast[T]]
@@ -152,11 +155,14 @@ object BroadcastUtils {
if (filtered.isEmpty) {
return ColumnarBatchSerializeResult.EMPTY
}
- val handleArray = filtered.map(ColumnarBatches.getNativeHandle)
+ val handleArray =
+ filtered.map(b => ColumnarBatches.getNativeHandle(BackendsApiManager.getBackendName, b))
val serializeResult =
try {
ColumnarBatchSerializerJniWrapper
- .create(Runtimes.contextInstance("BroadcastUtils#serializeStream"))
+ .create(
+ Runtimes
+ .contextInstance(BackendsApiManager.getBackendName, "BroadcastUtils#serializeStream"))
.serialize(handleArray)
} finally {
filtered.foreach(ColumnarBatches.release)
diff --git a/backends-velox/src/main/scala/org/apache/spark/sql/execution/ColumnarBuildSideRelation.scala b/backends-velox/src/main/scala/org/apache/spark/sql/execution/ColumnarBuildSideRelation.scala
index feaf72f64fb2..977357990c43 100644
--- a/backends-velox/src/main/scala/org/apache/spark/sql/execution/ColumnarBuildSideRelation.scala
+++ b/backends-velox/src/main/scala/org/apache/spark/sql/execution/ColumnarBuildSideRelation.scala
@@ -16,6 +16,7 @@
*/
package org.apache.spark.sql.execution
+import org.apache.gluten.backendsapi.BackendsApiManager
import org.apache.gluten.columnarbatch.ColumnarBatches
import org.apache.gluten.iterator.Iterators
import org.apache.gluten.memory.arrow.alloc.ArrowBufferAllocators
@@ -26,7 +27,10 @@ import org.apache.gluten.vectorized.{ColumnarBatchSerializerJniWrapper, NativeCo
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, UnsafeProjection, UnsafeRow}
+import org.apache.spark.sql.catalyst.plans.physical.BroadcastMode
+import org.apache.spark.sql.catalyst.plans.physical.IdentityBroadcastMode
import org.apache.spark.sql.execution.joins.BuildSideRelation
+import org.apache.spark.sql.execution.joins.HashedRelationBroadcastMode
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.utils.SparkArrowUtil
import org.apache.spark.sql.vectorized.ColumnarBatch
@@ -36,11 +40,22 @@ import org.apache.arrow.c.ArrowSchema
import scala.collection.JavaConverters.asScalaIteratorConverter
-case class ColumnarBuildSideRelation(output: Seq[Attribute], batches: Array[Array[Byte]])
+case class ColumnarBuildSideRelation(
+ output: Seq[Attribute],
+ batches: Array[Array[Byte]],
+ mode: BroadcastMode)
extends BuildSideRelation {
+ private def transformProjection: UnsafeProjection = {
+ mode match {
+ case HashedRelationBroadcastMode(k, _) => UnsafeProjection.create(k)
+ case IdentityBroadcastMode => UnsafeProjection.create(output, output)
+ }
+ }
+
override def deserialized: Iterator[ColumnarBatch] = {
- val runtime = Runtimes.contextInstance("BuildSideRelation#deserialized")
+ val runtime =
+ Runtimes.contextInstance(BackendsApiManager.getBackendName, "BuildSideRelation#deserialized")
val jniWrapper = ColumnarBatchSerializerJniWrapper.create(runtime)
val serializeHandle: Long = {
val allocator = ArrowBufferAllocators.contextInstance()
@@ -82,11 +97,15 @@ case class ColumnarBuildSideRelation(output: Seq[Attribute], batches: Array[Arra
override def asReadOnlyCopy(): ColumnarBuildSideRelation = this
/**
- * Transform columnar broadcast value to Array[InternalRow] by key and distinct. NOTE: This method
- * was called in Spark Driver, should manage resources carefully.
+ * Transform columnar broadcast value to Array[InternalRow] by key.
+ *
+ * NOTE:
+ * - This method was called in Spark Driver, should manage resources carefully.
+ * - The "key" must be already been bound reference.
*/
override def transform(key: Expression): Array[InternalRow] = TaskResources.runUnsafe {
- val runtime = Runtimes.contextInstance("BuildSideRelation#transform")
+ val runtime =
+ Runtimes.contextInstance(BackendsApiManager.getBackendName, "BuildSideRelation#transform")
// This transformation happens in Spark driver, thus resources can not be managed automatically.
val serializerJniWrapper = ColumnarBatchSerializerJniWrapper.create(runtime)
val serializeHandle = {
@@ -103,6 +122,8 @@ case class ColumnarBuildSideRelation(output: Seq[Attribute], batches: Array[Arra
var closed = false
+ val proj = UnsafeProjection.create(Seq(key))
+
// Convert columnar to Row.
val jniWrapper = NativeColumnarToRowJniWrapper.create(runtime)
val c2rId = jniWrapper.nativeColumnarToRowInit()
@@ -138,10 +159,9 @@ case class ColumnarBuildSideRelation(output: Seq[Attribute], batches: Array[Arra
var info =
jniWrapper.nativeColumnarToRowConvert(
c2rId,
- ColumnarBatches.getNativeHandle(batch),
+ ColumnarBatches.getNativeHandle(BackendsApiManager.getBackendName, batch),
0)
batch.close()
- val proj = UnsafeProjection.create(Seq(key), output)
new Iterator[InternalRow] {
var rowId = 0
@@ -164,7 +184,7 @@ case class ColumnarBuildSideRelation(output: Seq[Attribute], batches: Array[Arra
rowId += 1
row
}
- }.map(proj).map(_.copy())
+ }.map(transformProjection).map(proj).map(_.copy())
}
}
}
diff --git a/backends-velox/src/main/scala/org/apache/spark/sql/execution/ColumnarCachedBatchSerializer.scala b/backends-velox/src/main/scala/org/apache/spark/sql/execution/ColumnarCachedBatchSerializer.scala
index d6e7aa5b2244..64ad105c7f7c 100644
--- a/backends-velox/src/main/scala/org/apache/spark/sql/execution/ColumnarCachedBatchSerializer.scala
+++ b/backends-velox/src/main/scala/org/apache/spark/sql/execution/ColumnarCachedBatchSerializer.scala
@@ -46,7 +46,7 @@ case class CachedColumnarBatch(
bytes: Array[Byte])
extends CachedBatch {}
-// spotless:off
+// format: off
/**
* Feature:
* 1. This serializer supports column pruning
@@ -75,7 +75,7 @@ case class CachedColumnarBatch(
* - Deserializer DefaultCachedBatch -> InternalRow (unsupport ColumnarToRow)
* -> Convert DefaultCachedBatch to InternalRow using vanilla Spark serializer
*/
-// spotless:on
+// format: on
class ColumnarCachedBatchSerializer extends CachedBatchSerializer with SQLConfHelper with Logging {
private lazy val rowBasedCachedBatchSerializer = new DefaultCachedBatchSerializer
@@ -176,8 +176,12 @@ class ColumnarCachedBatchSerializer extends CachedBatchSerializer with SQLConfHe
val batch = it.next()
val results =
ColumnarBatchSerializerJniWrapper
- .create(Runtimes.contextInstance("ColumnarCachedBatchSerializer#serialize"))
- .serialize(Array(ColumnarBatches.getNativeHandle(batch)))
+ .create(
+ Runtimes.contextInstance(
+ BackendsApiManager.getBackendName,
+ "ColumnarCachedBatchSerializer#serialize"))
+ .serialize(
+ Array(ColumnarBatches.getNativeHandle(BackendsApiManager.getBackendName, batch)))
CachedColumnarBatch(
results.getNumRows.toInt,
results.getSerialized.length,
@@ -201,7 +205,9 @@ class ColumnarCachedBatchSerializer extends CachedBatchSerializer with SQLConfHe
val timezoneId = SQLConf.get.sessionLocalTimeZone
input.mapPartitions {
it =>
- val runtime = Runtimes.contextInstance("ColumnarCachedBatchSerializer#read")
+ val runtime = Runtimes.contextInstance(
+ BackendsApiManager.getBackendName,
+ "ColumnarCachedBatchSerializer#read")
val jniWrapper = ColumnarBatchSerializerJniWrapper
.create(runtime)
val schema = SparkArrowUtil.toArrowSchema(localSchema, timezoneId)
@@ -224,7 +230,10 @@ class ColumnarCachedBatchSerializer extends CachedBatchSerializer with SQLConfHe
val batch = ColumnarBatches.create(batchHandle)
if (shouldSelectAttributes) {
try {
- ColumnarBatches.select(batch, requestedColumnIndices.toArray)
+ ColumnarBatches.select(
+ BackendsApiManager.getBackendName,
+ batch,
+ requestedColumnIndices.toArray)
} finally {
batch.close()
}
diff --git a/backends-velox/src/main/scala/org/apache/spark/sql/execution/datasources/velox/VeloxFormatWriterInjects.scala b/backends-velox/src/main/scala/org/apache/spark/sql/execution/datasources/velox/VeloxFormatWriterInjects.scala
index cd5f442bc765..c9651557fa70 100644
--- a/backends-velox/src/main/scala/org/apache/spark/sql/execution/datasources/velox/VeloxFormatWriterInjects.scala
+++ b/backends-velox/src/main/scala/org/apache/spark/sql/execution/datasources/velox/VeloxFormatWriterInjects.scala
@@ -16,6 +16,7 @@
*/
package org.apache.spark.sql.execution.datasources.velox
+import org.apache.gluten.backendsapi.BackendsApiManager
import org.apache.gluten.columnarbatch.ColumnarBatches
import org.apache.gluten.datasource.{VeloxDataSourceJniWrapper, VeloxDataSourceUtil}
import org.apache.gluten.exception.GlutenException
@@ -57,7 +58,7 @@ trait VeloxFormatWriterInjects extends GlutenFormatWriterInjectsBase {
SparkArrowUtil.toArrowSchema(dataSchema, SQLConf.get.sessionLocalTimeZone)
val cSchema = ArrowSchema.allocateNew(ArrowBufferAllocators.contextInstance())
var dsHandle = -1L
- val runtime = Runtimes.contextInstance("VeloxWriter")
+ val runtime = Runtimes.contextInstance(BackendsApiManager.getBackendName, "VeloxWriter")
val datasourceJniWrapper = VeloxDataSourceJniWrapper.create(runtime)
val allocator = ArrowBufferAllocators.contextInstance()
try {
@@ -77,7 +78,7 @@ trait VeloxFormatWriterInjects extends GlutenFormatWriterInjectsBase {
ColumnarBatches.retain(batch)
val batchHandle = {
ColumnarBatches.checkOffloaded(batch)
- ColumnarBatches.getNativeHandle(batch)
+ ColumnarBatches.getNativeHandle(BackendsApiManager.getBackendName, batch)
}
datasourceJniWrapper.writeBatch(dsHandle, batchHandle)
batch.close()
@@ -108,8 +109,9 @@ class VeloxRowSplitter extends GlutenRowSplitter {
partitionColIndice: Array[Int],
hasBucket: Boolean,
reserve_partition_columns: Boolean = false): BlockStripes = {
- val handler = ColumnarBatches.getNativeHandle(row.batch)
- val runtime = Runtimes.contextInstance("VeloxPartitionWriter")
+ val handler = ColumnarBatches.getNativeHandle(BackendsApiManager.getBackendName, row.batch)
+ val runtime =
+ Runtimes.contextInstance(BackendsApiManager.getBackendName, "VeloxPartitionWriter")
val datasourceJniWrapper = VeloxDataSourceJniWrapper.create(runtime)
val originalColumns: Array[Int] = Array.range(0, row.batch.numCols())
val dataColIndice = originalColumns.filterNot(partitionColIndice.contains(_))
diff --git a/backends-velox/src/main/scala/org/apache/spark/sql/execution/utils/ExecUtil.scala b/backends-velox/src/main/scala/org/apache/spark/sql/execution/utils/ExecUtil.scala
index 32bac0204519..e3a84d4f074d 100644
--- a/backends-velox/src/main/scala/org/apache/spark/sql/execution/utils/ExecUtil.scala
+++ b/backends-velox/src/main/scala/org/apache/spark/sql/execution/utils/ExecUtil.scala
@@ -16,6 +16,7 @@
*/
package org.apache.spark.sql.execution.utils
+import org.apache.gluten.backendsapi.BackendsApiManager
import org.apache.gluten.columnarbatch.{ColumnarBatches, VeloxColumnarBatches}
import org.apache.gluten.iterator.Iterators
import org.apache.gluten.memory.arrow.alloc.ArrowBufferAllocators
@@ -41,10 +42,11 @@ import org.apache.spark.util.MutablePair
object ExecUtil {
def convertColumnarToRow(batch: ColumnarBatch): Iterator[InternalRow] = {
- val runtime = Runtimes.contextInstance("ExecUtil#ColumnarToRow")
+ val runtime =
+ Runtimes.contextInstance(BackendsApiManager.getBackendName, "ExecUtil#ColumnarToRow")
val jniWrapper = NativeColumnarToRowJniWrapper.create(runtime)
var info: NativeColumnarToRowInfo = null
- val batchHandle = ColumnarBatches.getNativeHandle(batch)
+ val batchHandle = ColumnarBatches.getNativeHandle(BackendsApiManager.getBackendName, batch)
val c2rHandle = jniWrapper.nativeColumnarToRowInit()
info = jniWrapper.nativeColumnarToRowConvert(c2rHandle, batchHandle, 0)
diff --git a/backends-velox/src/main/scala/org/apache/spark/sql/hive/VeloxHiveUDFTransformer.scala b/backends-velox/src/main/scala/org/apache/spark/sql/hive/VeloxHiveUDFTransformer.scala
index d895faa31702..b3524e20f019 100644
--- a/backends-velox/src/main/scala/org/apache/spark/sql/hive/VeloxHiveUDFTransformer.scala
+++ b/backends-velox/src/main/scala/org/apache/spark/sql/hive/VeloxHiveUDFTransformer.scala
@@ -37,11 +37,11 @@ object VeloxHiveUDFTransformer {
}
if (UDFResolver.UDFNames.contains(udfClassName)) {
- UDFResolver
+ val udfExpression = UDFResolver
.getUdfExpression(udfClassName, udfName)(expr.children)
- .getTransformer(
- ExpressionConverter.replaceWithExpressionTransformer(expr.children, attributeSeq)
- )
+ udfExpression.getTransformer(
+ ExpressionConverter.replaceWithExpressionTransformer(udfExpression.children, attributeSeq)
+ )
} else {
HiveUDFTransformer.genTransformerFromUDFMappings(udfName, expr, attributeSeq)
}
diff --git a/backends-velox/src/test-iceberg/scala/org/apache/gluten/execution/VeloxIcebergSuite.scala b/backends-velox/src/test-iceberg/scala/org/apache/gluten/execution/VeloxIcebergSuite.scala
new file mode 100644
index 000000000000..1573042884b2
--- /dev/null
+++ b/backends-velox/src/test-iceberg/scala/org/apache/gluten/execution/VeloxIcebergSuite.scala
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+package org.apache.gluten.execution
+
+class VeloxIcebergSuite extends IcebergSuite
diff --git a/gluten-iceberg/src/test/scala/org/apache/gluten/execution/VeloxTPCHIcebergSuite.scala b/backends-velox/src/test-iceberg/scala/org/apache/gluten/execution/VeloxTPCHIcebergSuite.scala
similarity index 96%
rename from gluten-iceberg/src/test/scala/org/apache/gluten/execution/VeloxTPCHIcebergSuite.scala
rename to backends-velox/src/test-iceberg/scala/org/apache/gluten/execution/VeloxTPCHIcebergSuite.scala
index 857cbcb410f8..5bc26e81b7ca 100644
--- a/gluten-iceberg/src/test/scala/org/apache/gluten/execution/VeloxTPCHIcebergSuite.scala
+++ b/backends-velox/src/test-iceberg/scala/org/apache/gluten/execution/VeloxTPCHIcebergSuite.scala
@@ -24,13 +24,13 @@ import org.apache.iceberg.spark.SparkWriteOptions
import java.io.File
class VeloxTPCHIcebergSuite extends VeloxTPCHSuite {
-
- protected val tpchBasePath: String = new File(
- "../backends-velox/src/test/resources").getAbsolutePath
+ protected val tpchBasePath: String =
+ getClass.getResource("/").getPath + "../../../src/test/resources"
override protected val resourcePath: String =
new File(tpchBasePath, "tpch-data-parquet").getCanonicalPath
+ // FIXME: Unused.
override protected val queriesResults: String =
new File(tpchBasePath, "queries-output").getCanonicalPath
diff --git a/backends-velox/src/test/java/org/apache/gluten/columnarbatch/ColumnarBatchTest.java b/backends-velox/src/test/java/org/apache/gluten/columnarbatch/ColumnarBatchTest.java
index 54803aa1930f..f02caf8f2d1b 100644
--- a/backends-velox/src/test/java/org/apache/gluten/columnarbatch/ColumnarBatchTest.java
+++ b/backends-velox/src/test/java/org/apache/gluten/columnarbatch/ColumnarBatchTest.java
@@ -16,6 +16,7 @@
*/
package org.apache.gluten.columnarbatch;
+import org.apache.gluten.backendsapi.BackendsApiManager;
import org.apache.gluten.execution.RowToVeloxColumnarExec;
import org.apache.gluten.memory.arrow.alloc.ArrowBufferAllocators;
import org.apache.gluten.test.VeloxBackendTestBase;
@@ -114,9 +115,12 @@ public void testCreateByHandle() {
final ColumnarBatch offloaded =
ColumnarBatches.offload(ArrowBufferAllocators.contextInstance(), batch);
Assert.assertEquals(1, ColumnarBatches.getRefCnt(offloaded));
- final long handle = ColumnarBatches.getNativeHandle(offloaded);
+ final long handle =
+ ColumnarBatches.getNativeHandle(BackendsApiManager.getBackendName(), offloaded);
final ColumnarBatch created = ColumnarBatches.create(handle);
- Assert.assertEquals(handle, ColumnarBatches.getNativeHandle(created));
+ Assert.assertEquals(
+ handle,
+ ColumnarBatches.getNativeHandle(BackendsApiManager.getBackendName(), created));
Assert.assertEquals(1, ColumnarBatches.getRefCnt(offloaded));
Assert.assertEquals(1, ColumnarBatches.getRefCnt(created));
ColumnarBatches.retain(created);
diff --git a/backends-velox/src/test/java/org/apache/gluten/test/MockVeloxBackend.java b/backends-velox/src/test/java/org/apache/gluten/test/MockVeloxBackend.java
index b5ce30e39b25..254e8e2830f4 100644
--- a/backends-velox/src/test/java/org/apache/gluten/test/MockVeloxBackend.java
+++ b/backends-velox/src/test/java/org/apache/gluten/test/MockVeloxBackend.java
@@ -71,6 +71,7 @@ public Object ask(Object message) throws Exception {
private static SparkConf newSparkConf() {
final SparkConf conf = new SparkConf();
conf.set(GlutenConfig.SPARK_OFFHEAP_SIZE_KEY(), "1g");
+ conf.set(GlutenConfig.COLUMNAR_VELOX_CONNECTOR_IO_THREADS(), "0");
return conf;
}
}
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 efe7f3908506..617489f1aa35 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
@@ -16,7 +16,7 @@ AdaptiveSparkPlan (30)
+- ^ FlushableHashAggregateExecTransformer (4)
+- ^ ProjectExecTransformer (3)
+- ^ FilterExecTransformer (2)
- +- ^ Scan parquet (1)
+ +- ^ ScanTransformer parquet (1)
+- == Initial Plan ==
Sort (29)
+- Exchange (28)
@@ -28,7 +28,7 @@ AdaptiveSparkPlan (30)
+- Scan parquet (22)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [7]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_tax#X, l_returnflag#X, l_linestatus#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
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 5b913a06db13..7197e83986a8 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
@@ -18,24 +18,24 @@ AdaptiveSparkPlan (68)
: :- ^ ProjectExecTransformer (12)
: : +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (11)
: : :- ^ FilterExecTransformer (2)
- : : : +- ^ Scan parquet (1)
+ : : : +- ^ ScanTransformer parquet (1)
: : +- ^ InputIteratorTransformer (10)
: : +- BroadcastQueryStage (8)
: : +- ColumnarBroadcastExchange (7)
: : +- ^ ProjectExecTransformer (5)
: : +- ^ FilterExecTransformer (4)
- : : +- ^ Scan parquet (3)
+ : : +- ^ ScanTransformer parquet (3)
: +- ^ InputIteratorTransformer (20)
: +- BroadcastQueryStage (18)
: +- ColumnarBroadcastExchange (17)
: +- ^ ProjectExecTransformer (15)
: +- ^ FilterExecTransformer (14)
- : +- ^ Scan parquet (13)
+ : +- ^ ScanTransformer parquet (13)
+- ^ InputIteratorTransformer (29)
+- BroadcastQueryStage (27)
+- ColumnarBroadcastExchange (26)
+- ^ FilterExecTransformer (24)
- +- ^ Scan parquet (23)
+ +- ^ ScanTransformer parquet (23)
+- == Initial Plan ==
TakeOrderedAndProject (67)
+- HashAggregate (66)
@@ -62,7 +62,7 @@ AdaptiveSparkPlan (68)
+- Scan parquet (59)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -73,7 +73,7 @@ ReadSchema: struct
-(2) Scan parquet
+(2) ScanTransformer parquet
Output [3]: [o_orderkey#X, o_custkey#X, o_comment#X]
Batched: true
Location: InMemoryFileIndex [*]
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 946c150d8654..4687bd81bcc8 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
@@ -13,12 +13,12 @@ AdaptiveSparkPlan (35)
+- ^ BroadcastHashJoinExecTransformer Inner BuildRight (11)
:- ^ ProjectExecTransformer (3)
: +- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
+ : +- ^ ScanTransformer parquet (1)
+- ^ InputIteratorTransformer (10)
+- BroadcastQueryStage (8)
+- ColumnarBroadcastExchange (7)
+- ^ FilterExecTransformer (5)
- +- ^ Scan parquet (4)
+ +- ^ ScanTransformer parquet (4)
+- == Initial Plan ==
HashAggregate (34)
+- Exchange (33)
@@ -33,7 +33,7 @@ AdaptiveSparkPlan (35)
+- Scan parquet (27)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -48,7 +48,7 @@ Arguments: (((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1995-09-01)) AND (l_s
Output [3]: [l_partkey#X, l_extendedprice#X, l_discount#X]
Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
-(4) Scan parquet
+(4) ScanTransformer parquet
Output [2]: [p_partkey#X, p_type#X]
Batched: true
Location: InMemoryFileIndex [*]
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 39b1c80d441b..cd765112a70f 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
@@ -13,7 +13,7 @@ AdaptiveSparkPlan (46)
: +- BroadcastQueryStage (5)
: +- ColumnarBroadcastExchange (4)
: +- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
+ : +- ^ ScanTransformer parquet (1)
+- ^ FilterExecTransformer (20)
+- ^ RegularHashAggregateExecTransformer (19)
+- ^ InputIteratorTransformer (18)
@@ -24,7 +24,7 @@ AdaptiveSparkPlan (46)
+- ^ FlushableHashAggregateExecTransformer (11)
+- ^ ProjectExecTransformer (10)
+- ^ FilterExecTransformer (9)
- +- ^ Scan parquet (8)
+ +- ^ ScanTransformer parquet (8)
+- == Initial Plan ==
Sort (45)
+- Exchange (44)
@@ -42,7 +42,7 @@ AdaptiveSparkPlan (46)
+- Scan parquet (35)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -71,7 +71,7 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X]
(7) InputIteratorTransformer
Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X]
-(8) Scan parquet
+(8) ScanTransformer parquet
Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
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 f7b9f730e38b..10972d6c2f1a 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
@@ -24,12 +24,12 @@ AdaptiveSparkPlan (59)
+- ^ ProjectExecTransformer (11)
+- ^ BroadcastHashJoinExecTransformer Inner BuildRight (10)
:- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
+ : +- ^ ScanTransformer parquet (1)
+- ^ InputIteratorTransformer (9)
+- BroadcastQueryStage (7)
+- ColumnarBroadcastExchange (6)
+- ^ FilterExecTransformer (4)
- +- ^ Scan parquet (3)
+ +- ^ ScanTransformer parquet (3)
+- == Initial Plan ==
Sort (58)
+- Exchange (57)
@@ -53,7 +53,7 @@ AdaptiveSparkPlan (59)
+- Scan parquet (46)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [ps_partkey#X, ps_suppkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -64,7 +64,7 @@ ReadSchema: struct
Input [2]: [ps_partkey#X, ps_suppkey#X]
Arguments: isnotnull(ps_partkey#X)
-(3) Scan parquet
+(3) ScanTransformer parquet
Output [4]: [p_partkey#X, p_brand#X, p_type#X, p_size#X]
Batched: true
Location: InMemoryFileIndex [*]
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 2cd26d020ae7..88d48cf5dfa3 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
@@ -18,10 +18,10 @@ AdaptiveSparkPlan (88)
: : +- BroadcastQueryStage (5)
: : +- ColumnarBroadcastExchange (4)
: : +- ^ FilterExecTransformer (2)
- : : +- ^ Scan parquet (1)
+ : : +- ^ ScanTransformer parquet (1)
: +- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (27)
: :- ^ FilterExecTransformer (9)
- : : +- ^ Scan parquet (8)
+ : : +- ^ ScanTransformer parquet (8)
: +- ^ InputIteratorTransformer (26)
: +- BroadcastQueryStage (24)
: +- ColumnarBroadcastExchange (23)
@@ -34,13 +34,13 @@ AdaptiveSparkPlan (88)
: +- VeloxResizeBatches (14)
: +- ^ ProjectExecTransformer (12)
: +- ^ FlushableHashAggregateExecTransformer (11)
- : +- ^ Scan parquet (10)
+ : +- ^ ScanTransformer parquet (10)
+- ^ InputIteratorTransformer (41)
+- BroadcastQueryStage (39)
+- ColumnarBroadcastExchange (38)
+- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (36)
:- ^ FilterExecTransformer (31)
- : +- ^ Scan parquet (30)
+ : +- ^ ScanTransformer parquet (30)
+- ^ InputIteratorTransformer (35)
+- BroadcastQueryStage (33)
+- ReusedExchange (32)
@@ -79,7 +79,7 @@ AdaptiveSparkPlan (88)
+- Scan parquet (73)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [c_custkey#X, c_name#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -108,7 +108,7 @@ Input [2]: [c_custkey#X, c_name#X]
(7) InputIteratorTransformer
Input [2]: [c_custkey#X, c_name#X]
-(8) Scan parquet
+(8) ScanTransformer parquet
Output [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -119,7 +119,7 @@ ReadSchema: struct
Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
Arguments: ((isnotnull(ps_availqty#X) AND isnotnull(ps_partkey#X)) AND isnotnull(ps_suppkey#X))
-(13) Scan parquet
+(13) ScanTransformer parquet
Output [2]: [p_partkey#X, p_name#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -207,7 +207,7 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
(26) InputIteratorTransformer
Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
-(27) Scan parquet
+(27) ScanTransformer parquet
Output [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -324,7 +324,7 @@ Join condition: None
Output [3]: [s_name#X, s_address#X, s_nationkey#X]
Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X]
-(55) Scan parquet
+(55) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 2ef6280501aa..a1adaf31bf28 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
@@ -20,34 +20,34 @@ AdaptiveSparkPlan (93)
: : : +- BroadcastQueryStage (5)
: : : +- ColumnarBroadcastExchange (4)
: : : +- ^ FilterExecTransformer (2)
- : : : +- ^ Scan parquet (1)
+ : : : +- ^ ScanTransformer parquet (1)
: : +- ^ BroadcastHashJoinExecTransformer LeftAnti BuildRight (26)
: : :- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (17)
: : : :- ^ ProjectExecTransformer (10)
: : : : +- ^ FilterExecTransformer (9)
- : : : : +- ^ Scan parquet (8)
+ : : : : +- ^ ScanTransformer parquet (8)
: : : +- ^ InputIteratorTransformer (16)
: : : +- BroadcastQueryStage (14)
: : : +- ColumnarBroadcastExchange (13)
- : : : +- ^ Scan parquet (11)
+ : : : +- ^ ScanTransformer parquet (11)
: : +- ^ InputIteratorTransformer (25)
: : +- BroadcastQueryStage (23)
: : +- ColumnarBroadcastExchange (22)
: : +- ^ ProjectExecTransformer (20)
: : +- ^ FilterExecTransformer (19)
- : : +- ^ Scan parquet (18)
+ : : +- ^ ScanTransformer parquet (18)
: +- ^ InputIteratorTransformer (36)
: +- BroadcastQueryStage (34)
: +- ColumnarBroadcastExchange (33)
: +- ^ ProjectExecTransformer (31)
: +- ^ FilterExecTransformer (30)
- : +- ^ Scan parquet (29)
+ : +- ^ ScanTransformer parquet (29)
+- ^ InputIteratorTransformer (46)
+- BroadcastQueryStage (44)
+- ColumnarBroadcastExchange (43)
+- ^ ProjectExecTransformer (41)
+- ^ FilterExecTransformer (40)
- +- ^ Scan parquet (39)
+ +- ^ ScanTransformer parquet (39)
+- == Initial Plan ==
TakeOrderedAndProject (92)
+- HashAggregate (91)
@@ -83,7 +83,7 @@ AdaptiveSparkPlan (93)
+- Scan parquet (83)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [3]: [s_suppkey#X, s_name#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -112,7 +112,7 @@ Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X]
(7) InputIteratorTransformer
Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X]
-(8) Scan parquet
+(8) ScanTransformer parquet
Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -127,7 +127,7 @@ Arguments: ((((isnotnull(l_receiptdate#X) AND isnotnull(l_commitdate#X)) AND (l_
Output [2]: [l_orderkey#X, l_suppkey#X]
Input [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X]
-(11) Scan parquet
+(11) ScanTransformer parquet
Output [2]: [l_orderkey#X, l_suppkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -156,7 +156,7 @@ Left keys [1]: [l_orderkey#X]
Right keys [1]: [l_orderkey#X]
Join condition: NOT (l_suppkey#X = l_suppkey#X)
-(18) Scan parquet
+(18) ScanTransformer parquet
Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -203,7 +203,7 @@ Join condition: None
Output [3]: [s_name#X, s_nationkey#X, l_orderkey#X]
Input [5]: [s_suppkey#X, s_name#X, s_nationkey#X, l_orderkey#X, l_suppkey#X]
-(29) Scan parquet
+(29) ScanTransformer parquet
Output [2]: [o_orderkey#X, o_orderstatus#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -245,7 +245,7 @@ Join condition: None
Output [2]: [s_name#X, s_nationkey#X]
Input [4]: [s_name#X, s_nationkey#X, l_orderkey#X, o_orderkey#X]
-(39) Scan parquet
+(39) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 271e9e810ec3..afb719f19903 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
@@ -17,11 +17,11 @@ AdaptiveSparkPlan (40)
+- ^ ProjectExecTransformer (10)
+- ^ BroadcastHashJoinExecTransformer LeftAnti BuildRight (9)
:- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
+ : +- ^ ScanTransformer parquet (1)
+- ^ InputIteratorTransformer (8)
+- BroadcastQueryStage (6)
+- ColumnarBroadcastExchange (5)
- +- ^ Scan parquet (3)
+ +- ^ ScanTransformer parquet (3)
+- == Initial Plan ==
Sort (39)
+- Exchange (38)
@@ -36,7 +36,7 @@ AdaptiveSparkPlan (40)
+- Scan parquet (31)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [3]: [c_custkey#X, c_phone#X, c_acctbal#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -47,7 +47,7 @@ ReadSchema: struct
Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X]
Arguments: ((isnotnull(c_acctbal#X) AND substring(c_phone#X, 1, 2) IN (13,31,23,29,30,18,17)) AND (cast(c_acctbal#X as decimal(16,6)) > Subquery subquery#X, [id=#X]))
-(3) Scan parquet
+(3) ScanTransformer parquet
Output [1]: [o_custkey#X]
Batched: true
Location: InMemoryFileIndex [*]
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 f0fca318bf20..9335e4fbe92c 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
@@ -20,15 +20,15 @@ AdaptiveSparkPlan (54)
: : +- ColumnarBroadcastExchange (5)
: : +- ^ ProjectExecTransformer (3)
: : +- ^ FilterExecTransformer (2)
- : : +- ^ Scan parquet (1)
+ : : +- ^ ScanTransformer parquet (1)
: +- ^ FilterExecTransformer (10)
- : +- ^ Scan parquet (9)
+ : +- ^ ScanTransformer parquet (9)
+- ^ InputIteratorTransformer (20)
+- BroadcastQueryStage (18)
+- ColumnarBroadcastExchange (17)
+- ^ ProjectExecTransformer (15)
+- ^ FilterExecTransformer (14)
- +- ^ Scan parquet (13)
+ +- ^ ScanTransformer parquet (13)
+- == Initial Plan ==
TakeOrderedAndProject (53)
+- HashAggregate (52)
@@ -50,7 +50,7 @@ AdaptiveSparkPlan (54)
+- Scan parquet (44)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [c_custkey#X, c_mktsegment#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -83,7 +83,7 @@ Input [1]: [c_custkey#X]
(8) InputIteratorTransformer
Input [1]: [c_custkey#X]
-(9) Scan parquet
+(9) ScanTransformer parquet
Output [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -103,7 +103,7 @@ Join condition: None
Output [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X]
Input [5]: [c_custkey#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X]
-(13) Scan parquet
+(13) ScanTransformer parquet
Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
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 b8cdf1656c05..9ff9aba0a986 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
@@ -18,13 +18,13 @@ AdaptiveSparkPlan (46)
+- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (12)
:- ^ ProjectExecTransformer (3)
: +- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
+ : +- ^ ScanTransformer parquet (1)
+- ^ InputIteratorTransformer (11)
+- BroadcastQueryStage (9)
+- ColumnarBroadcastExchange (8)
+- ^ ProjectExecTransformer (6)
+- ^ FilterExecTransformer (5)
- +- ^ Scan parquet (4)
+ +- ^ ScanTransformer parquet (4)
+- == Initial Plan ==
Sort (45)
+- Exchange (44)
@@ -42,7 +42,7 @@ AdaptiveSparkPlan (46)
+- Scan parquet (35)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -57,7 +57,7 @@ Arguments: ((isnotnull(o_orderdate#X) AND (o_orderdate#X >= 1993-07-01)) AND (o_
Output [2]: [o_orderkey#X, o_orderpriority#X]
Input [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X]
-(4) Scan parquet
+(4) ScanTransformer parquet
Output [3]: [l_orderkey#X, l_commitdate#X, l_receiptdate#X]
Batched: true
Location: InMemoryFileIndex [*]
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 0ef8d9d4fc1a..df5fa26f0098 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
@@ -28,31 +28,31 @@ AdaptiveSparkPlan (102)
: : : : : +- BroadcastQueryStage (5)
: : : : : +- ColumnarBroadcastExchange (4)
: : : : : +- ^ FilterExecTransformer (2)
- : : : : : +- ^ Scan parquet (1)
+ : : : : : +- ^ ScanTransformer parquet (1)
: : : : +- ^ ProjectExecTransformer (10)
: : : : +- ^ FilterExecTransformer (9)
- : : : : +- ^ Scan parquet (8)
+ : : : : +- ^ ScanTransformer parquet (8)
: : : +- ^ InputIteratorTransformer (19)
: : : +- BroadcastQueryStage (17)
: : : +- ColumnarBroadcastExchange (16)
: : : +- ^ FilterExecTransformer (14)
- : : : +- ^ Scan parquet (13)
+ : : : +- ^ ScanTransformer parquet (13)
: : +- ^ InputIteratorTransformer (28)
: : +- BroadcastQueryStage (26)
: : +- ColumnarBroadcastExchange (25)
: : +- ^ FilterExecTransformer (23)
- : : +- ^ Scan parquet (22)
+ : : +- ^ ScanTransformer parquet (22)
: +- ^ InputIteratorTransformer (37)
: +- BroadcastQueryStage (35)
: +- ColumnarBroadcastExchange (34)
: +- ^ FilterExecTransformer (32)
- : +- ^ Scan parquet (31)
+ : +- ^ ScanTransformer parquet (31)
+- ^ InputIteratorTransformer (47)
+- BroadcastQueryStage (45)
+- ColumnarBroadcastExchange (44)
+- ^ ProjectExecTransformer (42)
+- ^ FilterExecTransformer (41)
- +- ^ Scan parquet (40)
+ +- ^ ScanTransformer parquet (40)
+- == Initial Plan ==
Sort (101)
+- Exchange (100)
@@ -90,7 +90,7 @@ AdaptiveSparkPlan (102)
+- Scan parquet (91)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [c_custkey#X, c_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -119,7 +119,7 @@ Input [2]: [c_custkey#X, c_nationkey#X]
(7) InputIteratorTransformer
Input [2]: [c_custkey#X, c_nationkey#X]
-(8) Scan parquet
+(8) ScanTransformer parquet
Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -143,7 +143,7 @@ Join condition: None
Output [2]: [c_nationkey#X, o_orderkey#X]
Input [4]: [c_custkey#X, c_nationkey#X, o_orderkey#X, o_custkey#X]
-(13) Scan parquet
+(13) ScanTransformer parquet
Output [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -181,7 +181,7 @@ Join condition: None
Output [4]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
Input [6]: [c_nationkey#X, o_orderkey#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
-(22) Scan parquet
+(22) ScanTransformer parquet
Output [2]: [s_suppkey#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -219,7 +219,7 @@ Join condition: None
Output [3]: [l_extendedprice#X, l_discount#X, s_nationkey#X]
Input [6]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, s_suppkey#X, s_nationkey#X]
-(31) Scan parquet
+(31) ScanTransformer parquet
Output [3]: [n_nationkey#X, n_name#X, n_regionkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -257,7 +257,7 @@ Join condition: None
Output [4]: [l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X]
Input [6]: [l_extendedprice#X, l_discount#X, s_nationkey#X, n_nationkey#X, n_name#X, n_regionkey#X]
-(40) Scan parquet
+(40) ScanTransformer parquet
Output [2]: [r_regionkey#X, r_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 a1830678d0de..73d9d734e78a 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
@@ -10,7 +10,7 @@ AdaptiveSparkPlan (20)
+- ^ FlushableHashAggregateExecTransformer (4)
+- ^ ProjectExecTransformer (3)
+- ^ FilterExecTransformer (2)
- +- ^ Scan parquet (1)
+ +- ^ ScanTransformer parquet (1)
+- == Initial Plan ==
HashAggregate (19)
+- Exchange (18)
@@ -20,7 +20,7 @@ AdaptiveSparkPlan (20)
+- Scan parquet (14)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [4]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
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 cac53895b981..88f83c467e85 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
@@ -28,24 +28,24 @@ AdaptiveSparkPlan (95)
: : : : : +- BroadcastQueryStage (5)
: : : : : +- ColumnarBroadcastExchange (4)
: : : : : +- ^ FilterExecTransformer (2)
- : : : : : +- ^ Scan parquet (1)
+ : : : : : +- ^ ScanTransformer parquet (1)
: : : : +- ^ FilterExecTransformer (9)
- : : : : +- ^ Scan parquet (8)
+ : : : : +- ^ ScanTransformer parquet (8)
: : : +- ^ InputIteratorTransformer (18)
: : : +- BroadcastQueryStage (16)
: : : +- ColumnarBroadcastExchange (15)
: : : +- ^ FilterExecTransformer (13)
- : : : +- ^ Scan parquet (12)
+ : : : +- ^ ScanTransformer parquet (12)
: : +- ^ InputIteratorTransformer (27)
: : +- BroadcastQueryStage (25)
: : +- ColumnarBroadcastExchange (24)
: : +- ^ FilterExecTransformer (22)
- : : +- ^ Scan parquet (21)
+ : : +- ^ ScanTransformer parquet (21)
: +- ^ InputIteratorTransformer (36)
: +- BroadcastQueryStage (34)
: +- ColumnarBroadcastExchange (33)
: +- ^ FilterExecTransformer (31)
- : +- ^ Scan parquet (30)
+ : +- ^ ScanTransformer parquet (30)
+- ^ InputIteratorTransformer (42)
+- BroadcastQueryStage (40)
+- ReusedExchange (39)
@@ -84,7 +84,7 @@ AdaptiveSparkPlan (95)
+- Scan parquet (85)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [s_suppkey#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -113,7 +113,7 @@ Input [2]: [s_suppkey#X, s_nationkey#X]
(7) InputIteratorTransformer
Input [2]: [s_suppkey#X, s_nationkey#X]
-(8) Scan parquet
+(8) ScanTransformer parquet
Output [5]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -133,7 +133,7 @@ Join condition: None
Output [5]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Input [7]: [s_suppkey#X, s_nationkey#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
-(12) Scan parquet
+(12) ScanTransformer parquet
Output [2]: [o_orderkey#X, o_custkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -171,7 +171,7 @@ Join condition: None
Output [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X]
Input [7]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_orderkey#X, o_custkey#X]
-(21) Scan parquet
+(21) ScanTransformer parquet
Output [2]: [c_custkey#X, c_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -209,7 +209,7 @@ Join condition: None
Output [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X]
Input [7]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X, c_custkey#X, c_nationkey#X]
-(30) Scan parquet
+(30) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 33544f0796a5..c389ab06a45f 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
@@ -34,40 +34,40 @@ AdaptiveSparkPlan (131)
: : : : : : : +- ColumnarBroadcastExchange (5)
: : : : : : : +- ^ ProjectExecTransformer (3)
: : : : : : : +- ^ FilterExecTransformer (2)
- : : : : : : : +- ^ Scan parquet (1)
+ : : : : : : : +- ^ ScanTransformer parquet (1)
: : : : : : +- ^ FilterExecTransformer (10)
- : : : : : : +- ^ Scan parquet (9)
+ : : : : : : +- ^ ScanTransformer parquet (9)
: : : : : +- ^ InputIteratorTransformer (19)
: : : : : +- BroadcastQueryStage (17)
: : : : : +- ColumnarBroadcastExchange (16)
: : : : : +- ^ FilterExecTransformer (14)
- : : : : : +- ^ Scan parquet (13)
+ : : : : : +- ^ ScanTransformer parquet (13)
: : : : +- ^ InputIteratorTransformer (28)
: : : : +- BroadcastQueryStage (26)
: : : : +- ColumnarBroadcastExchange (25)
: : : : +- ^ FilterExecTransformer (23)
- : : : : +- ^ Scan parquet (22)
+ : : : : +- ^ ScanTransformer parquet (22)
: : : +- ^ InputIteratorTransformer (37)
: : : +- BroadcastQueryStage (35)
: : : +- ColumnarBroadcastExchange (34)
: : : +- ^ FilterExecTransformer (32)
- : : : +- ^ Scan parquet (31)
+ : : : +- ^ ScanTransformer parquet (31)
: : +- ^ InputIteratorTransformer (46)
: : +- BroadcastQueryStage (44)
: : +- ColumnarBroadcastExchange (43)
: : +- ^ FilterExecTransformer (41)
- : : +- ^ Scan parquet (40)
+ : : +- ^ ScanTransformer parquet (40)
: +- ^ InputIteratorTransformer (55)
: +- BroadcastQueryStage (53)
: +- ColumnarBroadcastExchange (52)
: +- ^ FilterExecTransformer (50)
- : +- ^ Scan parquet (49)
+ : +- ^ ScanTransformer parquet (49)
+- ^ InputIteratorTransformer (65)
+- BroadcastQueryStage (63)
+- ColumnarBroadcastExchange (62)
+- ^ ProjectExecTransformer (60)
+- ^ FilterExecTransformer (59)
- +- ^ Scan parquet (58)
+ +- ^ ScanTransformer parquet (58)
+- == Initial Plan ==
Sort (130)
+- Exchange (129)
@@ -115,7 +115,7 @@ AdaptiveSparkPlan (131)
+- Scan parquet (120)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [p_partkey#X, p_type#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -148,7 +148,7 @@ Input [1]: [p_partkey#X]
(8) InputIteratorTransformer
Input [1]: [p_partkey#X]
-(9) Scan parquet
+(9) ScanTransformer parquet
Output [5]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -168,7 +168,7 @@ Join condition: None
Output [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
Input [6]: [p_partkey#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
-(13) Scan parquet
+(13) ScanTransformer parquet
Output [2]: [s_suppkey#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -206,7 +206,7 @@ Join condition: None
Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X]
Input [6]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, s_suppkey#X, s_nationkey#X]
-(22) Scan parquet
+(22) ScanTransformer parquet
Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -244,7 +244,7 @@ Join condition: None
Output [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X]
Input [7]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderkey#X, o_custkey#X, o_orderdate#X]
-(31) Scan parquet
+(31) ScanTransformer parquet
Output [2]: [c_custkey#X, c_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -282,7 +282,7 @@ Join condition: None
Output [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X]
Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X, c_custkey#X, c_nationkey#X]
-(40) Scan parquet
+(40) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_regionkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -320,7 +320,7 @@ Join condition: None
Output [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X]
Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X, n_nationkey#X, n_regionkey#X]
-(49) Scan parquet
+(49) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -358,7 +358,7 @@ Join condition: None
Output [5]: [l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X]
Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X, n_nationkey#X, n_name#X]
-(58) Scan parquet
+(58) ScanTransformer parquet
Output [2]: [r_regionkey#X, r_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 7112bb021b0c..830259ba82bd 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
@@ -29,29 +29,29 @@ AdaptiveSparkPlan (100)
: : : : : +- ColumnarBroadcastExchange (5)
: : : : : +- ^ ProjectExecTransformer (3)
: : : : : +- ^ FilterExecTransformer (2)
- : : : : : +- ^ Scan parquet (1)
+ : : : : : +- ^ ScanTransformer parquet (1)
: : : : +- ^ FilterExecTransformer (10)
- : : : : +- ^ Scan parquet (9)
+ : : : : +- ^ ScanTransformer parquet (9)
: : : +- ^ InputIteratorTransformer (19)
: : : +- BroadcastQueryStage (17)
: : : +- ColumnarBroadcastExchange (16)
: : : +- ^ FilterExecTransformer (14)
- : : : +- ^ Scan parquet (13)
+ : : : +- ^ ScanTransformer parquet (13)
: : +- ^ InputIteratorTransformer (28)
: : +- BroadcastQueryStage (26)
: : +- ColumnarBroadcastExchange (25)
: : +- ^ FilterExecTransformer (23)
- : : +- ^ Scan parquet (22)
+ : : +- ^ ScanTransformer parquet (22)
: +- ^ InputIteratorTransformer (37)
: +- BroadcastQueryStage (35)
: +- ColumnarBroadcastExchange (34)
: +- ^ FilterExecTransformer (32)
- : +- ^ Scan parquet (31)
+ : +- ^ ScanTransformer parquet (31)
+- ^ InputIteratorTransformer (46)
+- BroadcastQueryStage (44)
+- ColumnarBroadcastExchange (43)
+- ^ FilterExecTransformer (41)
- +- ^ Scan parquet (40)
+ +- ^ ScanTransformer parquet (40)
+- == Initial Plan ==
Sort (99)
+- Exchange (98)
@@ -88,7 +88,7 @@ AdaptiveSparkPlan (100)
+- Scan parquet (90)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [p_partkey#X, p_name#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -121,7 +121,7 @@ Input [1]: [p_partkey#X]
(8) InputIteratorTransformer
Input [1]: [p_partkey#X]
-(9) Scan parquet
+(9) ScanTransformer parquet
Output [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -141,7 +141,7 @@ Join condition: None
Output [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X]
Input [7]: [p_partkey#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X]
-(13) Scan parquet
+(13) ScanTransformer parquet
Output [2]: [s_suppkey#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -179,7 +179,7 @@ Join condition: None
Output [7]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X]
Input [8]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_suppkey#X, s_nationkey#X]
-(22) Scan parquet
+(22) ScanTransformer parquet
Output [3]: [ps_partkey#X, ps_suppkey#X, ps_supplycost#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -217,7 +217,7 @@ Join condition: None
Output [6]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X]
Input [10]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_partkey#X, ps_suppkey#X, ps_supplycost#X]
-(31) Scan parquet
+(31) ScanTransformer parquet
Output [2]: [o_orderkey#X, o_orderdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -255,7 +255,7 @@ Join condition: None
Output [6]: [l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X]
Input [8]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderkey#X, o_orderdate#X]
-(40) Scan parquet
+(40) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 0851fa2447ea..ca3b50ab05c5 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
@@ -16,7 +16,7 @@ AdaptiveSparkPlan (30)
+- ^ FlushableHashAggregateExecTransformer (4)
+- ^ ProjectExecTransformer (3)
+- ^ FilterExecTransformer (2)
- +- ^ Scan parquet (1)
+ +- ^ ScanTransformer parquet (1)
+- == Initial Plan ==
Sort (29)
+- Exchange (28)
@@ -28,7 +28,7 @@ AdaptiveSparkPlan (30)
+- Scan parquet (22)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [7]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_tax#X, l_returnflag#X, l_linestatus#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
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 42cf8e1b7f58..c15913c284f5 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
@@ -18,24 +18,24 @@ AdaptiveSparkPlan (68)
: :- ^ ProjectExecTransformer (12)
: : +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (11)
: : :- ^ FilterExecTransformer (2)
- : : : +- ^ Scan parquet (1)
+ : : : +- ^ ScanTransformer parquet (1)
: : +- ^ InputIteratorTransformer (10)
: : +- BroadcastQueryStage (8), Statistics(X)
: : +- ColumnarBroadcastExchange (7)
: : +- ^ ProjectExecTransformer (5)
: : +- ^ FilterExecTransformer (4)
- : : +- ^ Scan parquet (3)
+ : : +- ^ ScanTransformer parquet (3)
: +- ^ InputIteratorTransformer (20)
: +- BroadcastQueryStage (18), Statistics(X)
: +- ColumnarBroadcastExchange (17)
: +- ^ ProjectExecTransformer (15)
: +- ^ FilterExecTransformer (14)
- : +- ^ Scan parquet (13)
+ : +- ^ ScanTransformer parquet (13)
+- ^ InputIteratorTransformer (29)
+- BroadcastQueryStage (27), Statistics(X)
+- ColumnarBroadcastExchange (26)
+- ^ FilterExecTransformer (24)
- +- ^ Scan parquet (23)
+ +- ^ ScanTransformer parquet (23)
+- == Initial Plan ==
TakeOrderedAndProject (67)
+- HashAggregate (66)
@@ -62,7 +62,7 @@ AdaptiveSparkPlan (68)
+- Scan parquet (59)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -73,7 +73,7 @@ ReadSchema: struct
-(2) Scan parquet
+(2) ScanTransformer parquet
Output [3]: [o_orderkey#X, o_custkey#X, o_comment#X]
Batched: true
Location: InMemoryFileIndex [*]
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 8956ddaa2184..b06cd55156e1 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
@@ -13,12 +13,12 @@ AdaptiveSparkPlan (35)
+- ^ BroadcastHashJoinExecTransformer Inner BuildRight (11)
:- ^ ProjectExecTransformer (3)
: +- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
+ : +- ^ ScanTransformer parquet (1)
+- ^ InputIteratorTransformer (10)
+- BroadcastQueryStage (8), Statistics(X)
+- ColumnarBroadcastExchange (7)
+- ^ FilterExecTransformer (5)
- +- ^ Scan parquet (4)
+ +- ^ ScanTransformer parquet (4)
+- == Initial Plan ==
HashAggregate (34)
+- Exchange (33)
@@ -33,7 +33,7 @@ AdaptiveSparkPlan (35)
+- Scan parquet (27)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -48,7 +48,7 @@ Arguments: (((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1995-09-01)) AND (l_s
Output [3]: [l_partkey#X, l_extendedprice#X, l_discount#X]
Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
-(4) Scan parquet
+(4) ScanTransformer parquet
Output [2]: [p_partkey#X, p_type#X]
Batched: true
Location: InMemoryFileIndex [*]
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 d7603e49ffb6..52a87f43535d 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
@@ -12,7 +12,7 @@ AdaptiveSparkPlan (43)
: +- BroadcastQueryStage (5), Statistics(X)
: +- ColumnarBroadcastExchange (4)
: +- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
+ : +- ^ ScanTransformer parquet (1)
+- ^ FilterExecTransformer (20)
+- ^ RegularHashAggregateExecTransformer (19)
+- ^ InputIteratorTransformer (18)
@@ -23,7 +23,7 @@ AdaptiveSparkPlan (43)
+- ^ FlushableHashAggregateExecTransformer (11)
+- ^ ProjectExecTransformer (10)
+- ^ FilterExecTransformer (9)
- +- ^ Scan parquet (8)
+ +- ^ ScanTransformer parquet (8)
+- == Initial Plan ==
Sort (42)
+- Exchange (41)
@@ -41,7 +41,7 @@ AdaptiveSparkPlan (43)
+- Scan parquet (32)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -70,7 +70,7 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X]
(7) InputIteratorTransformer
Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X]
-(8) Scan parquet
+(8) ScanTransformer parquet
Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -237,34 +237,33 @@ Arguments: isFinalPlan=true
===== Subqueries =====
Subquery:1 Hosting operator id = 20 Hosting Expression = Subquery subquery#X, [id=#X]
-AdaptiveSparkPlan (69)
+AdaptiveSparkPlan (68)
+- == Final Plan ==
- VeloxColumnarToRow (60)
- +- ^ RegularHashAggregateExecTransformer (58)
- +- ^ RegularHashAggregateExecTransformer (57)
- +- ^ ProjectExecTransformer (56)
- +- ^ RegularHashAggregateExecTransformer (55)
- +- ^ InputIteratorTransformer (54)
- +- ShuffleQueryStage (52), Statistics(X)
- +- ColumnarExchange (51)
- +- VeloxResizeBatches (50)
- +- ^ ProjectExecTransformer (48)
- +- ^ FlushableHashAggregateExecTransformer (47)
- +- ^ ProjectExecTransformer (46)
- +- ^ FilterExecTransformer (45)
- +- ^ Scan parquet (44)
+ VeloxColumnarToRow (59)
+ +- ^ RegularHashAggregateExecTransformer (57)
+ +- ^ ProjectExecTransformer (56)
+ +- ^ RegularHashAggregateExecTransformer (55)
+ +- ^ InputIteratorTransformer (54)
+ +- ShuffleQueryStage (52), Statistics(X)
+ +- ColumnarExchange (51)
+ +- VeloxResizeBatches (50)
+ +- ^ ProjectExecTransformer (48)
+ +- ^ FlushableHashAggregateExecTransformer (47)
+ +- ^ ProjectExecTransformer (46)
+ +- ^ FilterExecTransformer (45)
+ +- ^ ScanTransformer parquet (44)
+- == Initial Plan ==
- HashAggregate (68)
- +- HashAggregate (67)
- +- HashAggregate (66)
- +- Exchange (65)
- +- HashAggregate (64)
- +- Project (63)
- +- Filter (62)
- +- Scan parquet (61)
+ HashAggregate (67)
+ +- HashAggregate (66)
+ +- HashAggregate (65)
+ +- Exchange (64)
+ +- HashAggregate (63)
+ +- Project (62)
+ +- Filter (61)
+ +- Scan parquet (60)
-(44) Scan parquet
+(44) ScanTransformer parquet
Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -326,71 +325,64 @@ Input [2]: [l_suppkey#X, sum(CheckOverflow((promote_precision(cast(l_extendedpri
(57) RegularHashAggregateExecTransformer
Input [1]: [total_revenue#X]
Keys: []
-Functions [1]: [partial_max(total_revenue#X)]
-Aggregate Attributes [1]: [max#X]
-Results [1]: [max#X]
-
-(58) RegularHashAggregateExecTransformer
-Input [1]: [max#X]
-Keys: []
Functions [1]: [max(total_revenue#X)]
Aggregate Attributes [1]: [max(total_revenue#X)#X]
Results [1]: [max(total_revenue#X)#X AS max(total_revenue)#X]
-(59) WholeStageCodegenTransformer (X)
+(58) WholeStageCodegenTransformer (X)
Input [1]: [max(total_revenue)#X]
Arguments: false
-(60) VeloxColumnarToRow
+(59) VeloxColumnarToRow
Input [1]: [max(total_revenue)#X]
-(61) Scan parquet
+(60) Scan parquet
Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1996-01-01), LessThan(l_shipdate,1996-04-01)]
ReadSchema: struct
-(62) Filter
+(61) Filter
Input [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Condition : ((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1996-01-01)) AND (l_shipdate#X < 1996-04-01))
-(63) Project
+(62) Project
Output [3]: [l_suppkey#X, l_extendedprice#X, l_discount#X]
Input [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
-(64) HashAggregate
+(63) HashAggregate
Input [3]: [l_suppkey#X, l_extendedprice#X, l_discount#X]
Keys [1]: [l_suppkey#X]
Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))]
Aggregate Attributes [2]: [sum#X, isEmpty#X]
Results [3]: [l_suppkey#X, sum#X, isEmpty#X]
-(65) Exchange
+(64) Exchange
Input [3]: [l_suppkey#X, sum#X, isEmpty#X]
Arguments: hashpartitioning(l_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(66) HashAggregate
+(65) HashAggregate
Input [3]: [l_suppkey#X, sum#X, isEmpty#X]
Keys [1]: [l_suppkey#X]
Functions [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))]
Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X]
Results [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X AS total_revenue#X]
-(67) HashAggregate
+(66) HashAggregate
Input [1]: [total_revenue#X]
Keys: []
Functions [1]: [partial_max(total_revenue#X)]
Aggregate Attributes [1]: [max#X]
Results [1]: [max#X]
-(68) HashAggregate
+(67) HashAggregate
Input [1]: [max#X]
Keys: []
Functions [1]: [max(total_revenue#X)]
Aggregate Attributes [1]: [max(total_revenue#X)#X]
Results [1]: [max(total_revenue#X)#X AS max(total_revenue)#X]
-(69) AdaptiveSparkPlan
+(68) AdaptiveSparkPlan
Output [1]: [max(total_revenue)#X]
Arguments: isFinalPlan=true
\ No newline at end of file
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 394bbd2469c2..8cc2b417eb3f 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
@@ -24,12 +24,12 @@ AdaptiveSparkPlan (59)
+- ^ ProjectExecTransformer (11)
+- ^ BroadcastHashJoinExecTransformer Inner BuildRight (10)
:- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
+ : +- ^ ScanTransformer parquet (1)
+- ^ InputIteratorTransformer (9)
+- BroadcastQueryStage (7), Statistics(X)
+- ColumnarBroadcastExchange (6)
+- ^ FilterExecTransformer (4)
- +- ^ Scan parquet (3)
+ +- ^ ScanTransformer parquet (3)
+- == Initial Plan ==
Sort (58)
+- Exchange (57)
@@ -53,7 +53,7 @@ AdaptiveSparkPlan (59)
+- Scan parquet (46)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [ps_partkey#X, ps_suppkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -64,7 +64,7 @@ ReadSchema: struct
Input [2]: [ps_partkey#X, ps_suppkey#X]
Arguments: isnotnull(ps_partkey#X)
-(3) Scan parquet
+(3) ScanTransformer parquet
Output [4]: [p_partkey#X, p_brand#X, p_type#X, p_size#X]
Batched: true
Location: InMemoryFileIndex [*]
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 6c279f2d124d..8fd71a81a264 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
@@ -18,10 +18,10 @@ AdaptiveSparkPlan (88)
: : +- BroadcastQueryStage (5), Statistics(X)
: : +- ColumnarBroadcastExchange (4)
: : +- ^ FilterExecTransformer (2)
- : : +- ^ Scan parquet (1)
+ : : +- ^ ScanTransformer parquet (1)
: +- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (27)
: :- ^ FilterExecTransformer (9)
- : : +- ^ Scan parquet (8)
+ : : +- ^ ScanTransformer parquet (8)
: +- ^ InputIteratorTransformer (26)
: +- BroadcastQueryStage (24), Statistics(X)
: +- ColumnarBroadcastExchange (23)
@@ -34,13 +34,13 @@ AdaptiveSparkPlan (88)
: +- VeloxResizeBatches (14)
: +- ^ ProjectExecTransformer (12)
: +- ^ FlushableHashAggregateExecTransformer (11)
- : +- ^ Scan parquet (10)
+ : +- ^ ScanTransformer parquet (10)
+- ^ InputIteratorTransformer (41)
+- BroadcastQueryStage (39), Statistics(X)
+- ColumnarBroadcastExchange (38)
+- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (36)
:- ^ FilterExecTransformer (31)
- : +- ^ Scan parquet (30)
+ : +- ^ ScanTransformer parquet (30)
+- ^ InputIteratorTransformer (35)
+- BroadcastQueryStage (33), Statistics(X)
+- ReusedExchange (32)
@@ -79,7 +79,7 @@ AdaptiveSparkPlan (88)
+- Scan parquet (73)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [c_custkey#X, c_name#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -108,7 +108,7 @@ Input [2]: [c_custkey#X, c_name#X]
(7) InputIteratorTransformer
Input [2]: [c_custkey#X, c_name#X]
-(8) Scan parquet
+(8) ScanTransformer parquet
Output [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -119,7 +119,7 @@ ReadSchema: struct
Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
Arguments: ((isnotnull(ps_availqty#X) AND isnotnull(ps_partkey#X)) AND isnotnull(ps_suppkey#X))
-(13) Scan parquet
+(13) ScanTransformer parquet
Output [2]: [p_partkey#X, p_name#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -206,7 +206,7 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
(26) InputIteratorTransformer
Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
-(27) Scan parquet
+(27) ScanTransformer parquet
Output [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -323,7 +323,7 @@ Join condition: None
Output [3]: [s_name#X, s_address#X, s_nationkey#X]
Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X]
-(55) Scan parquet
+(55) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 eae4758613ee..505dc59dc4cb 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
@@ -19,34 +19,34 @@ AdaptiveSparkPlan (92)
: : : +- BroadcastQueryStage (5), Statistics(X)
: : : +- ColumnarBroadcastExchange (4)
: : : +- ^ FilterExecTransformer (2)
- : : : +- ^ Scan parquet (1)
+ : : : +- ^ ScanTransformer parquet (1)
: : +- ^ BroadcastHashJoinExecTransformer LeftAnti BuildRight (26)
: : :- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (17)
: : : :- ^ ProjectExecTransformer (10)
: : : : +- ^ FilterExecTransformer (9)
- : : : : +- ^ Scan parquet (8)
+ : : : : +- ^ ScanTransformer parquet (8)
: : : +- ^ InputIteratorTransformer (16)
: : : +- BroadcastQueryStage (14), Statistics(X)
: : : +- ColumnarBroadcastExchange (13)
- : : : +- ^ Scan parquet (11)
+ : : : +- ^ ScanTransformer parquet (11)
: : +- ^ InputIteratorTransformer (25)
: : +- BroadcastQueryStage (23), Statistics(X)
: : +- ColumnarBroadcastExchange (22)
: : +- ^ ProjectExecTransformer (20)
: : +- ^ FilterExecTransformer (19)
- : : +- ^ Scan parquet (18)
+ : : +- ^ ScanTransformer parquet (18)
: +- ^ InputIteratorTransformer (36)
: +- BroadcastQueryStage (34), Statistics(X)
: +- ColumnarBroadcastExchange (33)
: +- ^ ProjectExecTransformer (31)
: +- ^ FilterExecTransformer (30)
- : +- ^ Scan parquet (29)
+ : +- ^ ScanTransformer parquet (29)
+- ^ InputIteratorTransformer (46)
+- BroadcastQueryStage (44), Statistics(X)
+- ColumnarBroadcastExchange (43)
+- ^ ProjectExecTransformer (41)
+- ^ FilterExecTransformer (40)
- +- ^ Scan parquet (39)
+ +- ^ ScanTransformer parquet (39)
+- == Initial Plan ==
TakeOrderedAndProject (91)
+- HashAggregate (90)
@@ -82,7 +82,7 @@ AdaptiveSparkPlan (92)
+- Scan parquet (82)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [3]: [s_suppkey#X, s_name#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -111,7 +111,7 @@ Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X]
(7) InputIteratorTransformer
Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X]
-(8) Scan parquet
+(8) ScanTransformer parquet
Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -126,7 +126,7 @@ Arguments: ((((isnotnull(l_receiptdate#X) AND isnotnull(l_commitdate#X)) AND (l_
Output [2]: [l_orderkey#X, l_suppkey#X]
Input [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X]
-(11) Scan parquet
+(11) ScanTransformer parquet
Output [2]: [l_orderkey#X, l_suppkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -155,7 +155,7 @@ Left keys [1]: [l_orderkey#X]
Right keys [1]: [l_orderkey#X]
Join condition: NOT (l_suppkey#X = l_suppkey#X)
-(18) Scan parquet
+(18) ScanTransformer parquet
Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -202,7 +202,7 @@ Join condition: None
Output [3]: [s_name#X, s_nationkey#X, l_orderkey#X]
Input [5]: [s_suppkey#X, s_name#X, s_nationkey#X, l_orderkey#X, l_suppkey#X]
-(29) Scan parquet
+(29) ScanTransformer parquet
Output [2]: [o_orderkey#X, o_orderstatus#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -244,7 +244,7 @@ Join condition: None
Output [2]: [s_name#X, s_nationkey#X]
Input [4]: [s_name#X, s_nationkey#X, l_orderkey#X, o_orderkey#X]
-(39) Scan parquet
+(39) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 a3d3f0a08eeb..9113355acbdd 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
@@ -17,11 +17,11 @@ AdaptiveSparkPlan (40)
+- ^ ProjectExecTransformer (10)
+- ^ BroadcastHashJoinExecTransformer LeftAnti BuildRight (9)
:- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
+ : +- ^ ScanTransformer parquet (1)
+- ^ InputIteratorTransformer (8)
+- BroadcastQueryStage (6), Statistics(X)
+- ColumnarBroadcastExchange (5)
- +- ^ Scan parquet (3)
+ +- ^ ScanTransformer parquet (3)
+- == Initial Plan ==
Sort (39)
+- Exchange (38)
@@ -36,7 +36,7 @@ AdaptiveSparkPlan (40)
+- Scan parquet (31)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [3]: [c_custkey#X, c_phone#X, c_acctbal#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -47,7 +47,7 @@ ReadSchema: struct
Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X]
Arguments: ((isnotnull(c_acctbal#X) AND substring(c_phone#X, 1, 2) IN (13,31,23,29,30,18,17)) AND (cast(c_acctbal#X as decimal(16,6)) > Subquery subquery#X, [id=#X]))
-(3) Scan parquet
+(3) ScanTransformer parquet
Output [1]: [o_custkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -227,7 +227,7 @@ AdaptiveSparkPlan (60)
+- ^ FlushableHashAggregateExecTransformer (44)
+- ^ ProjectExecTransformer (43)
+- ^ FilterExecTransformer (42)
- +- ^ Scan parquet (41)
+ +- ^ ScanTransformer parquet (41)
+- == Initial Plan ==
HashAggregate (59)
+- Exchange (58)
@@ -237,7 +237,7 @@ AdaptiveSparkPlan (60)
+- Scan parquet (54)
-(41) Scan parquet
+(41) ScanTransformer parquet
Output [2]: [c_phone#X, c_acctbal#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -344,7 +344,7 @@ AdaptiveSparkPlan (60)
+- ^ FlushableHashAggregateExecTransformer (44)
+- ^ ProjectExecTransformer (43)
+- ^ FilterExecTransformer (42)
- +- ^ Scan parquet (41)
+ +- ^ ScanTransformer parquet (41)
+- == Initial Plan ==
HashAggregate (59)
+- Exchange (58)
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 e7fa21b6f1f1..befea454305b 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
@@ -20,15 +20,15 @@ AdaptiveSparkPlan (54)
: : +- ColumnarBroadcastExchange (5)
: : +- ^ ProjectExecTransformer (3)
: : +- ^ FilterExecTransformer (2)
- : : +- ^ Scan parquet (1)
+ : : +- ^ ScanTransformer parquet (1)
: +- ^ FilterExecTransformer (10)
- : +- ^ Scan parquet (9)
+ : +- ^ ScanTransformer parquet (9)
+- ^ InputIteratorTransformer (20)
+- BroadcastQueryStage (18), Statistics(X)
+- ColumnarBroadcastExchange (17)
+- ^ ProjectExecTransformer (15)
+- ^ FilterExecTransformer (14)
- +- ^ Scan parquet (13)
+ +- ^ ScanTransformer parquet (13)
+- == Initial Plan ==
TakeOrderedAndProject (53)
+- HashAggregate (52)
@@ -50,7 +50,7 @@ AdaptiveSparkPlan (54)
+- Scan parquet (44)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [c_custkey#X, c_mktsegment#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -83,7 +83,7 @@ Input [1]: [c_custkey#X]
(8) InputIteratorTransformer
Input [1]: [c_custkey#X]
-(9) Scan parquet
+(9) ScanTransformer parquet
Output [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -103,7 +103,7 @@ Join condition: None
Output [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X]
Input [5]: [c_custkey#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X]
-(13) Scan parquet
+(13) ScanTransformer parquet
Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
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 a1eaaca4d448..52c99692cf4d 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
@@ -18,13 +18,13 @@ AdaptiveSparkPlan (46)
+- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (12)
:- ^ ProjectExecTransformer (3)
: +- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
+ : +- ^ ScanTransformer parquet (1)
+- ^ InputIteratorTransformer (11)
+- BroadcastQueryStage (9), Statistics(X)
+- ColumnarBroadcastExchange (8)
+- ^ ProjectExecTransformer (6)
+- ^ FilterExecTransformer (5)
- +- ^ Scan parquet (4)
+ +- ^ ScanTransformer parquet (4)
+- == Initial Plan ==
Sort (45)
+- Exchange (44)
@@ -42,7 +42,7 @@ AdaptiveSparkPlan (46)
+- Scan parquet (35)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -57,7 +57,7 @@ Arguments: ((isnotnull(o_orderdate#X) AND (o_orderdate#X >= 1993-07-01)) AND (o_
Output [2]: [o_orderkey#X, o_orderpriority#X]
Input [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X]
-(4) Scan parquet
+(4) ScanTransformer parquet
Output [3]: [l_orderkey#X, l_commitdate#X, l_receiptdate#X]
Batched: true
Location: InMemoryFileIndex [*]
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 638a442362e3..a076ea86feb5 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
@@ -28,31 +28,31 @@ AdaptiveSparkPlan (102)
: : : : : +- BroadcastQueryStage (5), Statistics(X)
: : : : : +- ColumnarBroadcastExchange (4)
: : : : : +- ^ FilterExecTransformer (2)
- : : : : : +- ^ Scan parquet (1)
+ : : : : : +- ^ ScanTransformer parquet (1)
: : : : +- ^ ProjectExecTransformer (10)
: : : : +- ^ FilterExecTransformer (9)
- : : : : +- ^ Scan parquet (8)
+ : : : : +- ^ ScanTransformer parquet (8)
: : : +- ^ InputIteratorTransformer (19)
: : : +- BroadcastQueryStage (17), Statistics(X)
: : : +- ColumnarBroadcastExchange (16)
: : : +- ^ FilterExecTransformer (14)
- : : : +- ^ Scan parquet (13)
+ : : : +- ^ ScanTransformer parquet (13)
: : +- ^ InputIteratorTransformer (28)
: : +- BroadcastQueryStage (26), Statistics(X)
: : +- ColumnarBroadcastExchange (25)
: : +- ^ FilterExecTransformer (23)
- : : +- ^ Scan parquet (22)
+ : : +- ^ ScanTransformer parquet (22)
: +- ^ InputIteratorTransformer (37)
: +- BroadcastQueryStage (35), Statistics(X)
: +- ColumnarBroadcastExchange (34)
: +- ^ FilterExecTransformer (32)
- : +- ^ Scan parquet (31)
+ : +- ^ ScanTransformer parquet (31)
+- ^ InputIteratorTransformer (47)
+- BroadcastQueryStage (45), Statistics(X)
+- ColumnarBroadcastExchange (44)
+- ^ ProjectExecTransformer (42)
+- ^ FilterExecTransformer (41)
- +- ^ Scan parquet (40)
+ +- ^ ScanTransformer parquet (40)
+- == Initial Plan ==
Sort (101)
+- Exchange (100)
@@ -90,7 +90,7 @@ AdaptiveSparkPlan (102)
+- Scan parquet (91)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [c_custkey#X, c_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -119,7 +119,7 @@ Input [2]: [c_custkey#X, c_nationkey#X]
(7) InputIteratorTransformer
Input [2]: [c_custkey#X, c_nationkey#X]
-(8) Scan parquet
+(8) ScanTransformer parquet
Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -143,7 +143,7 @@ Join condition: None
Output [2]: [c_nationkey#X, o_orderkey#X]
Input [4]: [c_custkey#X, c_nationkey#X, o_orderkey#X, o_custkey#X]
-(13) Scan parquet
+(13) ScanTransformer parquet
Output [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -181,7 +181,7 @@ Join condition: None
Output [4]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
Input [6]: [c_nationkey#X, o_orderkey#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
-(22) Scan parquet
+(22) ScanTransformer parquet
Output [2]: [s_suppkey#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -219,7 +219,7 @@ Join condition: None
Output [3]: [l_extendedprice#X, l_discount#X, s_nationkey#X]
Input [6]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, s_suppkey#X, s_nationkey#X]
-(31) Scan parquet
+(31) ScanTransformer parquet
Output [3]: [n_nationkey#X, n_name#X, n_regionkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -257,7 +257,7 @@ Join condition: None
Output [4]: [l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X]
Input [6]: [l_extendedprice#X, l_discount#X, s_nationkey#X, n_nationkey#X, n_name#X, n_regionkey#X]
-(40) Scan parquet
+(40) ScanTransformer parquet
Output [2]: [r_regionkey#X, r_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 5df63675caee..d5569132a41f 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
@@ -10,7 +10,7 @@ AdaptiveSparkPlan (20)
+- ^ FlushableHashAggregateExecTransformer (4)
+- ^ ProjectExecTransformer (3)
+- ^ FilterExecTransformer (2)
- +- ^ Scan parquet (1)
+ +- ^ ScanTransformer parquet (1)
+- == Initial Plan ==
HashAggregate (19)
+- Exchange (18)
@@ -20,7 +20,7 @@ AdaptiveSparkPlan (20)
+- Scan parquet (14)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [4]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
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 86db01e58b53..5fdae3d7334b 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
@@ -28,24 +28,24 @@ AdaptiveSparkPlan (95)
: : : : : +- BroadcastQueryStage (5), Statistics(X)
: : : : : +- ColumnarBroadcastExchange (4)
: : : : : +- ^ FilterExecTransformer (2)
- : : : : : +- ^ Scan parquet (1)
+ : : : : : +- ^ ScanTransformer parquet (1)
: : : : +- ^ FilterExecTransformer (9)
- : : : : +- ^ Scan parquet (8)
+ : : : : +- ^ ScanTransformer parquet (8)
: : : +- ^ InputIteratorTransformer (18)
: : : +- BroadcastQueryStage (16), Statistics(X)
: : : +- ColumnarBroadcastExchange (15)
: : : +- ^ FilterExecTransformer (13)
- : : : +- ^ Scan parquet (12)
+ : : : +- ^ ScanTransformer parquet (12)
: : +- ^ InputIteratorTransformer (27)
: : +- BroadcastQueryStage (25), Statistics(X)
: : +- ColumnarBroadcastExchange (24)
: : +- ^ FilterExecTransformer (22)
- : : +- ^ Scan parquet (21)
+ : : +- ^ ScanTransformer parquet (21)
: +- ^ InputIteratorTransformer (36)
: +- BroadcastQueryStage (34), Statistics(X)
: +- ColumnarBroadcastExchange (33)
: +- ^ FilterExecTransformer (31)
- : +- ^ Scan parquet (30)
+ : +- ^ ScanTransformer parquet (30)
+- ^ InputIteratorTransformer (42)
+- BroadcastQueryStage (40), Statistics(X)
+- ReusedExchange (39)
@@ -84,7 +84,7 @@ AdaptiveSparkPlan (95)
+- Scan parquet (85)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [s_suppkey#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -113,7 +113,7 @@ Input [2]: [s_suppkey#X, s_nationkey#X]
(7) InputIteratorTransformer
Input [2]: [s_suppkey#X, s_nationkey#X]
-(8) Scan parquet
+(8) ScanTransformer parquet
Output [5]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -133,7 +133,7 @@ Join condition: None
Output [5]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Input [7]: [s_suppkey#X, s_nationkey#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
-(12) Scan parquet
+(12) ScanTransformer parquet
Output [2]: [o_orderkey#X, o_custkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -171,7 +171,7 @@ Join condition: None
Output [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X]
Input [7]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_orderkey#X, o_custkey#X]
-(21) Scan parquet
+(21) ScanTransformer parquet
Output [2]: [c_custkey#X, c_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -209,7 +209,7 @@ Join condition: None
Output [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X]
Input [7]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X, c_custkey#X, c_nationkey#X]
-(30) Scan parquet
+(30) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 ccc7941882b7..0b00093c0316 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
@@ -34,40 +34,40 @@ AdaptiveSparkPlan (131)
: : : : : : : +- ColumnarBroadcastExchange (5)
: : : : : : : +- ^ ProjectExecTransformer (3)
: : : : : : : +- ^ FilterExecTransformer (2)
- : : : : : : : +- ^ Scan parquet (1)
+ : : : : : : : +- ^ ScanTransformer parquet (1)
: : : : : : +- ^ FilterExecTransformer (10)
- : : : : : : +- ^ Scan parquet (9)
+ : : : : : : +- ^ ScanTransformer parquet (9)
: : : : : +- ^ InputIteratorTransformer (19)
: : : : : +- BroadcastQueryStage (17), Statistics(X)
: : : : : +- ColumnarBroadcastExchange (16)
: : : : : +- ^ FilterExecTransformer (14)
- : : : : : +- ^ Scan parquet (13)
+ : : : : : +- ^ ScanTransformer parquet (13)
: : : : +- ^ InputIteratorTransformer (28)
: : : : +- BroadcastQueryStage (26), Statistics(X)
: : : : +- ColumnarBroadcastExchange (25)
: : : : +- ^ FilterExecTransformer (23)
- : : : : +- ^ Scan parquet (22)
+ : : : : +- ^ ScanTransformer parquet (22)
: : : +- ^ InputIteratorTransformer (37)
: : : +- BroadcastQueryStage (35), Statistics(X)
: : : +- ColumnarBroadcastExchange (34)
: : : +- ^ FilterExecTransformer (32)
- : : : +- ^ Scan parquet (31)
+ : : : +- ^ ScanTransformer parquet (31)
: : +- ^ InputIteratorTransformer (46)
: : +- BroadcastQueryStage (44), Statistics(X)
: : +- ColumnarBroadcastExchange (43)
: : +- ^ FilterExecTransformer (41)
- : : +- ^ Scan parquet (40)
+ : : +- ^ ScanTransformer parquet (40)
: +- ^ InputIteratorTransformer (55)
: +- BroadcastQueryStage (53), Statistics(X)
: +- ColumnarBroadcastExchange (52)
: +- ^ FilterExecTransformer (50)
- : +- ^ Scan parquet (49)
+ : +- ^ ScanTransformer parquet (49)
+- ^ InputIteratorTransformer (65)
+- BroadcastQueryStage (63), Statistics(X)
+- ColumnarBroadcastExchange (62)
+- ^ ProjectExecTransformer (60)
+- ^ FilterExecTransformer (59)
- +- ^ Scan parquet (58)
+ +- ^ ScanTransformer parquet (58)
+- == Initial Plan ==
Sort (130)
+- Exchange (129)
@@ -115,7 +115,7 @@ AdaptiveSparkPlan (131)
+- Scan parquet (120)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [p_partkey#X, p_type#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -148,7 +148,7 @@ Input [1]: [p_partkey#X]
(8) InputIteratorTransformer
Input [1]: [p_partkey#X]
-(9) Scan parquet
+(9) ScanTransformer parquet
Output [5]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -168,7 +168,7 @@ Join condition: None
Output [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
Input [6]: [p_partkey#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
-(13) Scan parquet
+(13) ScanTransformer parquet
Output [2]: [s_suppkey#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -206,7 +206,7 @@ Join condition: None
Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X]
Input [6]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, s_suppkey#X, s_nationkey#X]
-(22) Scan parquet
+(22) ScanTransformer parquet
Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -244,7 +244,7 @@ Join condition: None
Output [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X]
Input [7]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderkey#X, o_custkey#X, o_orderdate#X]
-(31) Scan parquet
+(31) ScanTransformer parquet
Output [2]: [c_custkey#X, c_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -282,7 +282,7 @@ Join condition: None
Output [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X]
Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X, c_custkey#X, c_nationkey#X]
-(40) Scan parquet
+(40) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_regionkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -320,7 +320,7 @@ Join condition: None
Output [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X]
Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X, n_nationkey#X, n_regionkey#X]
-(49) Scan parquet
+(49) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -358,7 +358,7 @@ Join condition: None
Output [5]: [l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X]
Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X, n_nationkey#X, n_name#X]
-(58) Scan parquet
+(58) ScanTransformer parquet
Output [2]: [r_regionkey#X, r_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 212413568214..8c1db077485f 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
@@ -29,29 +29,29 @@ AdaptiveSparkPlan (100)
: : : : : +- ColumnarBroadcastExchange (5)
: : : : : +- ^ ProjectExecTransformer (3)
: : : : : +- ^ FilterExecTransformer (2)
- : : : : : +- ^ Scan parquet (1)
+ : : : : : +- ^ ScanTransformer parquet (1)
: : : : +- ^ FilterExecTransformer (10)
- : : : : +- ^ Scan parquet (9)
+ : : : : +- ^ ScanTransformer parquet (9)
: : : +- ^ InputIteratorTransformer (19)
: : : +- BroadcastQueryStage (17), Statistics(X)
: : : +- ColumnarBroadcastExchange (16)
: : : +- ^ FilterExecTransformer (14)
- : : : +- ^ Scan parquet (13)
+ : : : +- ^ ScanTransformer parquet (13)
: : +- ^ InputIteratorTransformer (28)
: : +- BroadcastQueryStage (26), Statistics(X)
: : +- ColumnarBroadcastExchange (25)
: : +- ^ FilterExecTransformer (23)
- : : +- ^ Scan parquet (22)
+ : : +- ^ ScanTransformer parquet (22)
: +- ^ InputIteratorTransformer (37)
: +- BroadcastQueryStage (35), Statistics(X)
: +- ColumnarBroadcastExchange (34)
: +- ^ FilterExecTransformer (32)
- : +- ^ Scan parquet (31)
+ : +- ^ ScanTransformer parquet (31)
+- ^ InputIteratorTransformer (46)
+- BroadcastQueryStage (44), Statistics(X)
+- ColumnarBroadcastExchange (43)
+- ^ FilterExecTransformer (41)
- +- ^ Scan parquet (40)
+ +- ^ ScanTransformer parquet (40)
+- == Initial Plan ==
Sort (99)
+- Exchange (98)
@@ -88,7 +88,7 @@ AdaptiveSparkPlan (100)
+- Scan parquet (90)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [p_partkey#X, p_name#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -121,7 +121,7 @@ Input [1]: [p_partkey#X]
(8) InputIteratorTransformer
Input [1]: [p_partkey#X]
-(9) Scan parquet
+(9) ScanTransformer parquet
Output [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -141,7 +141,7 @@ Join condition: None
Output [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X]
Input [7]: [p_partkey#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X]
-(13) Scan parquet
+(13) ScanTransformer parquet
Output [2]: [s_suppkey#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -179,7 +179,7 @@ Join condition: None
Output [7]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X]
Input [8]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_suppkey#X, s_nationkey#X]
-(22) Scan parquet
+(22) ScanTransformer parquet
Output [3]: [ps_partkey#X, ps_suppkey#X, ps_supplycost#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -217,7 +217,7 @@ Join condition: None
Output [6]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X]
Input [10]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_partkey#X, ps_suppkey#X, ps_supplycost#X]
-(31) Scan parquet
+(31) ScanTransformer parquet
Output [2]: [o_orderkey#X, o_orderdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -255,7 +255,7 @@ Join condition: None
Output [6]: [l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X]
Input [8]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderkey#X, o_orderdate#X]
-(40) Scan parquet
+(40) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 6dbee1ad7ffb..4d61f904dbe9 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
@@ -16,7 +16,7 @@ AdaptiveSparkPlan (30)
+- ^ FlushableHashAggregateExecTransformer (4)
+- ^ ProjectExecTransformer (3)
+- ^ FilterExecTransformer (2)
- +- ^ Scan parquet (1)
+ +- ^ ScanTransformer parquet (1)
+- == Initial Plan ==
Sort (29)
+- Exchange (28)
@@ -28,7 +28,7 @@ AdaptiveSparkPlan (30)
+- Scan parquet (22)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [7]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_tax#X, l_returnflag#X, l_linestatus#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
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 47dc04d100db..a4a335e80831 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
@@ -18,24 +18,24 @@ AdaptiveSparkPlan (68)
: :- ^ ProjectExecTransformer (12)
: : +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (11)
: : :- ^ FilterExecTransformer (2)
- : : : +- ^ Scan parquet (1)
+ : : : +- ^ ScanTransformer parquet (1)
: : +- ^ InputIteratorTransformer (10)
: : +- BroadcastQueryStage (8), Statistics(X)
: : +- ColumnarBroadcastExchange (7)
: : +- ^ ProjectExecTransformer (5)
: : +- ^ FilterExecTransformer (4)
- : : +- ^ Scan parquet (3)
+ : : +- ^ ScanTransformer parquet (3)
: +- ^ InputIteratorTransformer (20)
: +- BroadcastQueryStage (18), Statistics(X)
: +- ColumnarBroadcastExchange (17)
: +- ^ ProjectExecTransformer (15)
: +- ^ FilterExecTransformer (14)
- : +- ^ Scan parquet (13)
+ : +- ^ ScanTransformer parquet (13)
+- ^ InputIteratorTransformer (29)
+- BroadcastQueryStage (27), Statistics(X)
+- ColumnarBroadcastExchange (26)
+- ^ FilterExecTransformer (24)
- +- ^ Scan parquet (23)
+ +- ^ ScanTransformer parquet (23)
+- == Initial Plan ==
TakeOrderedAndProject (67)
+- HashAggregate (66)
@@ -62,7 +62,7 @@ AdaptiveSparkPlan (68)
+- Scan parquet (59)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -73,7 +73,7 @@ ReadSchema: struct
-(2) Scan parquet
+(2) ScanTransformer parquet
Output [3]: [o_orderkey#X, o_custkey#X, o_comment#X]
Batched: true
Location: InMemoryFileIndex [*]
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 6d2233b3e09f..61b9acc60dc0 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
@@ -13,12 +13,12 @@ AdaptiveSparkPlan (35)
+- ^ BroadcastHashJoinExecTransformer Inner BuildRight (11)
:- ^ ProjectExecTransformer (3)
: +- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
+ : +- ^ ScanTransformer parquet (1)
+- ^ InputIteratorTransformer (10)
+- BroadcastQueryStage (8), Statistics(X)
+- ColumnarBroadcastExchange (7)
+- ^ FilterExecTransformer (5)
- +- ^ Scan parquet (4)
+ +- ^ ScanTransformer parquet (4)
+- == Initial Plan ==
HashAggregate (34)
+- Exchange (33)
@@ -33,7 +33,7 @@ AdaptiveSparkPlan (35)
+- Scan parquet (27)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -48,7 +48,7 @@ Arguments: (((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1995-09-01)) AND (l_s
Output [3]: [l_partkey#X, l_extendedprice#X, l_discount#X]
Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
-(4) Scan parquet
+(4) ScanTransformer parquet
Output [2]: [p_partkey#X, p_type#X]
Batched: true
Location: InMemoryFileIndex [*]
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 1ec03a2f9edd..48b06986ee1b 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
@@ -12,7 +12,7 @@ AdaptiveSparkPlan (43)
: +- BroadcastQueryStage (5), Statistics(X)
: +- ColumnarBroadcastExchange (4)
: +- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
+ : +- ^ ScanTransformer parquet (1)
+- ^ FilterExecTransformer (20)
+- ^ RegularHashAggregateExecTransformer (19)
+- ^ InputIteratorTransformer (18)
@@ -23,7 +23,7 @@ AdaptiveSparkPlan (43)
+- ^ FlushableHashAggregateExecTransformer (11)
+- ^ ProjectExecTransformer (10)
+- ^ FilterExecTransformer (9)
- +- ^ Scan parquet (8)
+ +- ^ ScanTransformer parquet (8)
+- == Initial Plan ==
Sort (42)
+- Exchange (41)
@@ -41,7 +41,7 @@ AdaptiveSparkPlan (43)
+- Scan parquet (32)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -70,7 +70,7 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X]
(7) InputIteratorTransformer
Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X]
-(8) Scan parquet
+(8) ScanTransformer parquet
Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -239,34 +239,33 @@ Arguments: isFinalPlan=true
===== Subqueries =====
Subquery:1 Hosting operator id = 20 Hosting Expression = Subquery subquery#X, [id=#X]
-AdaptiveSparkPlan (69)
+AdaptiveSparkPlan (68)
+- == Final Plan ==
- VeloxColumnarToRow (60)
- +- ^ RegularHashAggregateExecTransformer (58)
- +- ^ RegularHashAggregateExecTransformer (57)
- +- ^ ProjectExecTransformer (56)
- +- ^ RegularHashAggregateExecTransformer (55)
- +- ^ InputIteratorTransformer (54)
- +- ShuffleQueryStage (52), Statistics(X)
- +- ColumnarExchange (51)
- +- VeloxResizeBatches (50)
- +- ^ ProjectExecTransformer (48)
- +- ^ FlushableHashAggregateExecTransformer (47)
- +- ^ ProjectExecTransformer (46)
- +- ^ FilterExecTransformer (45)
- +- ^ Scan parquet (44)
+ VeloxColumnarToRow (59)
+ +- ^ RegularHashAggregateExecTransformer (57)
+ +- ^ ProjectExecTransformer (56)
+ +- ^ RegularHashAggregateExecTransformer (55)
+ +- ^ InputIteratorTransformer (54)
+ +- ShuffleQueryStage (52), Statistics(X)
+ +- ColumnarExchange (51)
+ +- VeloxResizeBatches (50)
+ +- ^ ProjectExecTransformer (48)
+ +- ^ FlushableHashAggregateExecTransformer (47)
+ +- ^ ProjectExecTransformer (46)
+ +- ^ FilterExecTransformer (45)
+ +- ^ ScanTransformer parquet (44)
+- == Initial Plan ==
- HashAggregate (68)
- +- HashAggregate (67)
- +- HashAggregate (66)
- +- Exchange (65)
- +- HashAggregate (64)
- +- Project (63)
- +- Filter (62)
- +- Scan parquet (61)
+ HashAggregate (67)
+ +- HashAggregate (66)
+ +- HashAggregate (65)
+ +- Exchange (64)
+ +- HashAggregate (63)
+ +- Project (62)
+ +- Filter (61)
+ +- Scan parquet (60)
-(44) Scan parquet
+(44) ScanTransformer parquet
Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -328,71 +327,64 @@ Input [2]: [l_suppkey#X, sum((l_extendedprice#X * (1 - l_discount#X)))#X]
(57) RegularHashAggregateExecTransformer
Input [1]: [total_revenue#X]
Keys: []
-Functions [1]: [partial_max(total_revenue#X)]
-Aggregate Attributes [1]: [max#X]
-Results [1]: [max#X]
-
-(58) RegularHashAggregateExecTransformer
-Input [1]: [max#X]
-Keys: []
Functions [1]: [max(total_revenue#X)]
Aggregate Attributes [1]: [max(total_revenue#X)#X]
Results [1]: [max(total_revenue#X)#X AS max(total_revenue)#X]
-(59) WholeStageCodegenTransformer (X)
+(58) WholeStageCodegenTransformer (X)
Input [1]: [max(total_revenue)#X]
Arguments: false
-(60) VeloxColumnarToRow
+(59) VeloxColumnarToRow
Input [1]: [max(total_revenue)#X]
-(61) Scan parquet
+(60) Scan parquet
Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1996-01-01), LessThan(l_shipdate,1996-04-01)]
ReadSchema: struct
-(62) Filter
+(61) Filter
Input [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Condition : ((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1996-01-01)) AND (l_shipdate#X < 1996-04-01))
-(63) Project
+(62) Project
Output [3]: [l_suppkey#X, l_extendedprice#X, l_discount#X]
Input [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
-(64) HashAggregate
+(63) HashAggregate
Input [3]: [l_suppkey#X, l_extendedprice#X, l_discount#X]
Keys [1]: [l_suppkey#X]
Functions [1]: [partial_sum((l_extendedprice#X * (1 - l_discount#X)))]
Aggregate Attributes [2]: [sum#X, isEmpty#X]
Results [3]: [l_suppkey#X, sum#X, isEmpty#X]
-(65) Exchange
+(64) Exchange
Input [3]: [l_suppkey#X, sum#X, isEmpty#X]
Arguments: hashpartitioning(l_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(66) HashAggregate
+(65) HashAggregate
Input [3]: [l_suppkey#X, sum#X, isEmpty#X]
Keys [1]: [l_suppkey#X]
Functions [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))]
Aggregate Attributes [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))#X]
Results [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))#X AS total_revenue#X]
-(67) HashAggregate
+(66) HashAggregate
Input [1]: [total_revenue#X]
Keys: []
Functions [1]: [partial_max(total_revenue#X)]
Aggregate Attributes [1]: [max#X]
Results [1]: [max#X]
-(68) HashAggregate
+(67) HashAggregate
Input [1]: [max#X]
Keys: []
Functions [1]: [max(total_revenue#X)]
Aggregate Attributes [1]: [max(total_revenue#X)#X]
Results [1]: [max(total_revenue#X)#X AS max(total_revenue)#X]
-(69) AdaptiveSparkPlan
+(68) AdaptiveSparkPlan
Output [1]: [max(total_revenue)#X]
Arguments: isFinalPlan=true
\ No newline at end of file
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 e64cb0990026..3a9b17afd60c 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
@@ -24,12 +24,12 @@ AdaptiveSparkPlan (59)
+- ^ ProjectExecTransformer (11)
+- ^ BroadcastHashJoinExecTransformer Inner BuildRight (10)
:- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
+ : +- ^ ScanTransformer parquet (1)
+- ^ InputIteratorTransformer (9)
+- BroadcastQueryStage (7), Statistics(X)
+- ColumnarBroadcastExchange (6)
+- ^ FilterExecTransformer (4)
- +- ^ Scan parquet (3)
+ +- ^ ScanTransformer parquet (3)
+- == Initial Plan ==
Sort (58)
+- Exchange (57)
@@ -53,7 +53,7 @@ AdaptiveSparkPlan (59)
+- Scan parquet (46)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [ps_partkey#X, ps_suppkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -64,7 +64,7 @@ ReadSchema: struct
Input [2]: [ps_partkey#X, ps_suppkey#X]
Arguments: isnotnull(ps_partkey#X)
-(3) Scan parquet
+(3) ScanTransformer parquet
Output [4]: [p_partkey#X, p_brand#X, p_type#X, p_size#X]
Batched: true
Location: InMemoryFileIndex [*]
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 c92587c9d791..c3ab86e5acfb 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
@@ -18,10 +18,10 @@ AdaptiveSparkPlan (88)
: : +- BroadcastQueryStage (5), Statistics(X)
: : +- ColumnarBroadcastExchange (4)
: : +- ^ FilterExecTransformer (2)
- : : +- ^ Scan parquet (1)
+ : : +- ^ ScanTransformer parquet (1)
: +- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (27)
: :- ^ FilterExecTransformer (9)
- : : +- ^ Scan parquet (8)
+ : : +- ^ ScanTransformer parquet (8)
: +- ^ InputIteratorTransformer (26)
: +- BroadcastQueryStage (24), Statistics(X)
: +- ColumnarBroadcastExchange (23)
@@ -34,13 +34,13 @@ AdaptiveSparkPlan (88)
: +- VeloxResizeBatches (14)
: +- ^ ProjectExecTransformer (12)
: +- ^ FlushableHashAggregateExecTransformer (11)
- : +- ^ Scan parquet (10)
+ : +- ^ ScanTransformer parquet (10)
+- ^ InputIteratorTransformer (41)
+- BroadcastQueryStage (39), Statistics(X)
+- ColumnarBroadcastExchange (38)
+- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (36)
:- ^ FilterExecTransformer (31)
- : +- ^ Scan parquet (30)
+ : +- ^ ScanTransformer parquet (30)
+- ^ InputIteratorTransformer (35)
+- BroadcastQueryStage (33), Statistics(X)
+- ReusedExchange (32)
@@ -79,7 +79,7 @@ AdaptiveSparkPlan (88)
+- Scan parquet (73)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [c_custkey#X, c_name#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -108,7 +108,7 @@ Input [2]: [c_custkey#X, c_name#X]
(7) InputIteratorTransformer
Input [2]: [c_custkey#X, c_name#X]
-(8) Scan parquet
+(8) ScanTransformer parquet
Output [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -119,7 +119,7 @@ ReadSchema: struct
Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
Arguments: ((isnotnull(ps_availqty#X) AND isnotnull(ps_partkey#X)) AND isnotnull(ps_suppkey#X))
-(5) Scan parquet
+(5) ScanTransformer parquet
Output [2]: [p_partkey#X, p_name#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -168,7 +168,7 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
(18) InputIteratorTransformer
Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
-(19) Scan parquet
+(19) ScanTransformer parquet
Output [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -288,7 +288,7 @@ Join condition: None
Output [3]: [s_name#X, s_address#X, s_nationkey#X]
Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X]
-(47) Scan parquet
+(47) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 8382dbcd504e..5fab9fa9da42 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
@@ -19,34 +19,34 @@ AdaptiveSparkPlan (92)
: : : +- BroadcastQueryStage (5), Statistics(X)
: : : +- ColumnarBroadcastExchange (4)
: : : +- ^ FilterExecTransformer (2)
- : : : +- ^ Scan parquet (1)
+ : : : +- ^ ScanTransformer parquet (1)
: : +- ^ BroadcastHashJoinExecTransformer LeftAnti BuildRight (26)
: : :- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (17)
: : : :- ^ ProjectExecTransformer (10)
: : : : +- ^ FilterExecTransformer (9)
- : : : : +- ^ Scan parquet (8)
+ : : : : +- ^ ScanTransformer parquet (8)
: : : +- ^ InputIteratorTransformer (16)
: : : +- BroadcastQueryStage (14), Statistics(X)
: : : +- ColumnarBroadcastExchange (13)
- : : : +- ^ Scan parquet (11)
+ : : : +- ^ ScanTransformer parquet (11)
: : +- ^ InputIteratorTransformer (25)
: : +- BroadcastQueryStage (23), Statistics(X)
: : +- ColumnarBroadcastExchange (22)
: : +- ^ ProjectExecTransformer (20)
: : +- ^ FilterExecTransformer (19)
- : : +- ^ Scan parquet (18)
+ : : +- ^ ScanTransformer parquet (18)
: +- ^ InputIteratorTransformer (36)
: +- BroadcastQueryStage (34), Statistics(X)
: +- ColumnarBroadcastExchange (33)
: +- ^ ProjectExecTransformer (31)
: +- ^ FilterExecTransformer (30)
- : +- ^ Scan parquet (29)
+ : +- ^ ScanTransformer parquet (29)
+- ^ InputIteratorTransformer (46)
+- BroadcastQueryStage (44), Statistics(X)
+- ColumnarBroadcastExchange (43)
+- ^ ProjectExecTransformer (41)
+- ^ FilterExecTransformer (40)
- +- ^ Scan parquet (39)
+ +- ^ ScanTransformer parquet (39)
+- == Initial Plan ==
TakeOrderedAndProject (91)
+- HashAggregate (90)
@@ -82,7 +82,7 @@ AdaptiveSparkPlan (92)
+- Scan parquet (82)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [3]: [s_suppkey#X, s_name#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -111,7 +111,7 @@ Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X]
(7) InputIteratorTransformer
Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X]
-(8) Scan parquet
+(8) ScanTransformer parquet
Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -126,7 +126,7 @@ Arguments: ((((isnotnull(l_receiptdate#X) AND isnotnull(l_commitdate#X)) AND (l_
Output [2]: [l_orderkey#X, l_suppkey#X]
Input [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X]
-(11) Scan parquet
+(11) ScanTransformer parquet
Output [2]: [l_orderkey#X, l_suppkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -156,7 +156,7 @@ Right keys [1]: [l_orderkey#X]
Join type: LeftSemi
Join condition: NOT (l_suppkey#X = l_suppkey#X)
-(18) Scan parquet
+(18) ScanTransformer parquet
Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -205,7 +205,7 @@ Join condition: None
Output [3]: [s_name#X, s_nationkey#X, l_orderkey#X]
Input [5]: [s_suppkey#X, s_name#X, s_nationkey#X, l_orderkey#X, l_suppkey#X]
-(29) Scan parquet
+(29) ScanTransformer parquet
Output [2]: [o_orderkey#X, o_orderstatus#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -248,7 +248,7 @@ Join condition: None
Output [2]: [s_name#X, s_nationkey#X]
Input [4]: [s_name#X, s_nationkey#X, l_orderkey#X, o_orderkey#X]
-(39) Scan parquet
+(39) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 7f3836740155..0c2ce3c56b5a 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
@@ -17,11 +17,11 @@ AdaptiveSparkPlan (40)
+- ^ ProjectExecTransformer (10)
+- ^ BroadcastHashJoinExecTransformer LeftAnti BuildRight (9)
:- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
+ : +- ^ ScanTransformer parquet (1)
+- ^ InputIteratorTransformer (8)
+- BroadcastQueryStage (6), Statistics(X)
+- ColumnarBroadcastExchange (5)
- +- ^ Scan parquet (3)
+ +- ^ ScanTransformer parquet (3)
+- == Initial Plan ==
Sort (39)
+- Exchange (38)
@@ -36,7 +36,7 @@ AdaptiveSparkPlan (40)
+- Scan parquet (31)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [3]: [c_custkey#X, c_phone#X, c_acctbal#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -47,7 +47,7 @@ ReadSchema: struct
Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X]
Arguments: ((isnotnull(c_acctbal#X) AND substring(c_phone#X, 1, 2) IN (13,31,23,29,30,18,17)) AND (cast(c_acctbal#X as decimal(16,6)) > Subquery subquery#X, [id=#X]))
-(3) Scan parquet
+(3) ScanTransformer parquet
Output [1]: [o_custkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -229,7 +229,7 @@ AdaptiveSparkPlan (60)
+- ^ FlushableHashAggregateExecTransformer (44)
+- ^ ProjectExecTransformer (43)
+- ^ FilterExecTransformer (42)
- +- ^ Scan parquet (41)
+ +- ^ ScanTransformer parquet (41)
+- == Initial Plan ==
HashAggregate (59)
+- Exchange (58)
@@ -239,7 +239,7 @@ AdaptiveSparkPlan (60)
+- Scan parquet (54)
-(41) Scan parquet
+(41) ScanTransformer parquet
Output [2]: [c_phone#X, c_acctbal#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -346,7 +346,7 @@ AdaptiveSparkPlan (60)
+- ^ FlushableHashAggregateExecTransformer (44)
+- ^ ProjectExecTransformer (43)
+- ^ FilterExecTransformer (42)
- +- ^ Scan parquet (41)
+ +- ^ ScanTransformer parquet (41)
+- == Initial Plan ==
HashAggregate (59)
+- Exchange (58)
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 1035e0eb7715..cc68e1c9b249 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
@@ -20,15 +20,15 @@ AdaptiveSparkPlan (54)
: : +- ColumnarBroadcastExchange (5)
: : +- ^ ProjectExecTransformer (3)
: : +- ^ FilterExecTransformer (2)
- : : +- ^ Scan parquet (1)
+ : : +- ^ ScanTransformer parquet (1)
: +- ^ FilterExecTransformer (10)
- : +- ^ Scan parquet (9)
+ : +- ^ ScanTransformer parquet (9)
+- ^ InputIteratorTransformer (20)
+- BroadcastQueryStage (18), Statistics(X)
+- ColumnarBroadcastExchange (17)
+- ^ ProjectExecTransformer (15)
+- ^ FilterExecTransformer (14)
- +- ^ Scan parquet (13)
+ +- ^ ScanTransformer parquet (13)
+- == Initial Plan ==
TakeOrderedAndProject (53)
+- HashAggregate (52)
@@ -50,7 +50,7 @@ AdaptiveSparkPlan (54)
+- Scan parquet (44)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [c_custkey#X, c_mktsegment#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -83,7 +83,7 @@ Input [1]: [c_custkey#X]
(8) InputIteratorTransformer
Input [1]: [c_custkey#X]
-(9) Scan parquet
+(9) ScanTransformer parquet
Output [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -104,7 +104,7 @@ Join condition: None
Output [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X]
Input [5]: [c_custkey#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X]
-(13) Scan parquet
+(13) ScanTransformer parquet
Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
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 150788a3d7c1..a77b48298d37 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
@@ -18,13 +18,13 @@ AdaptiveSparkPlan (46)
+- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (12)
:- ^ ProjectExecTransformer (3)
: +- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
+ : +- ^ ScanTransformer parquet (1)
+- ^ InputIteratorTransformer (11)
+- BroadcastQueryStage (9), Statistics(X)
+- ColumnarBroadcastExchange (8)
+- ^ ProjectExecTransformer (6)
+- ^ FilterExecTransformer (5)
- +- ^ Scan parquet (4)
+ +- ^ ScanTransformer parquet (4)
+- == Initial Plan ==
Sort (45)
+- Exchange (44)
@@ -42,7 +42,7 @@ AdaptiveSparkPlan (46)
+- Scan parquet (35)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -57,7 +57,7 @@ Arguments: ((isnotnull(o_orderdate#X) AND (o_orderdate#X >= 1993-07-01)) AND (o_
Output [2]: [o_orderkey#X, o_orderpriority#X]
Input [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X]
-(4) Scan parquet
+(4) ScanTransformer parquet
Output [3]: [l_orderkey#X, l_commitdate#X, l_receiptdate#X]
Batched: true
Location: InMemoryFileIndex [*]
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 035c5c388972..58442a8ca86e 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
@@ -28,31 +28,31 @@ AdaptiveSparkPlan (102)
: : : : : +- BroadcastQueryStage (5), Statistics(X)
: : : : : +- ColumnarBroadcastExchange (4)
: : : : : +- ^ FilterExecTransformer (2)
- : : : : : +- ^ Scan parquet (1)
+ : : : : : +- ^ ScanTransformer parquet (1)
: : : : +- ^ ProjectExecTransformer (10)
: : : : +- ^ FilterExecTransformer (9)
- : : : : +- ^ Scan parquet (8)
+ : : : : +- ^ ScanTransformer parquet (8)
: : : +- ^ InputIteratorTransformer (19)
: : : +- BroadcastQueryStage (17), Statistics(X)
: : : +- ColumnarBroadcastExchange (16)
: : : +- ^ FilterExecTransformer (14)
- : : : +- ^ Scan parquet (13)
+ : : : +- ^ ScanTransformer parquet (13)
: : +- ^ InputIteratorTransformer (28)
: : +- BroadcastQueryStage (26), Statistics(X)
: : +- ColumnarBroadcastExchange (25)
: : +- ^ FilterExecTransformer (23)
- : : +- ^ Scan parquet (22)
+ : : +- ^ ScanTransformer parquet (22)
: +- ^ InputIteratorTransformer (37)
: +- BroadcastQueryStage (35), Statistics(X)
: +- ColumnarBroadcastExchange (34)
: +- ^ FilterExecTransformer (32)
- : +- ^ Scan parquet (31)
+ : +- ^ ScanTransformer parquet (31)
+- ^ InputIteratorTransformer (47)
+- BroadcastQueryStage (45), Statistics(X)
+- ColumnarBroadcastExchange (44)
+- ^ ProjectExecTransformer (42)
+- ^ FilterExecTransformer (41)
- +- ^ Scan parquet (40)
+ +- ^ ScanTransformer parquet (40)
+- == Initial Plan ==
Sort (101)
+- Exchange (100)
@@ -90,7 +90,7 @@ AdaptiveSparkPlan (102)
+- Scan parquet (91)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [c_custkey#X, c_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -119,7 +119,7 @@ Input [2]: [c_custkey#X, c_nationkey#X]
(7) InputIteratorTransformer
Input [2]: [c_custkey#X, c_nationkey#X]
-(8) Scan parquet
+(8) ScanTransformer parquet
Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -144,7 +144,7 @@ Join condition: None
Output [2]: [c_nationkey#X, o_orderkey#X]
Input [4]: [c_custkey#X, c_nationkey#X, o_orderkey#X, o_custkey#X]
-(13) Scan parquet
+(13) ScanTransformer parquet
Output [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -183,7 +183,7 @@ Join condition: None
Output [4]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
Input [6]: [c_nationkey#X, o_orderkey#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
-(22) Scan parquet
+(22) ScanTransformer parquet
Output [2]: [s_suppkey#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -222,7 +222,7 @@ Join condition: None
Output [3]: [l_extendedprice#X, l_discount#X, s_nationkey#X]
Input [6]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, s_suppkey#X, s_nationkey#X]
-(31) Scan parquet
+(31) ScanTransformer parquet
Output [3]: [n_nationkey#X, n_name#X, n_regionkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -261,7 +261,7 @@ Join condition: None
Output [4]: [l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X]
Input [6]: [l_extendedprice#X, l_discount#X, s_nationkey#X, n_nationkey#X, n_name#X, n_regionkey#X]
-(40) Scan parquet
+(40) ScanTransformer parquet
Output [2]: [r_regionkey#X, r_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 e9dc68456e55..319c457c1b4f 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
@@ -10,7 +10,7 @@ AdaptiveSparkPlan (20)
+- ^ FlushableHashAggregateExecTransformer (4)
+- ^ ProjectExecTransformer (3)
+- ^ FilterExecTransformer (2)
- +- ^ Scan parquet (1)
+ +- ^ ScanTransformer parquet (1)
+- == Initial Plan ==
HashAggregate (19)
+- Exchange (18)
@@ -20,7 +20,7 @@ AdaptiveSparkPlan (20)
+- Scan parquet (14)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [4]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
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 cd877f54bd7a..eaa84cfa46c2 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
@@ -28,24 +28,24 @@ AdaptiveSparkPlan (95)
: : : : : +- BroadcastQueryStage (5), Statistics(X)
: : : : : +- ColumnarBroadcastExchange (4)
: : : : : +- ^ FilterExecTransformer (2)
- : : : : : +- ^ Scan parquet (1)
+ : : : : : +- ^ ScanTransformer parquet (1)
: : : : +- ^ FilterExecTransformer (9)
- : : : : +- ^ Scan parquet (8)
+ : : : : +- ^ ScanTransformer parquet (8)
: : : +- ^ InputIteratorTransformer (18)
: : : +- BroadcastQueryStage (16), Statistics(X)
: : : +- ColumnarBroadcastExchange (15)
: : : +- ^ FilterExecTransformer (13)
- : : : +- ^ Scan parquet (12)
+ : : : +- ^ ScanTransformer parquet (12)
: : +- ^ InputIteratorTransformer (27)
: : +- BroadcastQueryStage (25), Statistics(X)
: : +- ColumnarBroadcastExchange (24)
: : +- ^ FilterExecTransformer (22)
- : : +- ^ Scan parquet (21)
+ : : +- ^ ScanTransformer parquet (21)
: +- ^ InputIteratorTransformer (36)
: +- BroadcastQueryStage (34), Statistics(X)
: +- ColumnarBroadcastExchange (33)
: +- ^ FilterExecTransformer (31)
- : +- ^ Scan parquet (30)
+ : +- ^ ScanTransformer parquet (30)
+- ^ InputIteratorTransformer (42)
+- BroadcastQueryStage (40), Statistics(X)
+- ReusedExchange (39)
@@ -84,7 +84,7 @@ AdaptiveSparkPlan (95)
+- Scan parquet (85)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [s_suppkey#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -113,7 +113,7 @@ Input [2]: [s_suppkey#X, s_nationkey#X]
(7) InputIteratorTransformer
Input [2]: [s_suppkey#X, s_nationkey#X]
-(8) Scan parquet
+(8) ScanTransformer parquet
Output [5]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -134,7 +134,7 @@ Join condition: None
Output [5]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Input [7]: [s_suppkey#X, s_nationkey#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
-(12) Scan parquet
+(12) ScanTransformer parquet
Output [2]: [o_orderkey#X, o_custkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -173,7 +173,7 @@ Join condition: None
Output [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X]
Input [7]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_orderkey#X, o_custkey#X]
-(21) Scan parquet
+(21) ScanTransformer parquet
Output [2]: [c_custkey#X, c_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -212,7 +212,7 @@ Join condition: None
Output [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X]
Input [7]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X, c_custkey#X, c_nationkey#X]
-(30) Scan parquet
+(30) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 d381d21da73d..9d1bc4a17558 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
@@ -34,40 +34,40 @@ AdaptiveSparkPlan (131)
: : : : : : : +- ColumnarBroadcastExchange (5)
: : : : : : : +- ^ ProjectExecTransformer (3)
: : : : : : : +- ^ FilterExecTransformer (2)
- : : : : : : : +- ^ Scan parquet (1)
+ : : : : : : : +- ^ ScanTransformer parquet (1)
: : : : : : +- ^ FilterExecTransformer (10)
- : : : : : : +- ^ Scan parquet (9)
+ : : : : : : +- ^ ScanTransformer parquet (9)
: : : : : +- ^ InputIteratorTransformer (19)
: : : : : +- BroadcastQueryStage (17), Statistics(X)
: : : : : +- ColumnarBroadcastExchange (16)
: : : : : +- ^ FilterExecTransformer (14)
- : : : : : +- ^ Scan parquet (13)
+ : : : : : +- ^ ScanTransformer parquet (13)
: : : : +- ^ InputIteratorTransformer (28)
: : : : +- BroadcastQueryStage (26), Statistics(X)
: : : : +- ColumnarBroadcastExchange (25)
: : : : +- ^ FilterExecTransformer (23)
- : : : : +- ^ Scan parquet (22)
+ : : : : +- ^ ScanTransformer parquet (22)
: : : +- ^ InputIteratorTransformer (37)
: : : +- BroadcastQueryStage (35), Statistics(X)
: : : +- ColumnarBroadcastExchange (34)
: : : +- ^ FilterExecTransformer (32)
- : : : +- ^ Scan parquet (31)
+ : : : +- ^ ScanTransformer parquet (31)
: : +- ^ InputIteratorTransformer (46)
: : +- BroadcastQueryStage (44), Statistics(X)
: : +- ColumnarBroadcastExchange (43)
: : +- ^ FilterExecTransformer (41)
- : : +- ^ Scan parquet (40)
+ : : +- ^ ScanTransformer parquet (40)
: +- ^ InputIteratorTransformer (55)
: +- BroadcastQueryStage (53), Statistics(X)
: +- ColumnarBroadcastExchange (52)
: +- ^ FilterExecTransformer (50)
- : +- ^ Scan parquet (49)
+ : +- ^ ScanTransformer parquet (49)
+- ^ InputIteratorTransformer (65)
+- BroadcastQueryStage (63), Statistics(X)
+- ColumnarBroadcastExchange (62)
+- ^ ProjectExecTransformer (60)
+- ^ FilterExecTransformer (59)
- +- ^ Scan parquet (58)
+ +- ^ ScanTransformer parquet (58)
+- == Initial Plan ==
Sort (130)
+- Exchange (129)
@@ -115,7 +115,7 @@ AdaptiveSparkPlan (131)
+- Scan parquet (120)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [p_partkey#X, p_type#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -148,7 +148,7 @@ Input [1]: [p_partkey#X]
(8) InputIteratorTransformer
Input [1]: [p_partkey#X]
-(9) Scan parquet
+(9) ScanTransformer parquet
Output [5]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -169,7 +169,7 @@ Join condition: None
Output [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
Input [6]: [p_partkey#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
-(13) Scan parquet
+(13) ScanTransformer parquet
Output [2]: [s_suppkey#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -208,7 +208,7 @@ Join condition: None
Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X]
Input [6]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, s_suppkey#X, s_nationkey#X]
-(22) Scan parquet
+(22) ScanTransformer parquet
Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -247,7 +247,7 @@ Join condition: None
Output [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X]
Input [7]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderkey#X, o_custkey#X, o_orderdate#X]
-(31) Scan parquet
+(31) ScanTransformer parquet
Output [2]: [c_custkey#X, c_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -286,7 +286,7 @@ Join condition: None
Output [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X]
Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X, c_custkey#X, c_nationkey#X]
-(40) Scan parquet
+(40) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_regionkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -325,7 +325,7 @@ Join condition: None
Output [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X]
Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X, n_nationkey#X, n_regionkey#X]
-(49) Scan parquet
+(49) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -364,7 +364,7 @@ Join condition: None
Output [5]: [l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X]
Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X, n_nationkey#X, n_name#X]
-(58) Scan parquet
+(58) ScanTransformer parquet
Output [2]: [r_regionkey#X, r_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 f0ebba6efac5..8baec4075ad3 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
@@ -29,29 +29,29 @@ AdaptiveSparkPlan (100)
: : : : : +- ColumnarBroadcastExchange (5)
: : : : : +- ^ ProjectExecTransformer (3)
: : : : : +- ^ FilterExecTransformer (2)
- : : : : : +- ^ Scan parquet (1)
+ : : : : : +- ^ ScanTransformer parquet (1)
: : : : +- ^ FilterExecTransformer (10)
- : : : : +- ^ Scan parquet (9)
+ : : : : +- ^ ScanTransformer parquet (9)
: : : +- ^ InputIteratorTransformer (19)
: : : +- BroadcastQueryStage (17), Statistics(X)
: : : +- ColumnarBroadcastExchange (16)
: : : +- ^ FilterExecTransformer (14)
- : : : +- ^ Scan parquet (13)
+ : : : +- ^ ScanTransformer parquet (13)
: : +- ^ InputIteratorTransformer (28)
: : +- BroadcastQueryStage (26), Statistics(X)
: : +- ColumnarBroadcastExchange (25)
: : +- ^ FilterExecTransformer (23)
- : : +- ^ Scan parquet (22)
+ : : +- ^ ScanTransformer parquet (22)
: +- ^ InputIteratorTransformer (37)
: +- BroadcastQueryStage (35), Statistics(X)
: +- ColumnarBroadcastExchange (34)
: +- ^ FilterExecTransformer (32)
- : +- ^ Scan parquet (31)
+ : +- ^ ScanTransformer parquet (31)
+- ^ InputIteratorTransformer (46)
+- BroadcastQueryStage (44), Statistics(X)
+- ColumnarBroadcastExchange (43)
+- ^ FilterExecTransformer (41)
- +- ^ Scan parquet (40)
+ +- ^ ScanTransformer parquet (40)
+- == Initial Plan ==
Sort (99)
+- Exchange (98)
@@ -88,7 +88,7 @@ AdaptiveSparkPlan (100)
+- Scan parquet (90)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [p_partkey#X, p_name#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -121,7 +121,7 @@ Input [1]: [p_partkey#X]
(8) InputIteratorTransformer
Input [1]: [p_partkey#X]
-(9) Scan parquet
+(9) ScanTransformer parquet
Output [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -142,7 +142,7 @@ Join condition: None
Output [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X]
Input [7]: [p_partkey#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X]
-(13) Scan parquet
+(13) ScanTransformer parquet
Output [2]: [s_suppkey#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -181,7 +181,7 @@ Join condition: None
Output [7]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X]
Input [8]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_suppkey#X, s_nationkey#X]
-(22) Scan parquet
+(22) ScanTransformer parquet
Output [3]: [ps_partkey#X, ps_suppkey#X, ps_supplycost#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -220,7 +220,7 @@ Join condition: None
Output [6]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X]
Input [10]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_partkey#X, ps_suppkey#X, ps_supplycost#X]
-(31) Scan parquet
+(31) ScanTransformer parquet
Output [2]: [o_orderkey#X, o_orderdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -259,7 +259,7 @@ Join condition: None
Output [6]: [l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X]
Input [8]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderkey#X, o_orderdate#X]
-(40) Scan parquet
+(40) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 efe7f3908506..617489f1aa35 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
@@ -16,7 +16,7 @@ AdaptiveSparkPlan (30)
+- ^ FlushableHashAggregateExecTransformer (4)
+- ^ ProjectExecTransformer (3)
+- ^ FilterExecTransformer (2)
- +- ^ Scan parquet (1)
+ +- ^ ScanTransformer parquet (1)
+- == Initial Plan ==
Sort (29)
+- Exchange (28)
@@ -28,7 +28,7 @@ AdaptiveSparkPlan (30)
+- Scan parquet (22)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [7]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_tax#X, l_returnflag#X, l_linestatus#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
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 5b913a06db13..7197e83986a8 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
@@ -18,24 +18,24 @@ AdaptiveSparkPlan (68)
: :- ^ ProjectExecTransformer (12)
: : +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (11)
: : :- ^ FilterExecTransformer (2)
- : : : +- ^ Scan parquet (1)
+ : : : +- ^ ScanTransformer parquet (1)
: : +- ^ InputIteratorTransformer (10)
: : +- BroadcastQueryStage (8)
: : +- ColumnarBroadcastExchange (7)
: : +- ^ ProjectExecTransformer (5)
: : +- ^ FilterExecTransformer (4)
- : : +- ^ Scan parquet (3)
+ : : +- ^ ScanTransformer parquet (3)
: +- ^ InputIteratorTransformer (20)
: +- BroadcastQueryStage (18)
: +- ColumnarBroadcastExchange (17)
: +- ^ ProjectExecTransformer (15)
: +- ^ FilterExecTransformer (14)
- : +- ^ Scan parquet (13)
+ : +- ^ ScanTransformer parquet (13)
+- ^ InputIteratorTransformer (29)
+- BroadcastQueryStage (27)
+- ColumnarBroadcastExchange (26)
+- ^ FilterExecTransformer (24)
- +- ^ Scan parquet (23)
+ +- ^ ScanTransformer parquet (23)
+- == Initial Plan ==
TakeOrderedAndProject (67)
+- HashAggregate (66)
@@ -62,7 +62,7 @@ AdaptiveSparkPlan (68)
+- Scan parquet (59)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -73,7 +73,7 @@ ReadSchema: struct
-(2) Scan parquet
+(2) ScanTransformer parquet
Output [3]: [o_orderkey#X, o_custkey#X, o_comment#X]
Batched: true
Location: InMemoryFileIndex [*]
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 946c150d8654..4687bd81bcc8 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
@@ -13,12 +13,12 @@ AdaptiveSparkPlan (35)
+- ^ BroadcastHashJoinExecTransformer Inner BuildRight (11)
:- ^ ProjectExecTransformer (3)
: +- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
+ : +- ^ ScanTransformer parquet (1)
+- ^ InputIteratorTransformer (10)
+- BroadcastQueryStage (8)
+- ColumnarBroadcastExchange (7)
+- ^ FilterExecTransformer (5)
- +- ^ Scan parquet (4)
+ +- ^ ScanTransformer parquet (4)
+- == Initial Plan ==
HashAggregate (34)
+- Exchange (33)
@@ -33,7 +33,7 @@ AdaptiveSparkPlan (35)
+- Scan parquet (27)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -48,7 +48,7 @@ Arguments: (((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1995-09-01)) AND (l_s
Output [3]: [l_partkey#X, l_extendedprice#X, l_discount#X]
Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
-(4) Scan parquet
+(4) ScanTransformer parquet
Output [2]: [p_partkey#X, p_type#X]
Batched: true
Location: InMemoryFileIndex [*]
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 39b1c80d441b..cd765112a70f 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
@@ -13,7 +13,7 @@ AdaptiveSparkPlan (46)
: +- BroadcastQueryStage (5)
: +- ColumnarBroadcastExchange (4)
: +- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
+ : +- ^ ScanTransformer parquet (1)
+- ^ FilterExecTransformer (20)
+- ^ RegularHashAggregateExecTransformer (19)
+- ^ InputIteratorTransformer (18)
@@ -24,7 +24,7 @@ AdaptiveSparkPlan (46)
+- ^ FlushableHashAggregateExecTransformer (11)
+- ^ ProjectExecTransformer (10)
+- ^ FilterExecTransformer (9)
- +- ^ Scan parquet (8)
+ +- ^ ScanTransformer parquet (8)
+- == Initial Plan ==
Sort (45)
+- Exchange (44)
@@ -42,7 +42,7 @@ AdaptiveSparkPlan (46)
+- Scan parquet (35)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -71,7 +71,7 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X]
(7) InputIteratorTransformer
Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X]
-(8) Scan parquet
+(8) ScanTransformer parquet
Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
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 f7b9f730e38b..10972d6c2f1a 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
@@ -24,12 +24,12 @@ AdaptiveSparkPlan (59)
+- ^ ProjectExecTransformer (11)
+- ^ BroadcastHashJoinExecTransformer Inner BuildRight (10)
:- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
+ : +- ^ ScanTransformer parquet (1)
+- ^ InputIteratorTransformer (9)
+- BroadcastQueryStage (7)
+- ColumnarBroadcastExchange (6)
+- ^ FilterExecTransformer (4)
- +- ^ Scan parquet (3)
+ +- ^ ScanTransformer parquet (3)
+- == Initial Plan ==
Sort (58)
+- Exchange (57)
@@ -53,7 +53,7 @@ AdaptiveSparkPlan (59)
+- Scan parquet (46)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [ps_partkey#X, ps_suppkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -64,7 +64,7 @@ ReadSchema: struct
Input [2]: [ps_partkey#X, ps_suppkey#X]
Arguments: isnotnull(ps_partkey#X)
-(3) Scan parquet
+(3) ScanTransformer parquet
Output [4]: [p_partkey#X, p_brand#X, p_type#X, p_size#X]
Batched: true
Location: InMemoryFileIndex [*]
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 2cd26d020ae7..88d48cf5dfa3 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
@@ -18,10 +18,10 @@ AdaptiveSparkPlan (88)
: : +- BroadcastQueryStage (5)
: : +- ColumnarBroadcastExchange (4)
: : +- ^ FilterExecTransformer (2)
- : : +- ^ Scan parquet (1)
+ : : +- ^ ScanTransformer parquet (1)
: +- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (27)
: :- ^ FilterExecTransformer (9)
- : : +- ^ Scan parquet (8)
+ : : +- ^ ScanTransformer parquet (8)
: +- ^ InputIteratorTransformer (26)
: +- BroadcastQueryStage (24)
: +- ColumnarBroadcastExchange (23)
@@ -34,13 +34,13 @@ AdaptiveSparkPlan (88)
: +- VeloxResizeBatches (14)
: +- ^ ProjectExecTransformer (12)
: +- ^ FlushableHashAggregateExecTransformer (11)
- : +- ^ Scan parquet (10)
+ : +- ^ ScanTransformer parquet (10)
+- ^ InputIteratorTransformer (41)
+- BroadcastQueryStage (39)
+- ColumnarBroadcastExchange (38)
+- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (36)
:- ^ FilterExecTransformer (31)
- : +- ^ Scan parquet (30)
+ : +- ^ ScanTransformer parquet (30)
+- ^ InputIteratorTransformer (35)
+- BroadcastQueryStage (33)
+- ReusedExchange (32)
@@ -79,7 +79,7 @@ AdaptiveSparkPlan (88)
+- Scan parquet (73)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [c_custkey#X, c_name#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -108,7 +108,7 @@ Input [2]: [c_custkey#X, c_name#X]
(7) InputIteratorTransformer
Input [2]: [c_custkey#X, c_name#X]
-(8) Scan parquet
+(8) ScanTransformer parquet
Output [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -119,7 +119,7 @@ ReadSchema: struct
Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
Arguments: ((isnotnull(ps_availqty#X) AND isnotnull(ps_partkey#X)) AND isnotnull(ps_suppkey#X))
-(13) Scan parquet
+(13) ScanTransformer parquet
Output [2]: [p_partkey#X, p_name#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -207,7 +207,7 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
(26) InputIteratorTransformer
Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
-(27) Scan parquet
+(27) ScanTransformer parquet
Output [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -324,7 +324,7 @@ Join condition: None
Output [3]: [s_name#X, s_address#X, s_nationkey#X]
Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X]
-(55) Scan parquet
+(55) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 2ef6280501aa..a1adaf31bf28 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
@@ -20,34 +20,34 @@ AdaptiveSparkPlan (93)
: : : +- BroadcastQueryStage (5)
: : : +- ColumnarBroadcastExchange (4)
: : : +- ^ FilterExecTransformer (2)
- : : : +- ^ Scan parquet (1)
+ : : : +- ^ ScanTransformer parquet (1)
: : +- ^ BroadcastHashJoinExecTransformer LeftAnti BuildRight (26)
: : :- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (17)
: : : :- ^ ProjectExecTransformer (10)
: : : : +- ^ FilterExecTransformer (9)
- : : : : +- ^ Scan parquet (8)
+ : : : : +- ^ ScanTransformer parquet (8)
: : : +- ^ InputIteratorTransformer (16)
: : : +- BroadcastQueryStage (14)
: : : +- ColumnarBroadcastExchange (13)
- : : : +- ^ Scan parquet (11)
+ : : : +- ^ ScanTransformer parquet (11)
: : +- ^ InputIteratorTransformer (25)
: : +- BroadcastQueryStage (23)
: : +- ColumnarBroadcastExchange (22)
: : +- ^ ProjectExecTransformer (20)
: : +- ^ FilterExecTransformer (19)
- : : +- ^ Scan parquet (18)
+ : : +- ^ ScanTransformer parquet (18)
: +- ^ InputIteratorTransformer (36)
: +- BroadcastQueryStage (34)
: +- ColumnarBroadcastExchange (33)
: +- ^ ProjectExecTransformer (31)
: +- ^ FilterExecTransformer (30)
- : +- ^ Scan parquet (29)
+ : +- ^ ScanTransformer parquet (29)
+- ^ InputIteratorTransformer (46)
+- BroadcastQueryStage (44)
+- ColumnarBroadcastExchange (43)
+- ^ ProjectExecTransformer (41)
+- ^ FilterExecTransformer (40)
- +- ^ Scan parquet (39)
+ +- ^ ScanTransformer parquet (39)
+- == Initial Plan ==
TakeOrderedAndProject (92)
+- HashAggregate (91)
@@ -83,7 +83,7 @@ AdaptiveSparkPlan (93)
+- Scan parquet (83)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [3]: [s_suppkey#X, s_name#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -112,7 +112,7 @@ Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X]
(7) InputIteratorTransformer
Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X]
-(8) Scan parquet
+(8) ScanTransformer parquet
Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -127,7 +127,7 @@ Arguments: ((((isnotnull(l_receiptdate#X) AND isnotnull(l_commitdate#X)) AND (l_
Output [2]: [l_orderkey#X, l_suppkey#X]
Input [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X]
-(11) Scan parquet
+(11) ScanTransformer parquet
Output [2]: [l_orderkey#X, l_suppkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -156,7 +156,7 @@ Left keys [1]: [l_orderkey#X]
Right keys [1]: [l_orderkey#X]
Join condition: NOT (l_suppkey#X = l_suppkey#X)
-(18) Scan parquet
+(18) ScanTransformer parquet
Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -203,7 +203,7 @@ Join condition: None
Output [3]: [s_name#X, s_nationkey#X, l_orderkey#X]
Input [5]: [s_suppkey#X, s_name#X, s_nationkey#X, l_orderkey#X, l_suppkey#X]
-(29) Scan parquet
+(29) ScanTransformer parquet
Output [2]: [o_orderkey#X, o_orderstatus#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -245,7 +245,7 @@ Join condition: None
Output [2]: [s_name#X, s_nationkey#X]
Input [4]: [s_name#X, s_nationkey#X, l_orderkey#X, o_orderkey#X]
-(39) Scan parquet
+(39) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 271e9e810ec3..afb719f19903 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
@@ -17,11 +17,11 @@ AdaptiveSparkPlan (40)
+- ^ ProjectExecTransformer (10)
+- ^ BroadcastHashJoinExecTransformer LeftAnti BuildRight (9)
:- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
+ : +- ^ ScanTransformer parquet (1)
+- ^ InputIteratorTransformer (8)
+- BroadcastQueryStage (6)
+- ColumnarBroadcastExchange (5)
- +- ^ Scan parquet (3)
+ +- ^ ScanTransformer parquet (3)
+- == Initial Plan ==
Sort (39)
+- Exchange (38)
@@ -36,7 +36,7 @@ AdaptiveSparkPlan (40)
+- Scan parquet (31)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [3]: [c_custkey#X, c_phone#X, c_acctbal#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -47,7 +47,7 @@ ReadSchema: struct
Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X]
Arguments: ((isnotnull(c_acctbal#X) AND substring(c_phone#X, 1, 2) IN (13,31,23,29,30,18,17)) AND (cast(c_acctbal#X as decimal(16,6)) > Subquery subquery#X, [id=#X]))
-(3) Scan parquet
+(3) ScanTransformer parquet
Output [1]: [o_custkey#X]
Batched: true
Location: InMemoryFileIndex [*]
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 f0fca318bf20..9335e4fbe92c 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
@@ -20,15 +20,15 @@ AdaptiveSparkPlan (54)
: : +- ColumnarBroadcastExchange (5)
: : +- ^ ProjectExecTransformer (3)
: : +- ^ FilterExecTransformer (2)
- : : +- ^ Scan parquet (1)
+ : : +- ^ ScanTransformer parquet (1)
: +- ^ FilterExecTransformer (10)
- : +- ^ Scan parquet (9)
+ : +- ^ ScanTransformer parquet (9)
+- ^ InputIteratorTransformer (20)
+- BroadcastQueryStage (18)
+- ColumnarBroadcastExchange (17)
+- ^ ProjectExecTransformer (15)
+- ^ FilterExecTransformer (14)
- +- ^ Scan parquet (13)
+ +- ^ ScanTransformer parquet (13)
+- == Initial Plan ==
TakeOrderedAndProject (53)
+- HashAggregate (52)
@@ -50,7 +50,7 @@ AdaptiveSparkPlan (54)
+- Scan parquet (44)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [c_custkey#X, c_mktsegment#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -83,7 +83,7 @@ Input [1]: [c_custkey#X]
(8) InputIteratorTransformer
Input [1]: [c_custkey#X]
-(9) Scan parquet
+(9) ScanTransformer parquet
Output [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -103,7 +103,7 @@ Join condition: None
Output [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X]
Input [5]: [c_custkey#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X]
-(13) Scan parquet
+(13) ScanTransformer parquet
Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
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 b8cdf1656c05..9ff9aba0a986 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
@@ -18,13 +18,13 @@ AdaptiveSparkPlan (46)
+- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (12)
:- ^ ProjectExecTransformer (3)
: +- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
+ : +- ^ ScanTransformer parquet (1)
+- ^ InputIteratorTransformer (11)
+- BroadcastQueryStage (9)
+- ColumnarBroadcastExchange (8)
+- ^ ProjectExecTransformer (6)
+- ^ FilterExecTransformer (5)
- +- ^ Scan parquet (4)
+ +- ^ ScanTransformer parquet (4)
+- == Initial Plan ==
Sort (45)
+- Exchange (44)
@@ -42,7 +42,7 @@ AdaptiveSparkPlan (46)
+- Scan parquet (35)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -57,7 +57,7 @@ Arguments: ((isnotnull(o_orderdate#X) AND (o_orderdate#X >= 1993-07-01)) AND (o_
Output [2]: [o_orderkey#X, o_orderpriority#X]
Input [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X]
-(4) Scan parquet
+(4) ScanTransformer parquet
Output [3]: [l_orderkey#X, l_commitdate#X, l_receiptdate#X]
Batched: true
Location: InMemoryFileIndex [*]
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 0ef8d9d4fc1a..df5fa26f0098 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
@@ -28,31 +28,31 @@ AdaptiveSparkPlan (102)
: : : : : +- BroadcastQueryStage (5)
: : : : : +- ColumnarBroadcastExchange (4)
: : : : : +- ^ FilterExecTransformer (2)
- : : : : : +- ^ Scan parquet (1)
+ : : : : : +- ^ ScanTransformer parquet (1)
: : : : +- ^ ProjectExecTransformer (10)
: : : : +- ^ FilterExecTransformer (9)
- : : : : +- ^ Scan parquet (8)
+ : : : : +- ^ ScanTransformer parquet (8)
: : : +- ^ InputIteratorTransformer (19)
: : : +- BroadcastQueryStage (17)
: : : +- ColumnarBroadcastExchange (16)
: : : +- ^ FilterExecTransformer (14)
- : : : +- ^ Scan parquet (13)
+ : : : +- ^ ScanTransformer parquet (13)
: : +- ^ InputIteratorTransformer (28)
: : +- BroadcastQueryStage (26)
: : +- ColumnarBroadcastExchange (25)
: : +- ^ FilterExecTransformer (23)
- : : +- ^ Scan parquet (22)
+ : : +- ^ ScanTransformer parquet (22)
: +- ^ InputIteratorTransformer (37)
: +- BroadcastQueryStage (35)
: +- ColumnarBroadcastExchange (34)
: +- ^ FilterExecTransformer (32)
- : +- ^ Scan parquet (31)
+ : +- ^ ScanTransformer parquet (31)
+- ^ InputIteratorTransformer (47)
+- BroadcastQueryStage (45)
+- ColumnarBroadcastExchange (44)
+- ^ ProjectExecTransformer (42)
+- ^ FilterExecTransformer (41)
- +- ^ Scan parquet (40)
+ +- ^ ScanTransformer parquet (40)
+- == Initial Plan ==
Sort (101)
+- Exchange (100)
@@ -90,7 +90,7 @@ AdaptiveSparkPlan (102)
+- Scan parquet (91)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [c_custkey#X, c_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -119,7 +119,7 @@ Input [2]: [c_custkey#X, c_nationkey#X]
(7) InputIteratorTransformer
Input [2]: [c_custkey#X, c_nationkey#X]
-(8) Scan parquet
+(8) ScanTransformer parquet
Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -143,7 +143,7 @@ Join condition: None
Output [2]: [c_nationkey#X, o_orderkey#X]
Input [4]: [c_custkey#X, c_nationkey#X, o_orderkey#X, o_custkey#X]
-(13) Scan parquet
+(13) ScanTransformer parquet
Output [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -181,7 +181,7 @@ Join condition: None
Output [4]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
Input [6]: [c_nationkey#X, o_orderkey#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
-(22) Scan parquet
+(22) ScanTransformer parquet
Output [2]: [s_suppkey#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -219,7 +219,7 @@ Join condition: None
Output [3]: [l_extendedprice#X, l_discount#X, s_nationkey#X]
Input [6]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, s_suppkey#X, s_nationkey#X]
-(31) Scan parquet
+(31) ScanTransformer parquet
Output [3]: [n_nationkey#X, n_name#X, n_regionkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -257,7 +257,7 @@ Join condition: None
Output [4]: [l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X]
Input [6]: [l_extendedprice#X, l_discount#X, s_nationkey#X, n_nationkey#X, n_name#X, n_regionkey#X]
-(40) Scan parquet
+(40) ScanTransformer parquet
Output [2]: [r_regionkey#X, r_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 a1830678d0de..73d9d734e78a 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
@@ -10,7 +10,7 @@ AdaptiveSparkPlan (20)
+- ^ FlushableHashAggregateExecTransformer (4)
+- ^ ProjectExecTransformer (3)
+- ^ FilterExecTransformer (2)
- +- ^ Scan parquet (1)
+ +- ^ ScanTransformer parquet (1)
+- == Initial Plan ==
HashAggregate (19)
+- Exchange (18)
@@ -20,7 +20,7 @@ AdaptiveSparkPlan (20)
+- Scan parquet (14)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [4]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
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 cac53895b981..88f83c467e85 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
@@ -28,24 +28,24 @@ AdaptiveSparkPlan (95)
: : : : : +- BroadcastQueryStage (5)
: : : : : +- ColumnarBroadcastExchange (4)
: : : : : +- ^ FilterExecTransformer (2)
- : : : : : +- ^ Scan parquet (1)
+ : : : : : +- ^ ScanTransformer parquet (1)
: : : : +- ^ FilterExecTransformer (9)
- : : : : +- ^ Scan parquet (8)
+ : : : : +- ^ ScanTransformer parquet (8)
: : : +- ^ InputIteratorTransformer (18)
: : : +- BroadcastQueryStage (16)
: : : +- ColumnarBroadcastExchange (15)
: : : +- ^ FilterExecTransformer (13)
- : : : +- ^ Scan parquet (12)
+ : : : +- ^ ScanTransformer parquet (12)
: : +- ^ InputIteratorTransformer (27)
: : +- BroadcastQueryStage (25)
: : +- ColumnarBroadcastExchange (24)
: : +- ^ FilterExecTransformer (22)
- : : +- ^ Scan parquet (21)
+ : : +- ^ ScanTransformer parquet (21)
: +- ^ InputIteratorTransformer (36)
: +- BroadcastQueryStage (34)
: +- ColumnarBroadcastExchange (33)
: +- ^ FilterExecTransformer (31)
- : +- ^ Scan parquet (30)
+ : +- ^ ScanTransformer parquet (30)
+- ^ InputIteratorTransformer (42)
+- BroadcastQueryStage (40)
+- ReusedExchange (39)
@@ -84,7 +84,7 @@ AdaptiveSparkPlan (95)
+- Scan parquet (85)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [s_suppkey#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -113,7 +113,7 @@ Input [2]: [s_suppkey#X, s_nationkey#X]
(7) InputIteratorTransformer
Input [2]: [s_suppkey#X, s_nationkey#X]
-(8) Scan parquet
+(8) ScanTransformer parquet
Output [5]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -133,7 +133,7 @@ Join condition: None
Output [5]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Input [7]: [s_suppkey#X, s_nationkey#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
-(12) Scan parquet
+(12) ScanTransformer parquet
Output [2]: [o_orderkey#X, o_custkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -171,7 +171,7 @@ Join condition: None
Output [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X]
Input [7]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_orderkey#X, o_custkey#X]
-(21) Scan parquet
+(21) ScanTransformer parquet
Output [2]: [c_custkey#X, c_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -209,7 +209,7 @@ Join condition: None
Output [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X]
Input [7]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X, c_custkey#X, c_nationkey#X]
-(30) Scan parquet
+(30) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 33544f0796a5..c389ab06a45f 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
@@ -34,40 +34,40 @@ AdaptiveSparkPlan (131)
: : : : : : : +- ColumnarBroadcastExchange (5)
: : : : : : : +- ^ ProjectExecTransformer (3)
: : : : : : : +- ^ FilterExecTransformer (2)
- : : : : : : : +- ^ Scan parquet (1)
+ : : : : : : : +- ^ ScanTransformer parquet (1)
: : : : : : +- ^ FilterExecTransformer (10)
- : : : : : : +- ^ Scan parquet (9)
+ : : : : : : +- ^ ScanTransformer parquet (9)
: : : : : +- ^ InputIteratorTransformer (19)
: : : : : +- BroadcastQueryStage (17)
: : : : : +- ColumnarBroadcastExchange (16)
: : : : : +- ^ FilterExecTransformer (14)
- : : : : : +- ^ Scan parquet (13)
+ : : : : : +- ^ ScanTransformer parquet (13)
: : : : +- ^ InputIteratorTransformer (28)
: : : : +- BroadcastQueryStage (26)
: : : : +- ColumnarBroadcastExchange (25)
: : : : +- ^ FilterExecTransformer (23)
- : : : : +- ^ Scan parquet (22)
+ : : : : +- ^ ScanTransformer parquet (22)
: : : +- ^ InputIteratorTransformer (37)
: : : +- BroadcastQueryStage (35)
: : : +- ColumnarBroadcastExchange (34)
: : : +- ^ FilterExecTransformer (32)
- : : : +- ^ Scan parquet (31)
+ : : : +- ^ ScanTransformer parquet (31)
: : +- ^ InputIteratorTransformer (46)
: : +- BroadcastQueryStage (44)
: : +- ColumnarBroadcastExchange (43)
: : +- ^ FilterExecTransformer (41)
- : : +- ^ Scan parquet (40)
+ : : +- ^ ScanTransformer parquet (40)
: +- ^ InputIteratorTransformer (55)
: +- BroadcastQueryStage (53)
: +- ColumnarBroadcastExchange (52)
: +- ^ FilterExecTransformer (50)
- : +- ^ Scan parquet (49)
+ : +- ^ ScanTransformer parquet (49)
+- ^ InputIteratorTransformer (65)
+- BroadcastQueryStage (63)
+- ColumnarBroadcastExchange (62)
+- ^ ProjectExecTransformer (60)
+- ^ FilterExecTransformer (59)
- +- ^ Scan parquet (58)
+ +- ^ ScanTransformer parquet (58)
+- == Initial Plan ==
Sort (130)
+- Exchange (129)
@@ -115,7 +115,7 @@ AdaptiveSparkPlan (131)
+- Scan parquet (120)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [p_partkey#X, p_type#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -148,7 +148,7 @@ Input [1]: [p_partkey#X]
(8) InputIteratorTransformer
Input [1]: [p_partkey#X]
-(9) Scan parquet
+(9) ScanTransformer parquet
Output [5]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -168,7 +168,7 @@ Join condition: None
Output [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
Input [6]: [p_partkey#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
-(13) Scan parquet
+(13) ScanTransformer parquet
Output [2]: [s_suppkey#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -206,7 +206,7 @@ Join condition: None
Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X]
Input [6]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, s_suppkey#X, s_nationkey#X]
-(22) Scan parquet
+(22) ScanTransformer parquet
Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -244,7 +244,7 @@ Join condition: None
Output [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X]
Input [7]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderkey#X, o_custkey#X, o_orderdate#X]
-(31) Scan parquet
+(31) ScanTransformer parquet
Output [2]: [c_custkey#X, c_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -282,7 +282,7 @@ Join condition: None
Output [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X]
Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X, c_custkey#X, c_nationkey#X]
-(40) Scan parquet
+(40) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_regionkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -320,7 +320,7 @@ Join condition: None
Output [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X]
Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X, n_nationkey#X, n_regionkey#X]
-(49) Scan parquet
+(49) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -358,7 +358,7 @@ Join condition: None
Output [5]: [l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X]
Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X, n_nationkey#X, n_name#X]
-(58) Scan parquet
+(58) ScanTransformer parquet
Output [2]: [r_regionkey#X, r_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 7112bb021b0c..830259ba82bd 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
@@ -29,29 +29,29 @@ AdaptiveSparkPlan (100)
: : : : : +- ColumnarBroadcastExchange (5)
: : : : : +- ^ ProjectExecTransformer (3)
: : : : : +- ^ FilterExecTransformer (2)
- : : : : : +- ^ Scan parquet (1)
+ : : : : : +- ^ ScanTransformer parquet (1)
: : : : +- ^ FilterExecTransformer (10)
- : : : : +- ^ Scan parquet (9)
+ : : : : +- ^ ScanTransformer parquet (9)
: : : +- ^ InputIteratorTransformer (19)
: : : +- BroadcastQueryStage (17)
: : : +- ColumnarBroadcastExchange (16)
: : : +- ^ FilterExecTransformer (14)
- : : : +- ^ Scan parquet (13)
+ : : : +- ^ ScanTransformer parquet (13)
: : +- ^ InputIteratorTransformer (28)
: : +- BroadcastQueryStage (26)
: : +- ColumnarBroadcastExchange (25)
: : +- ^ FilterExecTransformer (23)
- : : +- ^ Scan parquet (22)
+ : : +- ^ ScanTransformer parquet (22)
: +- ^ InputIteratorTransformer (37)
: +- BroadcastQueryStage (35)
: +- ColumnarBroadcastExchange (34)
: +- ^ FilterExecTransformer (32)
- : +- ^ Scan parquet (31)
+ : +- ^ ScanTransformer parquet (31)
+- ^ InputIteratorTransformer (46)
+- BroadcastQueryStage (44)
+- ColumnarBroadcastExchange (43)
+- ^ FilterExecTransformer (41)
- +- ^ Scan parquet (40)
+ +- ^ ScanTransformer parquet (40)
+- == Initial Plan ==
Sort (99)
+- Exchange (98)
@@ -88,7 +88,7 @@ AdaptiveSparkPlan (100)
+- Scan parquet (90)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [p_partkey#X, p_name#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -121,7 +121,7 @@ Input [1]: [p_partkey#X]
(8) InputIteratorTransformer
Input [1]: [p_partkey#X]
-(9) Scan parquet
+(9) ScanTransformer parquet
Output [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -141,7 +141,7 @@ Join condition: None
Output [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X]
Input [7]: [p_partkey#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X]
-(13) Scan parquet
+(13) ScanTransformer parquet
Output [2]: [s_suppkey#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -179,7 +179,7 @@ Join condition: None
Output [7]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X]
Input [8]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_suppkey#X, s_nationkey#X]
-(22) Scan parquet
+(22) ScanTransformer parquet
Output [3]: [ps_partkey#X, ps_suppkey#X, ps_supplycost#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -217,7 +217,7 @@ Join condition: None
Output [6]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X]
Input [10]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_partkey#X, ps_suppkey#X, ps_supplycost#X]
-(31) Scan parquet
+(31) ScanTransformer parquet
Output [2]: [o_orderkey#X, o_orderdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -255,7 +255,7 @@ Join condition: None
Output [6]: [l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X]
Input [8]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderkey#X, o_orderdate#X]
-(40) Scan parquet
+(40) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 0851fa2447ea..ca3b50ab05c5 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
@@ -16,7 +16,7 @@ AdaptiveSparkPlan (30)
+- ^ FlushableHashAggregateExecTransformer (4)
+- ^ ProjectExecTransformer (3)
+- ^ FilterExecTransformer (2)
- +- ^ Scan parquet (1)
+ +- ^ ScanTransformer parquet (1)
+- == Initial Plan ==
Sort (29)
+- Exchange (28)
@@ -28,7 +28,7 @@ AdaptiveSparkPlan (30)
+- Scan parquet (22)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [7]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_tax#X, l_returnflag#X, l_linestatus#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
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 42cf8e1b7f58..c15913c284f5 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
@@ -18,24 +18,24 @@ AdaptiveSparkPlan (68)
: :- ^ ProjectExecTransformer (12)
: : +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (11)
: : :- ^ FilterExecTransformer (2)
- : : : +- ^ Scan parquet (1)
+ : : : +- ^ ScanTransformer parquet (1)
: : +- ^ InputIteratorTransformer (10)
: : +- BroadcastQueryStage (8), Statistics(X)
: : +- ColumnarBroadcastExchange (7)
: : +- ^ ProjectExecTransformer (5)
: : +- ^ FilterExecTransformer (4)
- : : +- ^ Scan parquet (3)
+ : : +- ^ ScanTransformer parquet (3)
: +- ^ InputIteratorTransformer (20)
: +- BroadcastQueryStage (18), Statistics(X)
: +- ColumnarBroadcastExchange (17)
: +- ^ ProjectExecTransformer (15)
: +- ^ FilterExecTransformer (14)
- : +- ^ Scan parquet (13)
+ : +- ^ ScanTransformer parquet (13)
+- ^ InputIteratorTransformer (29)
+- BroadcastQueryStage (27), Statistics(X)
+- ColumnarBroadcastExchange (26)
+- ^ FilterExecTransformer (24)
- +- ^ Scan parquet (23)
+ +- ^ ScanTransformer parquet (23)
+- == Initial Plan ==
TakeOrderedAndProject (67)
+- HashAggregate (66)
@@ -62,7 +62,7 @@ AdaptiveSparkPlan (68)
+- Scan parquet (59)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -73,7 +73,7 @@ ReadSchema: struct
-(2) Scan parquet
+(2) ScanTransformer parquet
Output [3]: [o_orderkey#X, o_custkey#X, o_comment#X]
Batched: true
Location: InMemoryFileIndex [*]
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 8956ddaa2184..b06cd55156e1 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
@@ -13,12 +13,12 @@ AdaptiveSparkPlan (35)
+- ^ BroadcastHashJoinExecTransformer Inner BuildRight (11)
:- ^ ProjectExecTransformer (3)
: +- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
+ : +- ^ ScanTransformer parquet (1)
+- ^ InputIteratorTransformer (10)
+- BroadcastQueryStage (8), Statistics(X)
+- ColumnarBroadcastExchange (7)
+- ^ FilterExecTransformer (5)
- +- ^ Scan parquet (4)
+ +- ^ ScanTransformer parquet (4)
+- == Initial Plan ==
HashAggregate (34)
+- Exchange (33)
@@ -33,7 +33,7 @@ AdaptiveSparkPlan (35)
+- Scan parquet (27)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -48,7 +48,7 @@ Arguments: (((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1995-09-01)) AND (l_s
Output [3]: [l_partkey#X, l_extendedprice#X, l_discount#X]
Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
-(4) Scan parquet
+(4) ScanTransformer parquet
Output [2]: [p_partkey#X, p_type#X]
Batched: true
Location: InMemoryFileIndex [*]
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 d7603e49ffb6..52a87f43535d 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
@@ -12,7 +12,7 @@ AdaptiveSparkPlan (43)
: +- BroadcastQueryStage (5), Statistics(X)
: +- ColumnarBroadcastExchange (4)
: +- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
+ : +- ^ ScanTransformer parquet (1)
+- ^ FilterExecTransformer (20)
+- ^ RegularHashAggregateExecTransformer (19)
+- ^ InputIteratorTransformer (18)
@@ -23,7 +23,7 @@ AdaptiveSparkPlan (43)
+- ^ FlushableHashAggregateExecTransformer (11)
+- ^ ProjectExecTransformer (10)
+- ^ FilterExecTransformer (9)
- +- ^ Scan parquet (8)
+ +- ^ ScanTransformer parquet (8)
+- == Initial Plan ==
Sort (42)
+- Exchange (41)
@@ -41,7 +41,7 @@ AdaptiveSparkPlan (43)
+- Scan parquet (32)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -70,7 +70,7 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X]
(7) InputIteratorTransformer
Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X]
-(8) Scan parquet
+(8) ScanTransformer parquet
Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -237,34 +237,33 @@ Arguments: isFinalPlan=true
===== Subqueries =====
Subquery:1 Hosting operator id = 20 Hosting Expression = Subquery subquery#X, [id=#X]
-AdaptiveSparkPlan (69)
+AdaptiveSparkPlan (68)
+- == Final Plan ==
- VeloxColumnarToRow (60)
- +- ^ RegularHashAggregateExecTransformer (58)
- +- ^ RegularHashAggregateExecTransformer (57)
- +- ^ ProjectExecTransformer (56)
- +- ^ RegularHashAggregateExecTransformer (55)
- +- ^ InputIteratorTransformer (54)
- +- ShuffleQueryStage (52), Statistics(X)
- +- ColumnarExchange (51)
- +- VeloxResizeBatches (50)
- +- ^ ProjectExecTransformer (48)
- +- ^ FlushableHashAggregateExecTransformer (47)
- +- ^ ProjectExecTransformer (46)
- +- ^ FilterExecTransformer (45)
- +- ^ Scan parquet (44)
+ VeloxColumnarToRow (59)
+ +- ^ RegularHashAggregateExecTransformer (57)
+ +- ^ ProjectExecTransformer (56)
+ +- ^ RegularHashAggregateExecTransformer (55)
+ +- ^ InputIteratorTransformer (54)
+ +- ShuffleQueryStage (52), Statistics(X)
+ +- ColumnarExchange (51)
+ +- VeloxResizeBatches (50)
+ +- ^ ProjectExecTransformer (48)
+ +- ^ FlushableHashAggregateExecTransformer (47)
+ +- ^ ProjectExecTransformer (46)
+ +- ^ FilterExecTransformer (45)
+ +- ^ ScanTransformer parquet (44)
+- == Initial Plan ==
- HashAggregate (68)
- +- HashAggregate (67)
- +- HashAggregate (66)
- +- Exchange (65)
- +- HashAggregate (64)
- +- Project (63)
- +- Filter (62)
- +- Scan parquet (61)
+ HashAggregate (67)
+ +- HashAggregate (66)
+ +- HashAggregate (65)
+ +- Exchange (64)
+ +- HashAggregate (63)
+ +- Project (62)
+ +- Filter (61)
+ +- Scan parquet (60)
-(44) Scan parquet
+(44) ScanTransformer parquet
Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -326,71 +325,64 @@ Input [2]: [l_suppkey#X, sum(CheckOverflow((promote_precision(cast(l_extendedpri
(57) RegularHashAggregateExecTransformer
Input [1]: [total_revenue#X]
Keys: []
-Functions [1]: [partial_max(total_revenue#X)]
-Aggregate Attributes [1]: [max#X]
-Results [1]: [max#X]
-
-(58) RegularHashAggregateExecTransformer
-Input [1]: [max#X]
-Keys: []
Functions [1]: [max(total_revenue#X)]
Aggregate Attributes [1]: [max(total_revenue#X)#X]
Results [1]: [max(total_revenue#X)#X AS max(total_revenue)#X]
-(59) WholeStageCodegenTransformer (X)
+(58) WholeStageCodegenTransformer (X)
Input [1]: [max(total_revenue)#X]
Arguments: false
-(60) VeloxColumnarToRow
+(59) VeloxColumnarToRow
Input [1]: [max(total_revenue)#X]
-(61) Scan parquet
+(60) Scan parquet
Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1996-01-01), LessThan(l_shipdate,1996-04-01)]
ReadSchema: struct
-(62) Filter
+(61) Filter
Input [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Condition : ((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1996-01-01)) AND (l_shipdate#X < 1996-04-01))
-(63) Project
+(62) Project
Output [3]: [l_suppkey#X, l_extendedprice#X, l_discount#X]
Input [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
-(64) HashAggregate
+(63) HashAggregate
Input [3]: [l_suppkey#X, l_extendedprice#X, l_discount#X]
Keys [1]: [l_suppkey#X]
Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))]
Aggregate Attributes [2]: [sum#X, isEmpty#X]
Results [3]: [l_suppkey#X, sum#X, isEmpty#X]
-(65) Exchange
+(64) Exchange
Input [3]: [l_suppkey#X, sum#X, isEmpty#X]
Arguments: hashpartitioning(l_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(66) HashAggregate
+(65) HashAggregate
Input [3]: [l_suppkey#X, sum#X, isEmpty#X]
Keys [1]: [l_suppkey#X]
Functions [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))]
Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X]
Results [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X AS total_revenue#X]
-(67) HashAggregate
+(66) HashAggregate
Input [1]: [total_revenue#X]
Keys: []
Functions [1]: [partial_max(total_revenue#X)]
Aggregate Attributes [1]: [max#X]
Results [1]: [max#X]
-(68) HashAggregate
+(67) HashAggregate
Input [1]: [max#X]
Keys: []
Functions [1]: [max(total_revenue#X)]
Aggregate Attributes [1]: [max(total_revenue#X)#X]
Results [1]: [max(total_revenue#X)#X AS max(total_revenue)#X]
-(69) AdaptiveSparkPlan
+(68) AdaptiveSparkPlan
Output [1]: [max(total_revenue)#X]
Arguments: isFinalPlan=true
\ No newline at end of file
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 394bbd2469c2..8cc2b417eb3f 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
@@ -24,12 +24,12 @@ AdaptiveSparkPlan (59)
+- ^ ProjectExecTransformer (11)
+- ^ BroadcastHashJoinExecTransformer Inner BuildRight (10)
:- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
+ : +- ^ ScanTransformer parquet (1)
+- ^ InputIteratorTransformer (9)
+- BroadcastQueryStage (7), Statistics(X)
+- ColumnarBroadcastExchange (6)
+- ^ FilterExecTransformer (4)
- +- ^ Scan parquet (3)
+ +- ^ ScanTransformer parquet (3)
+- == Initial Plan ==
Sort (58)
+- Exchange (57)
@@ -53,7 +53,7 @@ AdaptiveSparkPlan (59)
+- Scan parquet (46)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [ps_partkey#X, ps_suppkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -64,7 +64,7 @@ ReadSchema: struct
Input [2]: [ps_partkey#X, ps_suppkey#X]
Arguments: isnotnull(ps_partkey#X)
-(3) Scan parquet
+(3) ScanTransformer parquet
Output [4]: [p_partkey#X, p_brand#X, p_type#X, p_size#X]
Batched: true
Location: InMemoryFileIndex [*]
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 6c279f2d124d..8fd71a81a264 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
@@ -18,10 +18,10 @@ AdaptiveSparkPlan (88)
: : +- BroadcastQueryStage (5), Statistics(X)
: : +- ColumnarBroadcastExchange (4)
: : +- ^ FilterExecTransformer (2)
- : : +- ^ Scan parquet (1)
+ : : +- ^ ScanTransformer parquet (1)
: +- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (27)
: :- ^ FilterExecTransformer (9)
- : : +- ^ Scan parquet (8)
+ : : +- ^ ScanTransformer parquet (8)
: +- ^ InputIteratorTransformer (26)
: +- BroadcastQueryStage (24), Statistics(X)
: +- ColumnarBroadcastExchange (23)
@@ -34,13 +34,13 @@ AdaptiveSparkPlan (88)
: +- VeloxResizeBatches (14)
: +- ^ ProjectExecTransformer (12)
: +- ^ FlushableHashAggregateExecTransformer (11)
- : +- ^ Scan parquet (10)
+ : +- ^ ScanTransformer parquet (10)
+- ^ InputIteratorTransformer (41)
+- BroadcastQueryStage (39), Statistics(X)
+- ColumnarBroadcastExchange (38)
+- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (36)
:- ^ FilterExecTransformer (31)
- : +- ^ Scan parquet (30)
+ : +- ^ ScanTransformer parquet (30)
+- ^ InputIteratorTransformer (35)
+- BroadcastQueryStage (33), Statistics(X)
+- ReusedExchange (32)
@@ -79,7 +79,7 @@ AdaptiveSparkPlan (88)
+- Scan parquet (73)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [c_custkey#X, c_name#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -108,7 +108,7 @@ Input [2]: [c_custkey#X, c_name#X]
(7) InputIteratorTransformer
Input [2]: [c_custkey#X, c_name#X]
-(8) Scan parquet
+(8) ScanTransformer parquet
Output [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -119,7 +119,7 @@ ReadSchema: struct
Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
Arguments: ((isnotnull(ps_availqty#X) AND isnotnull(ps_partkey#X)) AND isnotnull(ps_suppkey#X))
-(13) Scan parquet
+(13) ScanTransformer parquet
Output [2]: [p_partkey#X, p_name#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -206,7 +206,7 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
(26) InputIteratorTransformer
Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
-(27) Scan parquet
+(27) ScanTransformer parquet
Output [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -323,7 +323,7 @@ Join condition: None
Output [3]: [s_name#X, s_address#X, s_nationkey#X]
Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X]
-(55) Scan parquet
+(55) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 eae4758613ee..505dc59dc4cb 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
@@ -19,34 +19,34 @@ AdaptiveSparkPlan (92)
: : : +- BroadcastQueryStage (5), Statistics(X)
: : : +- ColumnarBroadcastExchange (4)
: : : +- ^ FilterExecTransformer (2)
- : : : +- ^ Scan parquet (1)
+ : : : +- ^ ScanTransformer parquet (1)
: : +- ^ BroadcastHashJoinExecTransformer LeftAnti BuildRight (26)
: : :- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (17)
: : : :- ^ ProjectExecTransformer (10)
: : : : +- ^ FilterExecTransformer (9)
- : : : : +- ^ Scan parquet (8)
+ : : : : +- ^ ScanTransformer parquet (8)
: : : +- ^ InputIteratorTransformer (16)
: : : +- BroadcastQueryStage (14), Statistics(X)
: : : +- ColumnarBroadcastExchange (13)
- : : : +- ^ Scan parquet (11)
+ : : : +- ^ ScanTransformer parquet (11)
: : +- ^ InputIteratorTransformer (25)
: : +- BroadcastQueryStage (23), Statistics(X)
: : +- ColumnarBroadcastExchange (22)
: : +- ^ ProjectExecTransformer (20)
: : +- ^ FilterExecTransformer (19)
- : : +- ^ Scan parquet (18)
+ : : +- ^ ScanTransformer parquet (18)
: +- ^ InputIteratorTransformer (36)
: +- BroadcastQueryStage (34), Statistics(X)
: +- ColumnarBroadcastExchange (33)
: +- ^ ProjectExecTransformer (31)
: +- ^ FilterExecTransformer (30)
- : +- ^ Scan parquet (29)
+ : +- ^ ScanTransformer parquet (29)
+- ^ InputIteratorTransformer (46)
+- BroadcastQueryStage (44), Statistics(X)
+- ColumnarBroadcastExchange (43)
+- ^ ProjectExecTransformer (41)
+- ^ FilterExecTransformer (40)
- +- ^ Scan parquet (39)
+ +- ^ ScanTransformer parquet (39)
+- == Initial Plan ==
TakeOrderedAndProject (91)
+- HashAggregate (90)
@@ -82,7 +82,7 @@ AdaptiveSparkPlan (92)
+- Scan parquet (82)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [3]: [s_suppkey#X, s_name#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -111,7 +111,7 @@ Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X]
(7) InputIteratorTransformer
Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X]
-(8) Scan parquet
+(8) ScanTransformer parquet
Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -126,7 +126,7 @@ Arguments: ((((isnotnull(l_receiptdate#X) AND isnotnull(l_commitdate#X)) AND (l_
Output [2]: [l_orderkey#X, l_suppkey#X]
Input [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X]
-(11) Scan parquet
+(11) ScanTransformer parquet
Output [2]: [l_orderkey#X, l_suppkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -155,7 +155,7 @@ Left keys [1]: [l_orderkey#X]
Right keys [1]: [l_orderkey#X]
Join condition: NOT (l_suppkey#X = l_suppkey#X)
-(18) Scan parquet
+(18) ScanTransformer parquet
Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -202,7 +202,7 @@ Join condition: None
Output [3]: [s_name#X, s_nationkey#X, l_orderkey#X]
Input [5]: [s_suppkey#X, s_name#X, s_nationkey#X, l_orderkey#X, l_suppkey#X]
-(29) Scan parquet
+(29) ScanTransformer parquet
Output [2]: [o_orderkey#X, o_orderstatus#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -244,7 +244,7 @@ Join condition: None
Output [2]: [s_name#X, s_nationkey#X]
Input [4]: [s_name#X, s_nationkey#X, l_orderkey#X, o_orderkey#X]
-(39) Scan parquet
+(39) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 a3d3f0a08eeb..9113355acbdd 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
@@ -17,11 +17,11 @@ AdaptiveSparkPlan (40)
+- ^ ProjectExecTransformer (10)
+- ^ BroadcastHashJoinExecTransformer LeftAnti BuildRight (9)
:- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
+ : +- ^ ScanTransformer parquet (1)
+- ^ InputIteratorTransformer (8)
+- BroadcastQueryStage (6), Statistics(X)
+- ColumnarBroadcastExchange (5)
- +- ^ Scan parquet (3)
+ +- ^ ScanTransformer parquet (3)
+- == Initial Plan ==
Sort (39)
+- Exchange (38)
@@ -36,7 +36,7 @@ AdaptiveSparkPlan (40)
+- Scan parquet (31)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [3]: [c_custkey#X, c_phone#X, c_acctbal#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -47,7 +47,7 @@ ReadSchema: struct
Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X]
Arguments: ((isnotnull(c_acctbal#X) AND substring(c_phone#X, 1, 2) IN (13,31,23,29,30,18,17)) AND (cast(c_acctbal#X as decimal(16,6)) > Subquery subquery#X, [id=#X]))
-(3) Scan parquet
+(3) ScanTransformer parquet
Output [1]: [o_custkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -227,7 +227,7 @@ AdaptiveSparkPlan (60)
+- ^ FlushableHashAggregateExecTransformer (44)
+- ^ ProjectExecTransformer (43)
+- ^ FilterExecTransformer (42)
- +- ^ Scan parquet (41)
+ +- ^ ScanTransformer parquet (41)
+- == Initial Plan ==
HashAggregate (59)
+- Exchange (58)
@@ -237,7 +237,7 @@ AdaptiveSparkPlan (60)
+- Scan parquet (54)
-(41) Scan parquet
+(41) ScanTransformer parquet
Output [2]: [c_phone#X, c_acctbal#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -344,7 +344,7 @@ AdaptiveSparkPlan (60)
+- ^ FlushableHashAggregateExecTransformer (44)
+- ^ ProjectExecTransformer (43)
+- ^ FilterExecTransformer (42)
- +- ^ Scan parquet (41)
+ +- ^ ScanTransformer parquet (41)
+- == Initial Plan ==
HashAggregate (59)
+- Exchange (58)
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 e7fa21b6f1f1..befea454305b 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
@@ -20,15 +20,15 @@ AdaptiveSparkPlan (54)
: : +- ColumnarBroadcastExchange (5)
: : +- ^ ProjectExecTransformer (3)
: : +- ^ FilterExecTransformer (2)
- : : +- ^ Scan parquet (1)
+ : : +- ^ ScanTransformer parquet (1)
: +- ^ FilterExecTransformer (10)
- : +- ^ Scan parquet (9)
+ : +- ^ ScanTransformer parquet (9)
+- ^ InputIteratorTransformer (20)
+- BroadcastQueryStage (18), Statistics(X)
+- ColumnarBroadcastExchange (17)
+- ^ ProjectExecTransformer (15)
+- ^ FilterExecTransformer (14)
- +- ^ Scan parquet (13)
+ +- ^ ScanTransformer parquet (13)
+- == Initial Plan ==
TakeOrderedAndProject (53)
+- HashAggregate (52)
@@ -50,7 +50,7 @@ AdaptiveSparkPlan (54)
+- Scan parquet (44)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [c_custkey#X, c_mktsegment#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -83,7 +83,7 @@ Input [1]: [c_custkey#X]
(8) InputIteratorTransformer
Input [1]: [c_custkey#X]
-(9) Scan parquet
+(9) ScanTransformer parquet
Output [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -103,7 +103,7 @@ Join condition: None
Output [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X]
Input [5]: [c_custkey#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X]
-(13) Scan parquet
+(13) ScanTransformer parquet
Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
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 a1eaaca4d448..52c99692cf4d 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
@@ -18,13 +18,13 @@ AdaptiveSparkPlan (46)
+- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (12)
:- ^ ProjectExecTransformer (3)
: +- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
+ : +- ^ ScanTransformer parquet (1)
+- ^ InputIteratorTransformer (11)
+- BroadcastQueryStage (9), Statistics(X)
+- ColumnarBroadcastExchange (8)
+- ^ ProjectExecTransformer (6)
+- ^ FilterExecTransformer (5)
- +- ^ Scan parquet (4)
+ +- ^ ScanTransformer parquet (4)
+- == Initial Plan ==
Sort (45)
+- Exchange (44)
@@ -42,7 +42,7 @@ AdaptiveSparkPlan (46)
+- Scan parquet (35)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -57,7 +57,7 @@ Arguments: ((isnotnull(o_orderdate#X) AND (o_orderdate#X >= 1993-07-01)) AND (o_
Output [2]: [o_orderkey#X, o_orderpriority#X]
Input [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X]
-(4) Scan parquet
+(4) ScanTransformer parquet
Output [3]: [l_orderkey#X, l_commitdate#X, l_receiptdate#X]
Batched: true
Location: InMemoryFileIndex [*]
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 638a442362e3..a076ea86feb5 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
@@ -28,31 +28,31 @@ AdaptiveSparkPlan (102)
: : : : : +- BroadcastQueryStage (5), Statistics(X)
: : : : : +- ColumnarBroadcastExchange (4)
: : : : : +- ^ FilterExecTransformer (2)
- : : : : : +- ^ Scan parquet (1)
+ : : : : : +- ^ ScanTransformer parquet (1)
: : : : +- ^ ProjectExecTransformer (10)
: : : : +- ^ FilterExecTransformer (9)
- : : : : +- ^ Scan parquet (8)
+ : : : : +- ^ ScanTransformer parquet (8)
: : : +- ^ InputIteratorTransformer (19)
: : : +- BroadcastQueryStage (17), Statistics(X)
: : : +- ColumnarBroadcastExchange (16)
: : : +- ^ FilterExecTransformer (14)
- : : : +- ^ Scan parquet (13)
+ : : : +- ^ ScanTransformer parquet (13)
: : +- ^ InputIteratorTransformer (28)
: : +- BroadcastQueryStage (26), Statistics(X)
: : +- ColumnarBroadcastExchange (25)
: : +- ^ FilterExecTransformer (23)
- : : +- ^ Scan parquet (22)
+ : : +- ^ ScanTransformer parquet (22)
: +- ^ InputIteratorTransformer (37)
: +- BroadcastQueryStage (35), Statistics(X)
: +- ColumnarBroadcastExchange (34)
: +- ^ FilterExecTransformer (32)
- : +- ^ Scan parquet (31)
+ : +- ^ ScanTransformer parquet (31)
+- ^ InputIteratorTransformer (47)
+- BroadcastQueryStage (45), Statistics(X)
+- ColumnarBroadcastExchange (44)
+- ^ ProjectExecTransformer (42)
+- ^ FilterExecTransformer (41)
- +- ^ Scan parquet (40)
+ +- ^ ScanTransformer parquet (40)
+- == Initial Plan ==
Sort (101)
+- Exchange (100)
@@ -90,7 +90,7 @@ AdaptiveSparkPlan (102)
+- Scan parquet (91)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [c_custkey#X, c_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -119,7 +119,7 @@ Input [2]: [c_custkey#X, c_nationkey#X]
(7) InputIteratorTransformer
Input [2]: [c_custkey#X, c_nationkey#X]
-(8) Scan parquet
+(8) ScanTransformer parquet
Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -143,7 +143,7 @@ Join condition: None
Output [2]: [c_nationkey#X, o_orderkey#X]
Input [4]: [c_custkey#X, c_nationkey#X, o_orderkey#X, o_custkey#X]
-(13) Scan parquet
+(13) ScanTransformer parquet
Output [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -181,7 +181,7 @@ Join condition: None
Output [4]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
Input [6]: [c_nationkey#X, o_orderkey#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
-(22) Scan parquet
+(22) ScanTransformer parquet
Output [2]: [s_suppkey#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -219,7 +219,7 @@ Join condition: None
Output [3]: [l_extendedprice#X, l_discount#X, s_nationkey#X]
Input [6]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, s_suppkey#X, s_nationkey#X]
-(31) Scan parquet
+(31) ScanTransformer parquet
Output [3]: [n_nationkey#X, n_name#X, n_regionkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -257,7 +257,7 @@ Join condition: None
Output [4]: [l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X]
Input [6]: [l_extendedprice#X, l_discount#X, s_nationkey#X, n_nationkey#X, n_name#X, n_regionkey#X]
-(40) Scan parquet
+(40) ScanTransformer parquet
Output [2]: [r_regionkey#X, r_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 5df63675caee..d5569132a41f 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
@@ -10,7 +10,7 @@ AdaptiveSparkPlan (20)
+- ^ FlushableHashAggregateExecTransformer (4)
+- ^ ProjectExecTransformer (3)
+- ^ FilterExecTransformer (2)
- +- ^ Scan parquet (1)
+ +- ^ ScanTransformer parquet (1)
+- == Initial Plan ==
HashAggregate (19)
+- Exchange (18)
@@ -20,7 +20,7 @@ AdaptiveSparkPlan (20)
+- Scan parquet (14)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [4]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
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 86db01e58b53..5fdae3d7334b 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
@@ -28,24 +28,24 @@ AdaptiveSparkPlan (95)
: : : : : +- BroadcastQueryStage (5), Statistics(X)
: : : : : +- ColumnarBroadcastExchange (4)
: : : : : +- ^ FilterExecTransformer (2)
- : : : : : +- ^ Scan parquet (1)
+ : : : : : +- ^ ScanTransformer parquet (1)
: : : : +- ^ FilterExecTransformer (9)
- : : : : +- ^ Scan parquet (8)
+ : : : : +- ^ ScanTransformer parquet (8)
: : : +- ^ InputIteratorTransformer (18)
: : : +- BroadcastQueryStage (16), Statistics(X)
: : : +- ColumnarBroadcastExchange (15)
: : : +- ^ FilterExecTransformer (13)
- : : : +- ^ Scan parquet (12)
+ : : : +- ^ ScanTransformer parquet (12)
: : +- ^ InputIteratorTransformer (27)
: : +- BroadcastQueryStage (25), Statistics(X)
: : +- ColumnarBroadcastExchange (24)
: : +- ^ FilterExecTransformer (22)
- : : +- ^ Scan parquet (21)
+ : : +- ^ ScanTransformer parquet (21)
: +- ^ InputIteratorTransformer (36)
: +- BroadcastQueryStage (34), Statistics(X)
: +- ColumnarBroadcastExchange (33)
: +- ^ FilterExecTransformer (31)
- : +- ^ Scan parquet (30)
+ : +- ^ ScanTransformer parquet (30)
+- ^ InputIteratorTransformer (42)
+- BroadcastQueryStage (40), Statistics(X)
+- ReusedExchange (39)
@@ -84,7 +84,7 @@ AdaptiveSparkPlan (95)
+- Scan parquet (85)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [s_suppkey#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -113,7 +113,7 @@ Input [2]: [s_suppkey#X, s_nationkey#X]
(7) InputIteratorTransformer
Input [2]: [s_suppkey#X, s_nationkey#X]
-(8) Scan parquet
+(8) ScanTransformer parquet
Output [5]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -133,7 +133,7 @@ Join condition: None
Output [5]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Input [7]: [s_suppkey#X, s_nationkey#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
-(12) Scan parquet
+(12) ScanTransformer parquet
Output [2]: [o_orderkey#X, o_custkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -171,7 +171,7 @@ Join condition: None
Output [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X]
Input [7]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_orderkey#X, o_custkey#X]
-(21) Scan parquet
+(21) ScanTransformer parquet
Output [2]: [c_custkey#X, c_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -209,7 +209,7 @@ Join condition: None
Output [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X]
Input [7]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X, c_custkey#X, c_nationkey#X]
-(30) Scan parquet
+(30) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 ccc7941882b7..0b00093c0316 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
@@ -34,40 +34,40 @@ AdaptiveSparkPlan (131)
: : : : : : : +- ColumnarBroadcastExchange (5)
: : : : : : : +- ^ ProjectExecTransformer (3)
: : : : : : : +- ^ FilterExecTransformer (2)
- : : : : : : : +- ^ Scan parquet (1)
+ : : : : : : : +- ^ ScanTransformer parquet (1)
: : : : : : +- ^ FilterExecTransformer (10)
- : : : : : : +- ^ Scan parquet (9)
+ : : : : : : +- ^ ScanTransformer parquet (9)
: : : : : +- ^ InputIteratorTransformer (19)
: : : : : +- BroadcastQueryStage (17), Statistics(X)
: : : : : +- ColumnarBroadcastExchange (16)
: : : : : +- ^ FilterExecTransformer (14)
- : : : : : +- ^ Scan parquet (13)
+ : : : : : +- ^ ScanTransformer parquet (13)
: : : : +- ^ InputIteratorTransformer (28)
: : : : +- BroadcastQueryStage (26), Statistics(X)
: : : : +- ColumnarBroadcastExchange (25)
: : : : +- ^ FilterExecTransformer (23)
- : : : : +- ^ Scan parquet (22)
+ : : : : +- ^ ScanTransformer parquet (22)
: : : +- ^ InputIteratorTransformer (37)
: : : +- BroadcastQueryStage (35), Statistics(X)
: : : +- ColumnarBroadcastExchange (34)
: : : +- ^ FilterExecTransformer (32)
- : : : +- ^ Scan parquet (31)
+ : : : +- ^ ScanTransformer parquet (31)
: : +- ^ InputIteratorTransformer (46)
: : +- BroadcastQueryStage (44), Statistics(X)
: : +- ColumnarBroadcastExchange (43)
: : +- ^ FilterExecTransformer (41)
- : : +- ^ Scan parquet (40)
+ : : +- ^ ScanTransformer parquet (40)
: +- ^ InputIteratorTransformer (55)
: +- BroadcastQueryStage (53), Statistics(X)
: +- ColumnarBroadcastExchange (52)
: +- ^ FilterExecTransformer (50)
- : +- ^ Scan parquet (49)
+ : +- ^ ScanTransformer parquet (49)
+- ^ InputIteratorTransformer (65)
+- BroadcastQueryStage (63), Statistics(X)
+- ColumnarBroadcastExchange (62)
+- ^ ProjectExecTransformer (60)
+- ^ FilterExecTransformer (59)
- +- ^ Scan parquet (58)
+ +- ^ ScanTransformer parquet (58)
+- == Initial Plan ==
Sort (130)
+- Exchange (129)
@@ -115,7 +115,7 @@ AdaptiveSparkPlan (131)
+- Scan parquet (120)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [p_partkey#X, p_type#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -148,7 +148,7 @@ Input [1]: [p_partkey#X]
(8) InputIteratorTransformer
Input [1]: [p_partkey#X]
-(9) Scan parquet
+(9) ScanTransformer parquet
Output [5]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -168,7 +168,7 @@ Join condition: None
Output [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
Input [6]: [p_partkey#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
-(13) Scan parquet
+(13) ScanTransformer parquet
Output [2]: [s_suppkey#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -206,7 +206,7 @@ Join condition: None
Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X]
Input [6]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, s_suppkey#X, s_nationkey#X]
-(22) Scan parquet
+(22) ScanTransformer parquet
Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -244,7 +244,7 @@ Join condition: None
Output [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X]
Input [7]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderkey#X, o_custkey#X, o_orderdate#X]
-(31) Scan parquet
+(31) ScanTransformer parquet
Output [2]: [c_custkey#X, c_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -282,7 +282,7 @@ Join condition: None
Output [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X]
Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X, c_custkey#X, c_nationkey#X]
-(40) Scan parquet
+(40) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_regionkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -320,7 +320,7 @@ Join condition: None
Output [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X]
Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X, n_nationkey#X, n_regionkey#X]
-(49) Scan parquet
+(49) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -358,7 +358,7 @@ Join condition: None
Output [5]: [l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X]
Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X, n_nationkey#X, n_name#X]
-(58) Scan parquet
+(58) ScanTransformer parquet
Output [2]: [r_regionkey#X, r_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 212413568214..8c1db077485f 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
@@ -29,29 +29,29 @@ AdaptiveSparkPlan (100)
: : : : : +- ColumnarBroadcastExchange (5)
: : : : : +- ^ ProjectExecTransformer (3)
: : : : : +- ^ FilterExecTransformer (2)
- : : : : : +- ^ Scan parquet (1)
+ : : : : : +- ^ ScanTransformer parquet (1)
: : : : +- ^ FilterExecTransformer (10)
- : : : : +- ^ Scan parquet (9)
+ : : : : +- ^ ScanTransformer parquet (9)
: : : +- ^ InputIteratorTransformer (19)
: : : +- BroadcastQueryStage (17), Statistics(X)
: : : +- ColumnarBroadcastExchange (16)
: : : +- ^ FilterExecTransformer (14)
- : : : +- ^ Scan parquet (13)
+ : : : +- ^ ScanTransformer parquet (13)
: : +- ^ InputIteratorTransformer (28)
: : +- BroadcastQueryStage (26), Statistics(X)
: : +- ColumnarBroadcastExchange (25)
: : +- ^ FilterExecTransformer (23)
- : : +- ^ Scan parquet (22)
+ : : +- ^ ScanTransformer parquet (22)
: +- ^ InputIteratorTransformer (37)
: +- BroadcastQueryStage (35), Statistics(X)
: +- ColumnarBroadcastExchange (34)
: +- ^ FilterExecTransformer (32)
- : +- ^ Scan parquet (31)
+ : +- ^ ScanTransformer parquet (31)
+- ^ InputIteratorTransformer (46)
+- BroadcastQueryStage (44), Statistics(X)
+- ColumnarBroadcastExchange (43)
+- ^ FilterExecTransformer (41)
- +- ^ Scan parquet (40)
+ +- ^ ScanTransformer parquet (40)
+- == Initial Plan ==
Sort (99)
+- Exchange (98)
@@ -88,7 +88,7 @@ AdaptiveSparkPlan (100)
+- Scan parquet (90)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [p_partkey#X, p_name#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -121,7 +121,7 @@ Input [1]: [p_partkey#X]
(8) InputIteratorTransformer
Input [1]: [p_partkey#X]
-(9) Scan parquet
+(9) ScanTransformer parquet
Output [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -141,7 +141,7 @@ Join condition: None
Output [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X]
Input [7]: [p_partkey#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X]
-(13) Scan parquet
+(13) ScanTransformer parquet
Output [2]: [s_suppkey#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -179,7 +179,7 @@ Join condition: None
Output [7]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X]
Input [8]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_suppkey#X, s_nationkey#X]
-(22) Scan parquet
+(22) ScanTransformer parquet
Output [3]: [ps_partkey#X, ps_suppkey#X, ps_supplycost#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -217,7 +217,7 @@ Join condition: None
Output [6]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X]
Input [10]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_partkey#X, ps_suppkey#X, ps_supplycost#X]
-(31) Scan parquet
+(31) ScanTransformer parquet
Output [2]: [o_orderkey#X, o_orderdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -255,7 +255,7 @@ Join condition: None
Output [6]: [l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X]
Input [8]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderkey#X, o_orderdate#X]
-(40) Scan parquet
+(40) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 6dbee1ad7ffb..4d61f904dbe9 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
@@ -16,7 +16,7 @@ AdaptiveSparkPlan (30)
+- ^ FlushableHashAggregateExecTransformer (4)
+- ^ ProjectExecTransformer (3)
+- ^ FilterExecTransformer (2)
- +- ^ Scan parquet (1)
+ +- ^ ScanTransformer parquet (1)
+- == Initial Plan ==
Sort (29)
+- Exchange (28)
@@ -28,7 +28,7 @@ AdaptiveSparkPlan (30)
+- Scan parquet (22)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [7]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_tax#X, l_returnflag#X, l_linestatus#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
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 47dc04d100db..a4a335e80831 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
@@ -18,24 +18,24 @@ AdaptiveSparkPlan (68)
: :- ^ ProjectExecTransformer (12)
: : +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (11)
: : :- ^ FilterExecTransformer (2)
- : : : +- ^ Scan parquet (1)
+ : : : +- ^ ScanTransformer parquet (1)
: : +- ^ InputIteratorTransformer (10)
: : +- BroadcastQueryStage (8), Statistics(X)
: : +- ColumnarBroadcastExchange (7)
: : +- ^ ProjectExecTransformer (5)
: : +- ^ FilterExecTransformer (4)
- : : +- ^ Scan parquet (3)
+ : : +- ^ ScanTransformer parquet (3)
: +- ^ InputIteratorTransformer (20)
: +- BroadcastQueryStage (18), Statistics(X)
: +- ColumnarBroadcastExchange (17)
: +- ^ ProjectExecTransformer (15)
: +- ^ FilterExecTransformer (14)
- : +- ^ Scan parquet (13)
+ : +- ^ ScanTransformer parquet (13)
+- ^ InputIteratorTransformer (29)
+- BroadcastQueryStage (27), Statistics(X)
+- ColumnarBroadcastExchange (26)
+- ^ FilterExecTransformer (24)
- +- ^ Scan parquet (23)
+ +- ^ ScanTransformer parquet (23)
+- == Initial Plan ==
TakeOrderedAndProject (67)
+- HashAggregate (66)
@@ -62,7 +62,7 @@ AdaptiveSparkPlan (68)
+- Scan parquet (59)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -73,7 +73,7 @@ ReadSchema: struct
-(2) Scan parquet
+(2) ScanTransformer parquet
Output [3]: [o_orderkey#X, o_custkey#X, o_comment#X]
Batched: true
Location: InMemoryFileIndex [*]
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 6d2233b3e09f..61b9acc60dc0 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
@@ -13,12 +13,12 @@ AdaptiveSparkPlan (35)
+- ^ BroadcastHashJoinExecTransformer Inner BuildRight (11)
:- ^ ProjectExecTransformer (3)
: +- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
+ : +- ^ ScanTransformer parquet (1)
+- ^ InputIteratorTransformer (10)
+- BroadcastQueryStage (8), Statistics(X)
+- ColumnarBroadcastExchange (7)
+- ^ FilterExecTransformer (5)
- +- ^ Scan parquet (4)
+ +- ^ ScanTransformer parquet (4)
+- == Initial Plan ==
HashAggregate (34)
+- Exchange (33)
@@ -33,7 +33,7 @@ AdaptiveSparkPlan (35)
+- Scan parquet (27)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -48,7 +48,7 @@ Arguments: (((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1995-09-01)) AND (l_s
Output [3]: [l_partkey#X, l_extendedprice#X, l_discount#X]
Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
-(4) Scan parquet
+(4) ScanTransformer parquet
Output [2]: [p_partkey#X, p_type#X]
Batched: true
Location: InMemoryFileIndex [*]
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 1ec03a2f9edd..48b06986ee1b 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
@@ -12,7 +12,7 @@ AdaptiveSparkPlan (43)
: +- BroadcastQueryStage (5), Statistics(X)
: +- ColumnarBroadcastExchange (4)
: +- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
+ : +- ^ ScanTransformer parquet (1)
+- ^ FilterExecTransformer (20)
+- ^ RegularHashAggregateExecTransformer (19)
+- ^ InputIteratorTransformer (18)
@@ -23,7 +23,7 @@ AdaptiveSparkPlan (43)
+- ^ FlushableHashAggregateExecTransformer (11)
+- ^ ProjectExecTransformer (10)
+- ^ FilterExecTransformer (9)
- +- ^ Scan parquet (8)
+ +- ^ ScanTransformer parquet (8)
+- == Initial Plan ==
Sort (42)
+- Exchange (41)
@@ -41,7 +41,7 @@ AdaptiveSparkPlan (43)
+- Scan parquet (32)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -70,7 +70,7 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X]
(7) InputIteratorTransformer
Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X]
-(8) Scan parquet
+(8) ScanTransformer parquet
Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -239,34 +239,33 @@ Arguments: isFinalPlan=true
===== Subqueries =====
Subquery:1 Hosting operator id = 20 Hosting Expression = Subquery subquery#X, [id=#X]
-AdaptiveSparkPlan (69)
+AdaptiveSparkPlan (68)
+- == Final Plan ==
- VeloxColumnarToRow (60)
- +- ^ RegularHashAggregateExecTransformer (58)
- +- ^ RegularHashAggregateExecTransformer (57)
- +- ^ ProjectExecTransformer (56)
- +- ^ RegularHashAggregateExecTransformer (55)
- +- ^ InputIteratorTransformer (54)
- +- ShuffleQueryStage (52), Statistics(X)
- +- ColumnarExchange (51)
- +- VeloxResizeBatches (50)
- +- ^ ProjectExecTransformer (48)
- +- ^ FlushableHashAggregateExecTransformer (47)
- +- ^ ProjectExecTransformer (46)
- +- ^ FilterExecTransformer (45)
- +- ^ Scan parquet (44)
+ VeloxColumnarToRow (59)
+ +- ^ RegularHashAggregateExecTransformer (57)
+ +- ^ ProjectExecTransformer (56)
+ +- ^ RegularHashAggregateExecTransformer (55)
+ +- ^ InputIteratorTransformer (54)
+ +- ShuffleQueryStage (52), Statistics(X)
+ +- ColumnarExchange (51)
+ +- VeloxResizeBatches (50)
+ +- ^ ProjectExecTransformer (48)
+ +- ^ FlushableHashAggregateExecTransformer (47)
+ +- ^ ProjectExecTransformer (46)
+ +- ^ FilterExecTransformer (45)
+ +- ^ ScanTransformer parquet (44)
+- == Initial Plan ==
- HashAggregate (68)
- +- HashAggregate (67)
- +- HashAggregate (66)
- +- Exchange (65)
- +- HashAggregate (64)
- +- Project (63)
- +- Filter (62)
- +- Scan parquet (61)
+ HashAggregate (67)
+ +- HashAggregate (66)
+ +- HashAggregate (65)
+ +- Exchange (64)
+ +- HashAggregate (63)
+ +- Project (62)
+ +- Filter (61)
+ +- Scan parquet (60)
-(44) Scan parquet
+(44) ScanTransformer parquet
Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -328,71 +327,64 @@ Input [2]: [l_suppkey#X, sum((l_extendedprice#X * (1 - l_discount#X)))#X]
(57) RegularHashAggregateExecTransformer
Input [1]: [total_revenue#X]
Keys: []
-Functions [1]: [partial_max(total_revenue#X)]
-Aggregate Attributes [1]: [max#X]
-Results [1]: [max#X]
-
-(58) RegularHashAggregateExecTransformer
-Input [1]: [max#X]
-Keys: []
Functions [1]: [max(total_revenue#X)]
Aggregate Attributes [1]: [max(total_revenue#X)#X]
Results [1]: [max(total_revenue#X)#X AS max(total_revenue)#X]
-(59) WholeStageCodegenTransformer (X)
+(58) WholeStageCodegenTransformer (X)
Input [1]: [max(total_revenue)#X]
Arguments: false
-(60) VeloxColumnarToRow
+(59) VeloxColumnarToRow
Input [1]: [max(total_revenue)#X]
-(61) Scan parquet
+(60) Scan parquet
Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1996-01-01), LessThan(l_shipdate,1996-04-01)]
ReadSchema: struct
-(62) Filter
+(61) Filter
Input [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Condition : ((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1996-01-01)) AND (l_shipdate#X < 1996-04-01))
-(63) Project
+(62) Project
Output [3]: [l_suppkey#X, l_extendedprice#X, l_discount#X]
Input [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
-(64) HashAggregate
+(63) HashAggregate
Input [3]: [l_suppkey#X, l_extendedprice#X, l_discount#X]
Keys [1]: [l_suppkey#X]
Functions [1]: [partial_sum((l_extendedprice#X * (1 - l_discount#X)))]
Aggregate Attributes [2]: [sum#X, isEmpty#X]
Results [3]: [l_suppkey#X, sum#X, isEmpty#X]
-(65) Exchange
+(64) Exchange
Input [3]: [l_suppkey#X, sum#X, isEmpty#X]
Arguments: hashpartitioning(l_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(66) HashAggregate
+(65) HashAggregate
Input [3]: [l_suppkey#X, sum#X, isEmpty#X]
Keys [1]: [l_suppkey#X]
Functions [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))]
Aggregate Attributes [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))#X]
Results [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))#X AS total_revenue#X]
-(67) HashAggregate
+(66) HashAggregate
Input [1]: [total_revenue#X]
Keys: []
Functions [1]: [partial_max(total_revenue#X)]
Aggregate Attributes [1]: [max#X]
Results [1]: [max#X]
-(68) HashAggregate
+(67) HashAggregate
Input [1]: [max#X]
Keys: []
Functions [1]: [max(total_revenue#X)]
Aggregate Attributes [1]: [max(total_revenue#X)#X]
Results [1]: [max(total_revenue#X)#X AS max(total_revenue)#X]
-(69) AdaptiveSparkPlan
+(68) AdaptiveSparkPlan
Output [1]: [max(total_revenue)#X]
Arguments: isFinalPlan=true
\ No newline at end of file
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 e64cb0990026..3a9b17afd60c 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
@@ -24,12 +24,12 @@ AdaptiveSparkPlan (59)
+- ^ ProjectExecTransformer (11)
+- ^ BroadcastHashJoinExecTransformer Inner BuildRight (10)
:- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
+ : +- ^ ScanTransformer parquet (1)
+- ^ InputIteratorTransformer (9)
+- BroadcastQueryStage (7), Statistics(X)
+- ColumnarBroadcastExchange (6)
+- ^ FilterExecTransformer (4)
- +- ^ Scan parquet (3)
+ +- ^ ScanTransformer parquet (3)
+- == Initial Plan ==
Sort (58)
+- Exchange (57)
@@ -53,7 +53,7 @@ AdaptiveSparkPlan (59)
+- Scan parquet (46)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [ps_partkey#X, ps_suppkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -64,7 +64,7 @@ ReadSchema: struct
Input [2]: [ps_partkey#X, ps_suppkey#X]
Arguments: isnotnull(ps_partkey#X)
-(3) Scan parquet
+(3) ScanTransformer parquet
Output [4]: [p_partkey#X, p_brand#X, p_type#X, p_size#X]
Batched: true
Location: InMemoryFileIndex [*]
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 c92587c9d791..c3ab86e5acfb 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
@@ -18,10 +18,10 @@ AdaptiveSparkPlan (88)
: : +- BroadcastQueryStage (5), Statistics(X)
: : +- ColumnarBroadcastExchange (4)
: : +- ^ FilterExecTransformer (2)
- : : +- ^ Scan parquet (1)
+ : : +- ^ ScanTransformer parquet (1)
: +- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (27)
: :- ^ FilterExecTransformer (9)
- : : +- ^ Scan parquet (8)
+ : : +- ^ ScanTransformer parquet (8)
: +- ^ InputIteratorTransformer (26)
: +- BroadcastQueryStage (24), Statistics(X)
: +- ColumnarBroadcastExchange (23)
@@ -34,13 +34,13 @@ AdaptiveSparkPlan (88)
: +- VeloxResizeBatches (14)
: +- ^ ProjectExecTransformer (12)
: +- ^ FlushableHashAggregateExecTransformer (11)
- : +- ^ Scan parquet (10)
+ : +- ^ ScanTransformer parquet (10)
+- ^ InputIteratorTransformer (41)
+- BroadcastQueryStage (39), Statistics(X)
+- ColumnarBroadcastExchange (38)
+- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (36)
:- ^ FilterExecTransformer (31)
- : +- ^ Scan parquet (30)
+ : +- ^ ScanTransformer parquet (30)
+- ^ InputIteratorTransformer (35)
+- BroadcastQueryStage (33), Statistics(X)
+- ReusedExchange (32)
@@ -79,7 +79,7 @@ AdaptiveSparkPlan (88)
+- Scan parquet (73)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [c_custkey#X, c_name#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -108,7 +108,7 @@ Input [2]: [c_custkey#X, c_name#X]
(7) InputIteratorTransformer
Input [2]: [c_custkey#X, c_name#X]
-(8) Scan parquet
+(8) ScanTransformer parquet
Output [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -119,7 +119,7 @@ ReadSchema: struct
Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
Arguments: ((isnotnull(ps_availqty#X) AND isnotnull(ps_partkey#X)) AND isnotnull(ps_suppkey#X))
-(5) Scan parquet
+(5) ScanTransformer parquet
Output [2]: [p_partkey#X, p_name#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -168,7 +168,7 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
(18) InputIteratorTransformer
Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
-(19) Scan parquet
+(19) ScanTransformer parquet
Output [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -288,7 +288,7 @@ Join condition: None
Output [3]: [s_name#X, s_address#X, s_nationkey#X]
Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X]
-(47) Scan parquet
+(47) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 8382dbcd504e..5fab9fa9da42 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
@@ -19,34 +19,34 @@ AdaptiveSparkPlan (92)
: : : +- BroadcastQueryStage (5), Statistics(X)
: : : +- ColumnarBroadcastExchange (4)
: : : +- ^ FilterExecTransformer (2)
- : : : +- ^ Scan parquet (1)
+ : : : +- ^ ScanTransformer parquet (1)
: : +- ^ BroadcastHashJoinExecTransformer LeftAnti BuildRight (26)
: : :- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (17)
: : : :- ^ ProjectExecTransformer (10)
: : : : +- ^ FilterExecTransformer (9)
- : : : : +- ^ Scan parquet (8)
+ : : : : +- ^ ScanTransformer parquet (8)
: : : +- ^ InputIteratorTransformer (16)
: : : +- BroadcastQueryStage (14), Statistics(X)
: : : +- ColumnarBroadcastExchange (13)
- : : : +- ^ Scan parquet (11)
+ : : : +- ^ ScanTransformer parquet (11)
: : +- ^ InputIteratorTransformer (25)
: : +- BroadcastQueryStage (23), Statistics(X)
: : +- ColumnarBroadcastExchange (22)
: : +- ^ ProjectExecTransformer (20)
: : +- ^ FilterExecTransformer (19)
- : : +- ^ Scan parquet (18)
+ : : +- ^ ScanTransformer parquet (18)
: +- ^ InputIteratorTransformer (36)
: +- BroadcastQueryStage (34), Statistics(X)
: +- ColumnarBroadcastExchange (33)
: +- ^ ProjectExecTransformer (31)
: +- ^ FilterExecTransformer (30)
- : +- ^ Scan parquet (29)
+ : +- ^ ScanTransformer parquet (29)
+- ^ InputIteratorTransformer (46)
+- BroadcastQueryStage (44), Statistics(X)
+- ColumnarBroadcastExchange (43)
+- ^ ProjectExecTransformer (41)
+- ^ FilterExecTransformer (40)
- +- ^ Scan parquet (39)
+ +- ^ ScanTransformer parquet (39)
+- == Initial Plan ==
TakeOrderedAndProject (91)
+- HashAggregate (90)
@@ -82,7 +82,7 @@ AdaptiveSparkPlan (92)
+- Scan parquet (82)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [3]: [s_suppkey#X, s_name#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -111,7 +111,7 @@ Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X]
(7) InputIteratorTransformer
Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X]
-(8) Scan parquet
+(8) ScanTransformer parquet
Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -126,7 +126,7 @@ Arguments: ((((isnotnull(l_receiptdate#X) AND isnotnull(l_commitdate#X)) AND (l_
Output [2]: [l_orderkey#X, l_suppkey#X]
Input [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X]
-(11) Scan parquet
+(11) ScanTransformer parquet
Output [2]: [l_orderkey#X, l_suppkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -156,7 +156,7 @@ Right keys [1]: [l_orderkey#X]
Join type: LeftSemi
Join condition: NOT (l_suppkey#X = l_suppkey#X)
-(18) Scan parquet
+(18) ScanTransformer parquet
Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -205,7 +205,7 @@ Join condition: None
Output [3]: [s_name#X, s_nationkey#X, l_orderkey#X]
Input [5]: [s_suppkey#X, s_name#X, s_nationkey#X, l_orderkey#X, l_suppkey#X]
-(29) Scan parquet
+(29) ScanTransformer parquet
Output [2]: [o_orderkey#X, o_orderstatus#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -248,7 +248,7 @@ Join condition: None
Output [2]: [s_name#X, s_nationkey#X]
Input [4]: [s_name#X, s_nationkey#X, l_orderkey#X, o_orderkey#X]
-(39) Scan parquet
+(39) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 7f3836740155..0c2ce3c56b5a 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
@@ -17,11 +17,11 @@ AdaptiveSparkPlan (40)
+- ^ ProjectExecTransformer (10)
+- ^ BroadcastHashJoinExecTransformer LeftAnti BuildRight (9)
:- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
+ : +- ^ ScanTransformer parquet (1)
+- ^ InputIteratorTransformer (8)
+- BroadcastQueryStage (6), Statistics(X)
+- ColumnarBroadcastExchange (5)
- +- ^ Scan parquet (3)
+ +- ^ ScanTransformer parquet (3)
+- == Initial Plan ==
Sort (39)
+- Exchange (38)
@@ -36,7 +36,7 @@ AdaptiveSparkPlan (40)
+- Scan parquet (31)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [3]: [c_custkey#X, c_phone#X, c_acctbal#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -47,7 +47,7 @@ ReadSchema: struct
Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X]
Arguments: ((isnotnull(c_acctbal#X) AND substring(c_phone#X, 1, 2) IN (13,31,23,29,30,18,17)) AND (cast(c_acctbal#X as decimal(16,6)) > Subquery subquery#X, [id=#X]))
-(3) Scan parquet
+(3) ScanTransformer parquet
Output [1]: [o_custkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -229,7 +229,7 @@ AdaptiveSparkPlan (60)
+- ^ FlushableHashAggregateExecTransformer (44)
+- ^ ProjectExecTransformer (43)
+- ^ FilterExecTransformer (42)
- +- ^ Scan parquet (41)
+ +- ^ ScanTransformer parquet (41)
+- == Initial Plan ==
HashAggregate (59)
+- Exchange (58)
@@ -239,7 +239,7 @@ AdaptiveSparkPlan (60)
+- Scan parquet (54)
-(41) Scan parquet
+(41) ScanTransformer parquet
Output [2]: [c_phone#X, c_acctbal#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -346,7 +346,7 @@ AdaptiveSparkPlan (60)
+- ^ FlushableHashAggregateExecTransformer (44)
+- ^ ProjectExecTransformer (43)
+- ^ FilterExecTransformer (42)
- +- ^ Scan parquet (41)
+ +- ^ ScanTransformer parquet (41)
+- == Initial Plan ==
HashAggregate (59)
+- Exchange (58)
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 1035e0eb7715..cc68e1c9b249 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
@@ -20,15 +20,15 @@ AdaptiveSparkPlan (54)
: : +- ColumnarBroadcastExchange (5)
: : +- ^ ProjectExecTransformer (3)
: : +- ^ FilterExecTransformer (2)
- : : +- ^ Scan parquet (1)
+ : : +- ^ ScanTransformer parquet (1)
: +- ^ FilterExecTransformer (10)
- : +- ^ Scan parquet (9)
+ : +- ^ ScanTransformer parquet (9)
+- ^ InputIteratorTransformer (20)
+- BroadcastQueryStage (18), Statistics(X)
+- ColumnarBroadcastExchange (17)
+- ^ ProjectExecTransformer (15)
+- ^ FilterExecTransformer (14)
- +- ^ Scan parquet (13)
+ +- ^ ScanTransformer parquet (13)
+- == Initial Plan ==
TakeOrderedAndProject (53)
+- HashAggregate (52)
@@ -50,7 +50,7 @@ AdaptiveSparkPlan (54)
+- Scan parquet (44)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [c_custkey#X, c_mktsegment#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -83,7 +83,7 @@ Input [1]: [c_custkey#X]
(8) InputIteratorTransformer
Input [1]: [c_custkey#X]
-(9) Scan parquet
+(9) ScanTransformer parquet
Output [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -104,7 +104,7 @@ Join condition: None
Output [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X]
Input [5]: [c_custkey#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X]
-(13) Scan parquet
+(13) ScanTransformer parquet
Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
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 150788a3d7c1..a77b48298d37 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
@@ -18,13 +18,13 @@ AdaptiveSparkPlan (46)
+- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (12)
:- ^ ProjectExecTransformer (3)
: +- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
+ : +- ^ ScanTransformer parquet (1)
+- ^ InputIteratorTransformer (11)
+- BroadcastQueryStage (9), Statistics(X)
+- ColumnarBroadcastExchange (8)
+- ^ ProjectExecTransformer (6)
+- ^ FilterExecTransformer (5)
- +- ^ Scan parquet (4)
+ +- ^ ScanTransformer parquet (4)
+- == Initial Plan ==
Sort (45)
+- Exchange (44)
@@ -42,7 +42,7 @@ AdaptiveSparkPlan (46)
+- Scan parquet (35)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -57,7 +57,7 @@ Arguments: ((isnotnull(o_orderdate#X) AND (o_orderdate#X >= 1993-07-01)) AND (o_
Output [2]: [o_orderkey#X, o_orderpriority#X]
Input [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X]
-(4) Scan parquet
+(4) ScanTransformer parquet
Output [3]: [l_orderkey#X, l_commitdate#X, l_receiptdate#X]
Batched: true
Location: InMemoryFileIndex [*]
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 035c5c388972..58442a8ca86e 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
@@ -28,31 +28,31 @@ AdaptiveSparkPlan (102)
: : : : : +- BroadcastQueryStage (5), Statistics(X)
: : : : : +- ColumnarBroadcastExchange (4)
: : : : : +- ^ FilterExecTransformer (2)
- : : : : : +- ^ Scan parquet (1)
+ : : : : : +- ^ ScanTransformer parquet (1)
: : : : +- ^ ProjectExecTransformer (10)
: : : : +- ^ FilterExecTransformer (9)
- : : : : +- ^ Scan parquet (8)
+ : : : : +- ^ ScanTransformer parquet (8)
: : : +- ^ InputIteratorTransformer (19)
: : : +- BroadcastQueryStage (17), Statistics(X)
: : : +- ColumnarBroadcastExchange (16)
: : : +- ^ FilterExecTransformer (14)
- : : : +- ^ Scan parquet (13)
+ : : : +- ^ ScanTransformer parquet (13)
: : +- ^ InputIteratorTransformer (28)
: : +- BroadcastQueryStage (26), Statistics(X)
: : +- ColumnarBroadcastExchange (25)
: : +- ^ FilterExecTransformer (23)
- : : +- ^ Scan parquet (22)
+ : : +- ^ ScanTransformer parquet (22)
: +- ^ InputIteratorTransformer (37)
: +- BroadcastQueryStage (35), Statistics(X)
: +- ColumnarBroadcastExchange (34)
: +- ^ FilterExecTransformer (32)
- : +- ^ Scan parquet (31)
+ : +- ^ ScanTransformer parquet (31)
+- ^ InputIteratorTransformer (47)
+- BroadcastQueryStage (45), Statistics(X)
+- ColumnarBroadcastExchange (44)
+- ^ ProjectExecTransformer (42)
+- ^ FilterExecTransformer (41)
- +- ^ Scan parquet (40)
+ +- ^ ScanTransformer parquet (40)
+- == Initial Plan ==
Sort (101)
+- Exchange (100)
@@ -90,7 +90,7 @@ AdaptiveSparkPlan (102)
+- Scan parquet (91)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [c_custkey#X, c_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -119,7 +119,7 @@ Input [2]: [c_custkey#X, c_nationkey#X]
(7) InputIteratorTransformer
Input [2]: [c_custkey#X, c_nationkey#X]
-(8) Scan parquet
+(8) ScanTransformer parquet
Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -144,7 +144,7 @@ Join condition: None
Output [2]: [c_nationkey#X, o_orderkey#X]
Input [4]: [c_custkey#X, c_nationkey#X, o_orderkey#X, o_custkey#X]
-(13) Scan parquet
+(13) ScanTransformer parquet
Output [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -183,7 +183,7 @@ Join condition: None
Output [4]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
Input [6]: [c_nationkey#X, o_orderkey#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
-(22) Scan parquet
+(22) ScanTransformer parquet
Output [2]: [s_suppkey#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -222,7 +222,7 @@ Join condition: None
Output [3]: [l_extendedprice#X, l_discount#X, s_nationkey#X]
Input [6]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, s_suppkey#X, s_nationkey#X]
-(31) Scan parquet
+(31) ScanTransformer parquet
Output [3]: [n_nationkey#X, n_name#X, n_regionkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -261,7 +261,7 @@ Join condition: None
Output [4]: [l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X]
Input [6]: [l_extendedprice#X, l_discount#X, s_nationkey#X, n_nationkey#X, n_name#X, n_regionkey#X]
-(40) Scan parquet
+(40) ScanTransformer parquet
Output [2]: [r_regionkey#X, r_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 e9dc68456e55..319c457c1b4f 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
@@ -10,7 +10,7 @@ AdaptiveSparkPlan (20)
+- ^ FlushableHashAggregateExecTransformer (4)
+- ^ ProjectExecTransformer (3)
+- ^ FilterExecTransformer (2)
- +- ^ Scan parquet (1)
+ +- ^ ScanTransformer parquet (1)
+- == Initial Plan ==
HashAggregate (19)
+- Exchange (18)
@@ -20,7 +20,7 @@ AdaptiveSparkPlan (20)
+- Scan parquet (14)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [4]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
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 cd877f54bd7a..eaa84cfa46c2 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
@@ -28,24 +28,24 @@ AdaptiveSparkPlan (95)
: : : : : +- BroadcastQueryStage (5), Statistics(X)
: : : : : +- ColumnarBroadcastExchange (4)
: : : : : +- ^ FilterExecTransformer (2)
- : : : : : +- ^ Scan parquet (1)
+ : : : : : +- ^ ScanTransformer parquet (1)
: : : : +- ^ FilterExecTransformer (9)
- : : : : +- ^ Scan parquet (8)
+ : : : : +- ^ ScanTransformer parquet (8)
: : : +- ^ InputIteratorTransformer (18)
: : : +- BroadcastQueryStage (16), Statistics(X)
: : : +- ColumnarBroadcastExchange (15)
: : : +- ^ FilterExecTransformer (13)
- : : : +- ^ Scan parquet (12)
+ : : : +- ^ ScanTransformer parquet (12)
: : +- ^ InputIteratorTransformer (27)
: : +- BroadcastQueryStage (25), Statistics(X)
: : +- ColumnarBroadcastExchange (24)
: : +- ^ FilterExecTransformer (22)
- : : +- ^ Scan parquet (21)
+ : : +- ^ ScanTransformer parquet (21)
: +- ^ InputIteratorTransformer (36)
: +- BroadcastQueryStage (34), Statistics(X)
: +- ColumnarBroadcastExchange (33)
: +- ^ FilterExecTransformer (31)
- : +- ^ Scan parquet (30)
+ : +- ^ ScanTransformer parquet (30)
+- ^ InputIteratorTransformer (42)
+- BroadcastQueryStage (40), Statistics(X)
+- ReusedExchange (39)
@@ -84,7 +84,7 @@ AdaptiveSparkPlan (95)
+- Scan parquet (85)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [s_suppkey#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -113,7 +113,7 @@ Input [2]: [s_suppkey#X, s_nationkey#X]
(7) InputIteratorTransformer
Input [2]: [s_suppkey#X, s_nationkey#X]
-(8) Scan parquet
+(8) ScanTransformer parquet
Output [5]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -134,7 +134,7 @@ Join condition: None
Output [5]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Input [7]: [s_suppkey#X, s_nationkey#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
-(12) Scan parquet
+(12) ScanTransformer parquet
Output [2]: [o_orderkey#X, o_custkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -173,7 +173,7 @@ Join condition: None
Output [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X]
Input [7]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_orderkey#X, o_custkey#X]
-(21) Scan parquet
+(21) ScanTransformer parquet
Output [2]: [c_custkey#X, c_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -212,7 +212,7 @@ Join condition: None
Output [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X]
Input [7]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X, c_custkey#X, c_nationkey#X]
-(30) Scan parquet
+(30) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 d381d21da73d..9d1bc4a17558 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
@@ -34,40 +34,40 @@ AdaptiveSparkPlan (131)
: : : : : : : +- ColumnarBroadcastExchange (5)
: : : : : : : +- ^ ProjectExecTransformer (3)
: : : : : : : +- ^ FilterExecTransformer (2)
- : : : : : : : +- ^ Scan parquet (1)
+ : : : : : : : +- ^ ScanTransformer parquet (1)
: : : : : : +- ^ FilterExecTransformer (10)
- : : : : : : +- ^ Scan parquet (9)
+ : : : : : : +- ^ ScanTransformer parquet (9)
: : : : : +- ^ InputIteratorTransformer (19)
: : : : : +- BroadcastQueryStage (17), Statistics(X)
: : : : : +- ColumnarBroadcastExchange (16)
: : : : : +- ^ FilterExecTransformer (14)
- : : : : : +- ^ Scan parquet (13)
+ : : : : : +- ^ ScanTransformer parquet (13)
: : : : +- ^ InputIteratorTransformer (28)
: : : : +- BroadcastQueryStage (26), Statistics(X)
: : : : +- ColumnarBroadcastExchange (25)
: : : : +- ^ FilterExecTransformer (23)
- : : : : +- ^ Scan parquet (22)
+ : : : : +- ^ ScanTransformer parquet (22)
: : : +- ^ InputIteratorTransformer (37)
: : : +- BroadcastQueryStage (35), Statistics(X)
: : : +- ColumnarBroadcastExchange (34)
: : : +- ^ FilterExecTransformer (32)
- : : : +- ^ Scan parquet (31)
+ : : : +- ^ ScanTransformer parquet (31)
: : +- ^ InputIteratorTransformer (46)
: : +- BroadcastQueryStage (44), Statistics(X)
: : +- ColumnarBroadcastExchange (43)
: : +- ^ FilterExecTransformer (41)
- : : +- ^ Scan parquet (40)
+ : : +- ^ ScanTransformer parquet (40)
: +- ^ InputIteratorTransformer (55)
: +- BroadcastQueryStage (53), Statistics(X)
: +- ColumnarBroadcastExchange (52)
: +- ^ FilterExecTransformer (50)
- : +- ^ Scan parquet (49)
+ : +- ^ ScanTransformer parquet (49)
+- ^ InputIteratorTransformer (65)
+- BroadcastQueryStage (63), Statistics(X)
+- ColumnarBroadcastExchange (62)
+- ^ ProjectExecTransformer (60)
+- ^ FilterExecTransformer (59)
- +- ^ Scan parquet (58)
+ +- ^ ScanTransformer parquet (58)
+- == Initial Plan ==
Sort (130)
+- Exchange (129)
@@ -115,7 +115,7 @@ AdaptiveSparkPlan (131)
+- Scan parquet (120)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [p_partkey#X, p_type#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -148,7 +148,7 @@ Input [1]: [p_partkey#X]
(8) InputIteratorTransformer
Input [1]: [p_partkey#X]
-(9) Scan parquet
+(9) ScanTransformer parquet
Output [5]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -169,7 +169,7 @@ Join condition: None
Output [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
Input [6]: [p_partkey#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
-(13) Scan parquet
+(13) ScanTransformer parquet
Output [2]: [s_suppkey#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -208,7 +208,7 @@ Join condition: None
Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X]
Input [6]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, s_suppkey#X, s_nationkey#X]
-(22) Scan parquet
+(22) ScanTransformer parquet
Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -247,7 +247,7 @@ Join condition: None
Output [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X]
Input [7]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderkey#X, o_custkey#X, o_orderdate#X]
-(31) Scan parquet
+(31) ScanTransformer parquet
Output [2]: [c_custkey#X, c_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -286,7 +286,7 @@ Join condition: None
Output [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X]
Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X, c_custkey#X, c_nationkey#X]
-(40) Scan parquet
+(40) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_regionkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -325,7 +325,7 @@ Join condition: None
Output [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X]
Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X, n_nationkey#X, n_regionkey#X]
-(49) Scan parquet
+(49) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -364,7 +364,7 @@ Join condition: None
Output [5]: [l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X]
Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X, n_nationkey#X, n_name#X]
-(58) Scan parquet
+(58) ScanTransformer parquet
Output [2]: [r_regionkey#X, r_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 f0ebba6efac5..8baec4075ad3 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
@@ -29,29 +29,29 @@ AdaptiveSparkPlan (100)
: : : : : +- ColumnarBroadcastExchange (5)
: : : : : +- ^ ProjectExecTransformer (3)
: : : : : +- ^ FilterExecTransformer (2)
- : : : : : +- ^ Scan parquet (1)
+ : : : : : +- ^ ScanTransformer parquet (1)
: : : : +- ^ FilterExecTransformer (10)
- : : : : +- ^ Scan parquet (9)
+ : : : : +- ^ ScanTransformer parquet (9)
: : : +- ^ InputIteratorTransformer (19)
: : : +- BroadcastQueryStage (17), Statistics(X)
: : : +- ColumnarBroadcastExchange (16)
: : : +- ^ FilterExecTransformer (14)
- : : : +- ^ Scan parquet (13)
+ : : : +- ^ ScanTransformer parquet (13)
: : +- ^ InputIteratorTransformer (28)
: : +- BroadcastQueryStage (26), Statistics(X)
: : +- ColumnarBroadcastExchange (25)
: : +- ^ FilterExecTransformer (23)
- : : +- ^ Scan parquet (22)
+ : : +- ^ ScanTransformer parquet (22)
: +- ^ InputIteratorTransformer (37)
: +- BroadcastQueryStage (35), Statistics(X)
: +- ColumnarBroadcastExchange (34)
: +- ^ FilterExecTransformer (32)
- : +- ^ Scan parquet (31)
+ : +- ^ ScanTransformer parquet (31)
+- ^ InputIteratorTransformer (46)
+- BroadcastQueryStage (44), Statistics(X)
+- ColumnarBroadcastExchange (43)
+- ^ FilterExecTransformer (41)
- +- ^ Scan parquet (40)
+ +- ^ ScanTransformer parquet (40)
+- == Initial Plan ==
Sort (99)
+- Exchange (98)
@@ -88,7 +88,7 @@ AdaptiveSparkPlan (100)
+- Scan parquet (90)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [p_partkey#X, p_name#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -121,7 +121,7 @@ Input [1]: [p_partkey#X]
(8) InputIteratorTransformer
Input [1]: [p_partkey#X]
-(9) Scan parquet
+(9) ScanTransformer parquet
Output [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -142,7 +142,7 @@ Join condition: None
Output [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X]
Input [7]: [p_partkey#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X]
-(13) Scan parquet
+(13) ScanTransformer parquet
Output [2]: [s_suppkey#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -181,7 +181,7 @@ Join condition: None
Output [7]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X]
Input [8]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_suppkey#X, s_nationkey#X]
-(22) Scan parquet
+(22) ScanTransformer parquet
Output [3]: [ps_partkey#X, ps_suppkey#X, ps_supplycost#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -220,7 +220,7 @@ Join condition: None
Output [6]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X]
Input [10]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_partkey#X, ps_suppkey#X, ps_supplycost#X]
-(31) Scan parquet
+(31) ScanTransformer parquet
Output [2]: [o_orderkey#X, o_orderdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -259,7 +259,7 @@ Join condition: None
Output [6]: [l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X]
Input [8]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderkey#X, o_orderdate#X]
-(40) Scan parquet
+(40) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 efe7f3908506..617489f1aa35 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
@@ -16,7 +16,7 @@ AdaptiveSparkPlan (30)
+- ^ FlushableHashAggregateExecTransformer (4)
+- ^ ProjectExecTransformer (3)
+- ^ FilterExecTransformer (2)
- +- ^ Scan parquet (1)
+ +- ^ ScanTransformer parquet (1)
+- == Initial Plan ==
Sort (29)
+- Exchange (28)
@@ -28,7 +28,7 @@ AdaptiveSparkPlan (30)
+- Scan parquet (22)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [7]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_tax#X, l_returnflag#X, l_linestatus#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
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 a7ced66a82e7..7c8451dfd6dd 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
@@ -31,28 +31,28 @@ AdaptiveSparkPlan (100)
: : : +- VeloxResizeBatches (5)
: : : +- ^ ProjectExecTransformer (3)
: : : +- ^ FilterExecTransformer (2)
- : : : +- ^ Scan parquet (1)
+ : : : +- ^ ScanTransformer parquet (1)
: : +- ^ InputIteratorTransformer (18)
: : +- ShuffleQueryStage (16)
: : +- ColumnarExchange (15)
: : +- VeloxResizeBatches (14)
: : +- ^ ProjectExecTransformer (12)
: : +- ^ FilterExecTransformer (11)
- : : +- ^ Scan parquet (10)
+ : : +- ^ ScanTransformer parquet (10)
: +- ^ InputIteratorTransformer (35)
: +- ShuffleQueryStage (33)
: +- ColumnarExchange (32)
: +- VeloxResizeBatches (31)
: +- ^ ProjectExecTransformer (29)
: +- ^ FilterExecTransformer (28)
- : +- ^ Scan parquet (27)
+ : +- ^ ScanTransformer parquet (27)
+- ^ InputIteratorTransformer (52)
+- ShuffleQueryStage (50)
+- ColumnarExchange (49)
+- VeloxResizeBatches (48)
+- ^ ProjectExecTransformer (46)
+- ^ FilterExecTransformer (45)
- +- ^ Scan parquet (44)
+ +- ^ ScanTransformer parquet (44)
+- == Initial Plan ==
TakeOrderedAndProject (99)
+- HashAggregate (98)
@@ -88,7 +88,7 @@ AdaptiveSparkPlan (100)
+- Scan parquet (90)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -125,7 +125,7 @@ Input [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct
(9) InputIteratorTransformer
Input [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -193,7 +193,7 @@ Input [8]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct
(26) InputIteratorTransformer
Input [8]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, o_orderkey#X]
-(27) Scan parquet
+(27) ScanTransformer parquet
Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_returnflag#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -261,7 +261,7 @@ Input [9]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct
(43) InputIteratorTransformer
Input [9]: [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]
-(44) Scan parquet
+(44) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 b385b7719de2..f943422808e8 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
@@ -29,21 +29,21 @@ AdaptiveSparkPlan (82)
: : +- VeloxResizeBatches (5)
: : +- ^ ProjectExecTransformer (3)
: : +- ^ FilterExecTransformer (2)
- : : +- ^ Scan parquet (1)
+ : : +- ^ ScanTransformer parquet (1)
: +- ^ InputIteratorTransformer (18)
: +- ShuffleQueryStage (16)
: +- ColumnarExchange (15)
: +- VeloxResizeBatches (14)
: +- ^ ProjectExecTransformer (12)
: +- ^ FilterExecTransformer (11)
- : +- ^ Scan parquet (10)
+ : +- ^ ScanTransformer parquet (10)
+- ^ InputIteratorTransformer (35)
+- ShuffleQueryStage (33)
+- ColumnarExchange (32)
+- VeloxResizeBatches (31)
+- ^ ProjectExecTransformer (29)
+- ^ FilterExecTransformer (28)
- +- ^ Scan parquet (27)
+ +- ^ ScanTransformer parquet (27)
+- == Initial Plan ==
Sort (81)
+- Exchange (80)
@@ -72,7 +72,7 @@ AdaptiveSparkPlan (82)
+- Scan parquet (69)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -109,7 +109,7 @@ Input [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X]
(9) InputIteratorTransformer
Input [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [2]: [s_suppkey#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -177,7 +177,7 @@ Input [4]: [ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X]
(26) InputIteratorTransformer
Input [4]: [ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X]
-(27) Scan parquet
+(27) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 71dee80fab86..48e7e4d4e0d5 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
@@ -22,14 +22,14 @@ AdaptiveSparkPlan (55)
: +- VeloxResizeBatches (5)
: +- ^ ProjectExecTransformer (3)
: +- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
+ : +- ^ ScanTransformer parquet (1)
+- ^ InputIteratorTransformer (18)
+- ShuffleQueryStage (16)
+- ColumnarExchange (15)
+- VeloxResizeBatches (14)
+- ^ ProjectExecTransformer (12)
+- ^ FilterExecTransformer (11)
- +- ^ Scan parquet (10)
+ +- ^ ScanTransformer parquet (10)
+- == Initial Plan ==
Sort (54)
+- Exchange (53)
@@ -49,7 +49,7 @@ AdaptiveSparkPlan (55)
+- Scan parquet (43)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [o_orderkey#X, o_orderpriority#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -86,7 +86,7 @@ Input [2]: [o_orderkey#X, o_orderpriority#X]
(9) InputIteratorTransformer
Input [2]: [o_orderkey#X, o_orderpriority#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [5]: [l_orderkey#X, l_shipdate#X, l_commitdate#X, l_receiptdate#X, l_shipmode#X]
Batched: true
Location: InMemoryFileIndex [*]
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 edbcb50206df..c8ddb5437009 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
@@ -1,58 +1,57 @@
== Physical Plan ==
-AdaptiveSparkPlan (58)
+AdaptiveSparkPlan (57)
+- == Final Plan ==
- VeloxColumnarToRow (40)
- +- ^ SortExecTransformer (38)
- +- ^ InputIteratorTransformer (37)
- +- ShuffleQueryStage (35)
- +- ColumnarExchange (34)
- +- VeloxResizeBatches (33)
- +- ^ RegularHashAggregateExecTransformer (31)
- +- ^ InputIteratorTransformer (30)
- +- ShuffleQueryStage (28)
- +- ColumnarExchange (27)
- +- VeloxResizeBatches (26)
- +- ^ ProjectExecTransformer (24)
- +- ^ FlushableHashAggregateExecTransformer (23)
- +- ^ ProjectExecTransformer (22)
- +- ^ RegularHashAggregateExecTransformer (21)
- +- ^ RegularHashAggregateExecTransformer (20)
- +- ^ ProjectExecTransformer (19)
- +- ^ ShuffledHashJoinExecTransformer LeftOuter BuildLeft (18)
- :- ^ InputIteratorTransformer (8)
- : +- ShuffleQueryStage (6)
- : +- ColumnarExchange (5)
- : +- VeloxResizeBatches (4)
- : +- ^ ProjectExecTransformer (2)
- : +- ^ Scan parquet (1)
- +- ^ InputIteratorTransformer (17)
- +- ShuffleQueryStage (15)
- +- ColumnarExchange (14)
- +- VeloxResizeBatches (13)
- +- ^ ProjectExecTransformer (11)
- +- ^ FilterExecTransformer (10)
- +- ^ Scan parquet (9)
+ VeloxColumnarToRow (39)
+ +- ^ SortExecTransformer (37)
+ +- ^ InputIteratorTransformer (36)
+ +- ShuffleQueryStage (34)
+ +- ColumnarExchange (33)
+ +- VeloxResizeBatches (32)
+ +- ^ RegularHashAggregateExecTransformer (30)
+ +- ^ InputIteratorTransformer (29)
+ +- ShuffleQueryStage (27)
+ +- ColumnarExchange (26)
+ +- VeloxResizeBatches (25)
+ +- ^ ProjectExecTransformer (23)
+ +- ^ FlushableHashAggregateExecTransformer (22)
+ +- ^ ProjectExecTransformer (21)
+ +- ^ RegularHashAggregateExecTransformer (20)
+ +- ^ ProjectExecTransformer (19)
+ +- ^ ShuffledHashJoinExecTransformer LeftOuter BuildLeft (18)
+ :- ^ InputIteratorTransformer (8)
+ : +- ShuffleQueryStage (6)
+ : +- ColumnarExchange (5)
+ : +- VeloxResizeBatches (4)
+ : +- ^ ProjectExecTransformer (2)
+ : +- ^ ScanTransformer parquet (1)
+ +- ^ InputIteratorTransformer (17)
+ +- ShuffleQueryStage (15)
+ +- ColumnarExchange (14)
+ +- VeloxResizeBatches (13)
+ +- ^ ProjectExecTransformer (11)
+ +- ^ FilterExecTransformer (10)
+ +- ^ ScanTransformer parquet (9)
+- == Initial Plan ==
- Sort (57)
- +- Exchange (56)
- +- HashAggregate (55)
- +- Exchange (54)
- +- HashAggregate (53)
- +- HashAggregate (52)
- +- HashAggregate (51)
- +- Project (50)
- +- SortMergeJoin LeftOuter (49)
- :- Sort (43)
- : +- Exchange (42)
- : +- Scan parquet (41)
- +- Sort (48)
- +- Exchange (47)
- +- Project (46)
- +- Filter (45)
- +- Scan parquet (44)
-
-
-(1) Scan parquet
+ Sort (56)
+ +- Exchange (55)
+ +- HashAggregate (54)
+ +- Exchange (53)
+ +- HashAggregate (52)
+ +- HashAggregate (51)
+ +- HashAggregate (50)
+ +- Project (49)
+ +- SortMergeJoin LeftOuter (48)
+ :- Sort (42)
+ : +- Exchange (41)
+ : +- Scan parquet (40)
+ +- Sort (47)
+ +- Exchange (46)
+ +- Project (45)
+ +- Filter (44)
+ +- Scan parquet (43)
+
+
+(1) ScanTransformer parquet
Output [1]: [c_custkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -84,7 +83,7 @@ Input [1]: [c_custkey#X]
(8) InputIteratorTransformer
Input [1]: [c_custkey#X]
-(9) Scan parquet
+(9) ScanTransformer parquet
Output [3]: [o_orderkey#X, o_custkey#X, o_comment#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -133,180 +132,173 @@ Input [3]: [c_custkey#X, o_orderkey#X, o_custkey#X]
(20) RegularHashAggregateExecTransformer
Input [2]: [c_custkey#X, o_orderkey#X]
Keys [1]: [c_custkey#X]
-Functions [1]: [partial_count(o_orderkey#X)]
-Aggregate Attributes [1]: [count#X]
-Results [2]: [c_custkey#X, count#X]
-
-(21) RegularHashAggregateExecTransformer
-Input [2]: [c_custkey#X, count#X]
-Keys [1]: [c_custkey#X]
Functions [1]: [count(o_orderkey#X)]
Aggregate Attributes [1]: [count(o_orderkey#X)#X]
Results [2]: [c_custkey#X, count(o_orderkey#X)#X]
-(22) ProjectExecTransformer
+(21) ProjectExecTransformer
Output [1]: [count(o_orderkey#X)#X AS c_count#X]
Input [2]: [c_custkey#X, count(o_orderkey#X)#X]
-(23) FlushableHashAggregateExecTransformer
+(22) FlushableHashAggregateExecTransformer
Input [1]: [c_count#X]
Keys [1]: [c_count#X]
Functions [1]: [partial_count(1)]
Aggregate Attributes [1]: [count#X]
Results [2]: [c_count#X, count#X]
-(24) ProjectExecTransformer
+(23) ProjectExecTransformer
Output [3]: [hash(c_count#X, 42) AS hash_partition_key#X, c_count#X, count#X]
Input [2]: [c_count#X, count#X]
-(25) WholeStageCodegenTransformer (X)
+(24) WholeStageCodegenTransformer (X)
Input [3]: [hash_partition_key#X, c_count#X, count#X]
Arguments: false
-(26) VeloxResizeBatches
+(25) VeloxResizeBatches
Input [3]: [hash_partition_key#X, c_count#X, count#X]
Arguments: X, X
-(27) ColumnarExchange
+(26) ColumnarExchange
Input [3]: [hash_partition_key#X, c_count#X, count#X]
Arguments: hashpartitioning(c_count#X, 1), ENSURE_REQUIREMENTS, [c_count#X, count#X], [plan_id=X], [shuffle_writer_type=hash]
-(28) ShuffleQueryStage
+(27) ShuffleQueryStage
Output [2]: [c_count#X, count#X]
Arguments: X
-(29) InputAdapter
+(28) InputAdapter
Input [2]: [c_count#X, count#X]
-(30) InputIteratorTransformer
+(29) InputIteratorTransformer
Input [2]: [c_count#X, count#X]
-(31) RegularHashAggregateExecTransformer
+(30) RegularHashAggregateExecTransformer
Input [2]: [c_count#X, count#X]
Keys [1]: [c_count#X]
Functions [1]: [count(1)]
Aggregate Attributes [1]: [count(1)#X]
Results [2]: [c_count#X, count(1)#X AS custdist#X]
-(32) WholeStageCodegenTransformer (X)
+(31) WholeStageCodegenTransformer (X)
Input [2]: [c_count#X, custdist#X]
Arguments: false
-(33) VeloxResizeBatches
+(32) VeloxResizeBatches
Input [2]: [c_count#X, custdist#X]
Arguments: X, X
-(34) ColumnarExchange
+(33) ColumnarExchange
Input [2]: [c_count#X, custdist#X]
Arguments: rangepartitioning(custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST, 1), ENSURE_REQUIREMENTS, [plan_id=X], [shuffle_writer_type=hash]
-(35) ShuffleQueryStage
+(34) ShuffleQueryStage
Output [2]: [c_count#X, custdist#X]
Arguments: X
-(36) InputAdapter
+(35) InputAdapter
Input [2]: [c_count#X, custdist#X]
-(37) InputIteratorTransformer
+(36) InputIteratorTransformer
Input [2]: [c_count#X, custdist#X]
-(38) SortExecTransformer
+(37) SortExecTransformer
Input [2]: [c_count#X, custdist#X]
Arguments: [custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST], true, 0
-(39) WholeStageCodegenTransformer (X)
+(38) WholeStageCodegenTransformer (X)
Input [2]: [c_count#X, custdist#X]
Arguments: false
-(40) VeloxColumnarToRow
+(39) VeloxColumnarToRow
Input [2]: [c_count#X, custdist#X]
-(41) Scan parquet
+(40) Scan parquet
Output [1]: [c_custkey#X]
Batched: true
Location: InMemoryFileIndex [*]
ReadSchema: struct
-(42) Exchange
+(41) Exchange
Input [1]: [c_custkey#X]
Arguments: hashpartitioning(c_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(43) Sort
+(42) Sort
Input [1]: [c_custkey#X]
Arguments: [c_custkey#X ASC NULLS FIRST], false, 0
-(44) Scan parquet
+(43) Scan parquet
Output [3]: [o_orderkey#X, o_custkey#X, o_comment#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(o_comment), IsNotNull(o_custkey)]
ReadSchema: struct
-(45) Filter
+(44) Filter
Input [3]: [o_orderkey#X, o_custkey#X, o_comment#X]
Condition : ((isnotnull(o_comment#X) AND NOT o_comment#X LIKE %special%requests%) AND isnotnull(o_custkey#X))
-(46) Project
+(45) Project
Output [2]: [o_orderkey#X, o_custkey#X]
Input [3]: [o_orderkey#X, o_custkey#X, o_comment#X]
-(47) Exchange
+(46) Exchange
Input [2]: [o_orderkey#X, o_custkey#X]
Arguments: hashpartitioning(o_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(48) Sort
+(47) Sort
Input [2]: [o_orderkey#X, o_custkey#X]
Arguments: [o_custkey#X ASC NULLS FIRST], false, 0
-(49) SortMergeJoin
+(48) SortMergeJoin
Left keys [1]: [c_custkey#X]
Right keys [1]: [o_custkey#X]
Join condition: None
-(50) Project
+(49) Project
Output [2]: [c_custkey#X, o_orderkey#X]
Input [3]: [c_custkey#X, o_orderkey#X, o_custkey#X]
-(51) HashAggregate
+(50) HashAggregate
Input [2]: [c_custkey#X, o_orderkey#X]
Keys [1]: [c_custkey#X]
Functions [1]: [partial_count(o_orderkey#X)]
Aggregate Attributes [1]: [count#X]
Results [2]: [c_custkey#X, count#X]
-(52) HashAggregate
+(51) HashAggregate
Input [2]: [c_custkey#X, count#X]
Keys [1]: [c_custkey#X]
Functions [1]: [count(o_orderkey#X)]
Aggregate Attributes [1]: [count(o_orderkey#X)#X]
Results [1]: [count(o_orderkey#X)#X AS c_count#X]
-(53) HashAggregate
+(52) HashAggregate
Input [1]: [c_count#X]
Keys [1]: [c_count#X]
Functions [1]: [partial_count(1)]
Aggregate Attributes [1]: [count#X]
Results [2]: [c_count#X, count#X]
-(54) Exchange
+(53) Exchange
Input [2]: [c_count#X, count#X]
Arguments: hashpartitioning(c_count#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(55) HashAggregate
+(54) HashAggregate
Input [2]: [c_count#X, count#X]
Keys [1]: [c_count#X]
Functions [1]: [count(1)]
Aggregate Attributes [1]: [count(1)#X]
Results [2]: [c_count#X, count(1)#X AS custdist#X]
-(56) Exchange
+(55) Exchange
Input [2]: [c_count#X, custdist#X]
Arguments: rangepartitioning(custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(57) Sort
+(56) Sort
Input [2]: [c_count#X, custdist#X]
Arguments: [custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST], true, 0
-(58) AdaptiveSparkPlan
+(57) AdaptiveSparkPlan
Output [2]: [c_count#X, custdist#X]
Arguments: isFinalPlan=true
\ No newline at end of file
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 91df642595a3..40a891738143 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
@@ -1,43 +1,42 @@
== Physical Plan ==
-AdaptiveSparkPlan (39)
+AdaptiveSparkPlan (38)
+- == Final Plan ==
- VeloxColumnarToRow (25)
- +- ^ ProjectExecTransformer (23)
- +- ^ RegularHashAggregateExecTransformer (22)
- +- ^ RegularHashAggregateExecTransformer (21)
- +- ^ ProjectExecTransformer (20)
- +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19)
- :- ^ InputIteratorTransformer (9)
- : +- ShuffleQueryStage (7)
- : +- ColumnarExchange (6)
- : +- VeloxResizeBatches (5)
- : +- ^ ProjectExecTransformer (3)
- : +- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
- +- ^ InputIteratorTransformer (18)
- +- ShuffleQueryStage (16)
- +- ColumnarExchange (15)
- +- VeloxResizeBatches (14)
- +- ^ ProjectExecTransformer (12)
- +- ^ FilterExecTransformer (11)
- +- ^ Scan parquet (10)
+ VeloxColumnarToRow (24)
+ +- ^ ProjectExecTransformer (22)
+ +- ^ RegularHashAggregateExecTransformer (21)
+ +- ^ ProjectExecTransformer (20)
+ +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19)
+ :- ^ InputIteratorTransformer (9)
+ : +- ShuffleQueryStage (7)
+ : +- ColumnarExchange (6)
+ : +- VeloxResizeBatches (5)
+ : +- ^ ProjectExecTransformer (3)
+ : +- ^ FilterExecTransformer (2)
+ : +- ^ ScanTransformer parquet (1)
+ +- ^ InputIteratorTransformer (18)
+ +- ShuffleQueryStage (16)
+ +- ColumnarExchange (15)
+ +- VeloxResizeBatches (14)
+ +- ^ ProjectExecTransformer (12)
+ +- ^ FilterExecTransformer (11)
+ +- ^ ScanTransformer parquet (10)
+- == Initial Plan ==
- HashAggregate (38)
- +- HashAggregate (37)
- +- Project (36)
- +- SortMergeJoin Inner (35)
- :- Sort (30)
- : +- Exchange (29)
- : +- Project (28)
- : +- Filter (27)
- : +- Scan parquet (26)
- +- Sort (34)
- +- Exchange (33)
- +- Filter (32)
- +- Scan parquet (31)
-
-
-(1) Scan parquet
+ HashAggregate (37)
+ +- HashAggregate (36)
+ +- Project (35)
+ +- SortMergeJoin Inner (34)
+ :- Sort (29)
+ : +- Exchange (28)
+ : +- Project (27)
+ : +- Filter (26)
+ : +- Scan parquet (25)
+ +- Sort (33)
+ +- Exchange (32)
+ +- Filter (31)
+ +- Scan parquet (30)
+
+
+(1) ScanTransformer parquet
Output [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -74,7 +73,7 @@ Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X]
(9) InputIteratorTransformer
Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [2]: [p_partkey#X, p_type#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -123,93 +122,86 @@ Input [5]: [l_partkey#X, l_extendedprice#X, l_discount#X, p_partkey#X, p_type#X]
(21) RegularHashAggregateExecTransformer
Input [5]: [l_extendedprice#X, l_discount#X, p_type#X, _pre_X#X, _pre_X#X]
Keys: []
-Functions [2]: [partial_sum(_pre_X#X), partial_sum(_pre_X#X)]
-Aggregate Attributes [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X]
-Results [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X]
-
-(22) RegularHashAggregateExecTransformer
-Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X]
-Keys: []
-Functions [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true) ELSE 0.0000 END), sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))]
+Functions [2]: [sum(_pre_X#X), sum(_pre_X#X)]
Aggregate Attributes [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true) ELSE 0.0000 END)#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X]
Results [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true) ELSE 0.0000 END)#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X]
-(23) ProjectExecTransformer
+(22) ProjectExecTransformer
Output [1]: [CheckOverflow((promote_precision(CheckOverflow((100.0000 * promote_precision(sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true) ELSE 0.0000 END)#X)), DecimalType(38,6), true)) / promote_precision(cast(sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X as decimal(38,6)))), DecimalType(38,6), true) AS promo_revenue#X]
Input [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true) ELSE 0.0000 END)#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X]
-(24) WholeStageCodegenTransformer (X)
+(23) WholeStageCodegenTransformer (X)
Input [1]: [promo_revenue#X]
Arguments: false
-(25) VeloxColumnarToRow
+(24) VeloxColumnarToRow
Input [1]: [promo_revenue#X]
-(26) Scan parquet
+(25) Scan parquet
Output [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1995-09-01), LessThan(l_shipdate,1995-10-01), IsNotNull(l_partkey)]
ReadSchema: struct
-(27) Filter
+(26) Filter
Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Condition : (((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1995-09-01)) AND (l_shipdate#X < 1995-10-01)) AND isnotnull(l_partkey#X))
-(28) Project
+(27) Project
Output [3]: [l_partkey#X, l_extendedprice#X, l_discount#X]
Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
-(29) Exchange
+(28) Exchange
Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X]
Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(30) Sort
+(29) Sort
Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X]
Arguments: [l_partkey#X ASC NULLS FIRST], false, 0
-(31) Scan parquet
+(30) Scan parquet
Output [2]: [p_partkey#X, p_type#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(p_partkey)]
ReadSchema: struct
-(32) Filter
+(31) Filter
Input [2]: [p_partkey#X, p_type#X]
Condition : isnotnull(p_partkey#X)
-(33) Exchange
+(32) Exchange
Input [2]: [p_partkey#X, p_type#X]
Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(34) Sort
+(33) Sort
Input [2]: [p_partkey#X, p_type#X]
Arguments: [p_partkey#X ASC NULLS FIRST], false, 0
-(35) SortMergeJoin
+(34) SortMergeJoin
Left keys [1]: [l_partkey#X]
Right keys [1]: [p_partkey#X]
Join condition: None
-(36) Project
+(35) Project
Output [3]: [l_extendedprice#X, l_discount#X, p_type#X]
Input [5]: [l_partkey#X, l_extendedprice#X, l_discount#X, p_partkey#X, p_type#X]
-(37) HashAggregate
+(36) HashAggregate
Input [3]: [l_extendedprice#X, l_discount#X, p_type#X]
Keys: []
Functions [2]: [partial_sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true) ELSE 0.0000 END), partial_sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))]
Aggregate Attributes [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X]
Results [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X]
-(38) HashAggregate
+(37) HashAggregate
Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X]
Keys: []
Functions [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true) ELSE 0.0000 END), sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))]
Aggregate Attributes [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true) ELSE 0.0000 END)#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X]
Results [1]: [CheckOverflow((promote_precision(CheckOverflow((100.0000 * promote_precision(sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true) ELSE 0.0000 END)#X)), DecimalType(38,6), true)) / promote_precision(cast(sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X as decimal(38,6)))), DecimalType(38,6), true) AS promo_revenue#X]
-(39) AdaptiveSparkPlan
+(38) AdaptiveSparkPlan
Output [1]: [promo_revenue#X]
Arguments: isFinalPlan=true
\ No newline at end of file
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 56169a244bf9..9317c2f895a5 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
@@ -15,7 +15,7 @@ AdaptiveSparkPlan (50)
: +- VeloxResizeBatches (5)
: +- ^ ProjectExecTransformer (3)
: +- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
+ : +- ^ ScanTransformer parquet (1)
+- ^ FilterExecTransformer (22)
+- ^ RegularHashAggregateExecTransformer (21)
+- ^ InputIteratorTransformer (20)
@@ -26,7 +26,7 @@ AdaptiveSparkPlan (50)
+- ^ FlushableHashAggregateExecTransformer (13)
+- ^ ProjectExecTransformer (12)
+- ^ FilterExecTransformer (11)
- +- ^ Scan parquet (10)
+ +- ^ ScanTransformer parquet (10)
+- == Initial Plan ==
Sort (49)
+- Exchange (48)
@@ -46,7 +46,7 @@ AdaptiveSparkPlan (50)
+- Scan parquet (38)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -83,7 +83,7 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X]
(9) InputIteratorTransformer
Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
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 8ef7d4d51441..2c41822beeeb 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
@@ -29,14 +29,14 @@ AdaptiveSparkPlan (71)
: +- VeloxResizeBatches (5)
: +- ^ ProjectExecTransformer (3)
: +- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
+ : +- ^ ScanTransformer parquet (1)
+- ^ InputIteratorTransformer (18)
+- ShuffleQueryStage (16)
+- ColumnarExchange (15)
+- VeloxResizeBatches (14)
+- ^ ProjectExecTransformer (12)
+- ^ FilterExecTransformer (11)
- +- ^ Scan parquet (10)
+ +- ^ ScanTransformer parquet (10)
+- == Initial Plan ==
Sort (70)
+- Exchange (69)
@@ -63,7 +63,7 @@ AdaptiveSparkPlan (71)
+- Scan parquet (57)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [ps_partkey#X, ps_suppkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -100,7 +100,7 @@ Input [2]: [ps_partkey#X, ps_suppkey#X]
(9) InputIteratorTransformer
Input [2]: [ps_partkey#X, ps_suppkey#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [4]: [p_partkey#X, p_brand#X, p_type#X, p_size#X]
Batched: true
Location: InMemoryFileIndex [*]
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 fe915989f9e6..30fd4b0b1124 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
@@ -1,65 +1,64 @@
== Physical Plan ==
-AdaptiveSparkPlan (63)
+AdaptiveSparkPlan (62)
+- == Final Plan ==
- VeloxColumnarToRow (40)
- +- ^ ProjectExecTransformer (38)
- +- ^ RegularHashAggregateExecTransformer (37)
- +- ^ RegularHashAggregateExecTransformer (36)
- +- ^ ProjectExecTransformer (35)
- +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (34)
- :- ^ ProjectExecTransformer (20)
- : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19)
- : :- ^ InputIteratorTransformer (9)
- : : +- ShuffleQueryStage (7)
- : : +- ColumnarExchange (6)
- : : +- VeloxResizeBatches (5)
- : : +- ^ ProjectExecTransformer (3)
- : : +- ^ FilterExecTransformer (2)
- : : +- ^ Scan parquet (1)
- : +- ^ InputIteratorTransformer (18)
- : +- ShuffleQueryStage (16)
- : +- ColumnarExchange (15)
- : +- VeloxResizeBatches (14)
- : +- ^ ProjectExecTransformer (12)
- : +- ^ FilterExecTransformer (11)
- : +- ^ Scan parquet (10)
- +- ^ FilterExecTransformer (33)
- +- ^ ProjectExecTransformer (32)
- +- ^ RegularHashAggregateExecTransformer (31)
- +- ^ InputIteratorTransformer (30)
- +- ShuffleQueryStage (28)
- +- ColumnarExchange (27)
- +- VeloxResizeBatches (26)
- +- ^ ProjectExecTransformer (24)
- +- ^ FlushableHashAggregateExecTransformer (23)
- +- ^ FilterExecTransformer (22)
- +- ^ Scan parquet (21)
+ VeloxColumnarToRow (39)
+ +- ^ ProjectExecTransformer (37)
+ +- ^ RegularHashAggregateExecTransformer (36)
+ +- ^ ProjectExecTransformer (35)
+ +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (34)
+ :- ^ ProjectExecTransformer (20)
+ : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19)
+ : :- ^ InputIteratorTransformer (9)
+ : : +- ShuffleQueryStage (7)
+ : : +- ColumnarExchange (6)
+ : : +- VeloxResizeBatches (5)
+ : : +- ^ ProjectExecTransformer (3)
+ : : +- ^ FilterExecTransformer (2)
+ : : +- ^ ScanTransformer parquet (1)
+ : +- ^ InputIteratorTransformer (18)
+ : +- ShuffleQueryStage (16)
+ : +- ColumnarExchange (15)
+ : +- VeloxResizeBatches (14)
+ : +- ^ ProjectExecTransformer (12)
+ : +- ^ FilterExecTransformer (11)
+ : +- ^ ScanTransformer parquet (10)
+ +- ^ FilterExecTransformer (33)
+ +- ^ ProjectExecTransformer (32)
+ +- ^ RegularHashAggregateExecTransformer (31)
+ +- ^ InputIteratorTransformer (30)
+ +- ShuffleQueryStage (28)
+ +- ColumnarExchange (27)
+ +- VeloxResizeBatches (26)
+ +- ^ ProjectExecTransformer (24)
+ +- ^ FlushableHashAggregateExecTransformer (23)
+ +- ^ FilterExecTransformer (22)
+ +- ^ ScanTransformer parquet (21)
+- == Initial Plan ==
- HashAggregate (62)
- +- HashAggregate (61)
- +- Project (60)
- +- SortMergeJoin Inner (59)
- :- Project (51)
- : +- SortMergeJoin Inner (50)
- : :- Sort (44)
- : : +- Exchange (43)
- : : +- Filter (42)
- : : +- Scan parquet (41)
- : +- Sort (49)
- : +- Exchange (48)
- : +- Project (47)
- : +- Filter (46)
- : +- Scan parquet (45)
- +- Sort (58)
- +- Filter (57)
- +- HashAggregate (56)
- +- Exchange (55)
- +- HashAggregate (54)
- +- Filter (53)
- +- Scan parquet (52)
-
-
-(1) Scan parquet
+ HashAggregate (61)
+ +- HashAggregate (60)
+ +- Project (59)
+ +- SortMergeJoin Inner (58)
+ :- Project (50)
+ : +- SortMergeJoin Inner (49)
+ : :- Sort (43)
+ : : +- Exchange (42)
+ : : +- Filter (41)
+ : : +- Scan parquet (40)
+ : +- Sort (48)
+ : +- Exchange (47)
+ : +- Project (46)
+ : +- Filter (45)
+ : +- Scan parquet (44)
+ +- Sort (57)
+ +- Filter (56)
+ +- HashAggregate (55)
+ +- Exchange (54)
+ +- HashAggregate (53)
+ +- Filter (52)
+ +- Scan parquet (51)
+
+
+(1) ScanTransformer parquet
Output [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -96,7 +95,7 @@ Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X]
(9) InputIteratorTransformer
Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [3]: [p_partkey#X, p_brand#X, p_container#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -142,7 +141,7 @@ Join condition: None
Output [3]: [l_quantity#X, l_extendedprice#X, p_partkey#X]
Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, p_partkey#X]
-(21) Scan parquet
+(21) ScanTransformer parquet
Output [2]: [l_partkey#X, l_quantity#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -213,139 +212,132 @@ Input [5]: [l_quantity#X, l_extendedprice#X, p_partkey#X, (0.2 * avg(l_quantity)
(36) RegularHashAggregateExecTransformer
Input [1]: [l_extendedprice#X]
Keys: []
-Functions [1]: [partial_sum(l_extendedprice#X)]
-Aggregate Attributes [2]: [sum#X, isEmpty#X]
-Results [2]: [sum#X, isEmpty#X]
-
-(37) RegularHashAggregateExecTransformer
-Input [2]: [sum#X, isEmpty#X]
-Keys: []
Functions [1]: [sum(l_extendedprice#X)]
Aggregate Attributes [1]: [sum(l_extendedprice#X)#X]
Results [1]: [sum(l_extendedprice#X)#X]
-(38) ProjectExecTransformer
+(37) ProjectExecTransformer
Output [1]: [CheckOverflow((promote_precision(sum(l_extendedprice#X)#X) / 7.00), DecimalType(27,6), true) AS avg_yearly#X]
Input [1]: [sum(l_extendedprice#X)#X]
-(39) WholeStageCodegenTransformer (X)
+(38) WholeStageCodegenTransformer (X)
Input [1]: [avg_yearly#X]
Arguments: false
-(40) VeloxColumnarToRow
+(39) VeloxColumnarToRow
Input [1]: [avg_yearly#X]
-(41) Scan parquet
+(40) Scan parquet
Output [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(l_partkey), IsNotNull(l_quantity)]
ReadSchema: struct
-(42) Filter
+(41) Filter
Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X]
Condition : (isnotnull(l_partkey#X) AND isnotnull(l_quantity#X))
-(43) Exchange
+(42) Exchange
Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X]
Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(44) Sort
+(43) Sort
Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X]
Arguments: [l_partkey#X ASC NULLS FIRST], false, 0
-(45) Scan parquet
+(44) Scan parquet
Output [3]: [p_partkey#X, p_brand#X, p_container#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(p_brand), IsNotNull(p_container), EqualTo(p_brand,Brand#X), EqualTo(p_container,MED BOX), IsNotNull(p_partkey)]
ReadSchema: struct
-(46) Filter
+(45) Filter
Input [3]: [p_partkey#X, p_brand#X, p_container#X]
Condition : ((((isnotnull(p_brand#X) AND isnotnull(p_container#X)) AND (p_brand#X = Brand#X)) AND (p_container#X = MED BOX)) AND isnotnull(p_partkey#X))
-(47) Project
+(46) Project
Output [1]: [p_partkey#X]
Input [3]: [p_partkey#X, p_brand#X, p_container#X]
-(48) Exchange
+(47) Exchange
Input [1]: [p_partkey#X]
Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(49) Sort
+(48) Sort
Input [1]: [p_partkey#X]
Arguments: [p_partkey#X ASC NULLS FIRST], false, 0
-(50) SortMergeJoin
+(49) SortMergeJoin
Left keys [1]: [l_partkey#X]
Right keys [1]: [p_partkey#X]
Join condition: None
-(51) Project
+(50) Project
Output [3]: [l_quantity#X, l_extendedprice#X, p_partkey#X]
Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, p_partkey#X]
-(52) Scan parquet
+(51) Scan parquet
Output [2]: [l_partkey#X, l_quantity#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(l_partkey)]
ReadSchema: struct
-(53) Filter
+(52) Filter
Input [2]: [l_partkey#X, l_quantity#X]
Condition : isnotnull(l_partkey#X)
-(54) HashAggregate
+(53) HashAggregate
Input [2]: [l_partkey#X, l_quantity#X]
Keys [1]: [l_partkey#X]
Functions [1]: [partial_avg(l_quantity#X)]
Aggregate Attributes [2]: [sum#X, count#X]
Results [3]: [l_partkey#X, sum#X, count#X]
-(55) Exchange
+(54) Exchange
Input [3]: [l_partkey#X, sum#X, count#X]
Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(56) HashAggregate
+(55) HashAggregate
Input [3]: [l_partkey#X, sum#X, count#X]
Keys [1]: [l_partkey#X]
Functions [1]: [avg(l_quantity#X)]
Aggregate Attributes [1]: [avg(l_quantity#X)#X]
Results [2]: [CheckOverflow((0.200000 * promote_precision(avg(l_quantity#X)#X)), DecimalType(18,7), true) AS (0.2 * avg(l_quantity))#X, l_partkey#X]
-(57) Filter
+(56) Filter
Input [2]: [(0.2 * avg(l_quantity))#X, l_partkey#X]
Condition : isnotnull((0.2 * avg(l_quantity))#X)
-(58) Sort
+(57) Sort
Input [2]: [(0.2 * avg(l_quantity))#X, l_partkey#X]
Arguments: [l_partkey#X ASC NULLS FIRST], false, 0
-(59) SortMergeJoin
+(58) SortMergeJoin
Left keys [1]: [p_partkey#X]
Right keys [1]: [l_partkey#X]
Join condition: (cast(l_quantity#X as decimal(18,7)) < (0.2 * avg(l_quantity))#X)
-(60) Project
+(59) Project
Output [1]: [l_extendedprice#X]
Input [5]: [l_quantity#X, l_extendedprice#X, p_partkey#X, (0.2 * avg(l_quantity))#X, l_partkey#X]
-(61) HashAggregate
+(60) HashAggregate
Input [1]: [l_extendedprice#X]
Keys: []
Functions [1]: [partial_sum(l_extendedprice#X)]
Aggregate Attributes [2]: [sum#X, isEmpty#X]
Results [2]: [sum#X, isEmpty#X]
-(62) HashAggregate
+(61) HashAggregate
Input [2]: [sum#X, isEmpty#X]
Keys: []
Functions [1]: [sum(l_extendedprice#X)]
Aggregate Attributes [1]: [sum(l_extendedprice#X)#X]
Results [1]: [CheckOverflow((promote_precision(sum(l_extendedprice#X)#X) / 7.00), DecimalType(27,6), true) AS avg_yearly#X]
-(63) AdaptiveSparkPlan
+(62) AdaptiveSparkPlan
Output [1]: [avg_yearly#X]
Arguments: isFinalPlan=true
\ No newline at end of file
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 a2345ca9ed35..dd4cad538e2c 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
@@ -1,105 +1,104 @@
== Physical Plan ==
-AdaptiveSparkPlan (110)
+AdaptiveSparkPlan (109)
+- == Final Plan ==
- VeloxColumnarToRow (70)
- +- TakeOrderedAndProjectExecTransformer (69)
- +- ^ RegularHashAggregateExecTransformer (67)
- +- ^ RegularHashAggregateExecTransformer (66)
- +- ^ ProjectExecTransformer (65)
- +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (64)
- :- ^ InputIteratorTransformer (46)
- : +- ShuffleQueryStage (44)
- : +- ColumnarExchange (43)
- : +- VeloxResizeBatches (42)
- : +- ^ ProjectExecTransformer (40)
- : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (39)
- : :- ^ InputIteratorTransformer (9)
- : : +- ShuffleQueryStage (7)
- : : +- ColumnarExchange (6)
- : : +- VeloxResizeBatches (5)
- : : +- ^ ProjectExecTransformer (3)
- : : +- ^ FilterExecTransformer (2)
- : : +- ^ Scan parquet (1)
- : +- ^ InputIteratorTransformer (38)
- : +- ShuffleQueryStage (36)
- : +- ColumnarExchange (35)
- : +- VeloxResizeBatches (34)
- : +- ^ ProjectExecTransformer (32)
- : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (31)
- : :- ^ InputIteratorTransformer (18)
- : : +- ShuffleQueryStage (16)
- : : +- ColumnarExchange (15)
- : : +- VeloxResizeBatches (14)
- : : +- ^ ProjectExecTransformer (12)
- : : +- ^ FilterExecTransformer (11)
- : : +- ^ Scan parquet (10)
- : +- ^ ProjectExecTransformer (30)
- : +- ^ FilterExecTransformer (29)
- : +- ^ RegularHashAggregateExecTransformer (28)
- : +- ^ InputIteratorTransformer (27)
- : +- ShuffleQueryStage (25)
- : +- ColumnarExchange (24)
- : +- VeloxResizeBatches (23)
- : +- ^ ProjectExecTransformer (21)
- : +- ^ FlushableHashAggregateExecTransformer (20)
- : +- ^ Scan parquet (19)
- +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (63)
- :- ^ InputIteratorTransformer (55)
- : +- ShuffleQueryStage (53)
- : +- ColumnarExchange (52)
- : +- VeloxResizeBatches (51)
- : +- ^ ProjectExecTransformer (49)
- : +- ^ FilterExecTransformer (48)
- : +- ^ Scan parquet (47)
- +- ^ ProjectExecTransformer (62)
- +- ^ FilterExecTransformer (61)
- +- ^ RegularHashAggregateExecTransformer (60)
- +- ^ InputIteratorTransformer (59)
- +- ShuffleQueryStage (57)
- +- ReusedExchange (56)
+ VeloxColumnarToRow (69)
+ +- TakeOrderedAndProjectExecTransformer (68)
+ +- ^ RegularHashAggregateExecTransformer (66)
+ +- ^ ProjectExecTransformer (65)
+ +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (64)
+ :- ^ InputIteratorTransformer (46)
+ : +- ShuffleQueryStage (44)
+ : +- ColumnarExchange (43)
+ : +- VeloxResizeBatches (42)
+ : +- ^ ProjectExecTransformer (40)
+ : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (39)
+ : :- ^ InputIteratorTransformer (9)
+ : : +- ShuffleQueryStage (7)
+ : : +- ColumnarExchange (6)
+ : : +- VeloxResizeBatches (5)
+ : : +- ^ ProjectExecTransformer (3)
+ : : +- ^ FilterExecTransformer (2)
+ : : +- ^ ScanTransformer parquet (1)
+ : +- ^ InputIteratorTransformer (38)
+ : +- ShuffleQueryStage (36)
+ : +- ColumnarExchange (35)
+ : +- VeloxResizeBatches (34)
+ : +- ^ ProjectExecTransformer (32)
+ : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (31)
+ : :- ^ InputIteratorTransformer (18)
+ : : +- ShuffleQueryStage (16)
+ : : +- ColumnarExchange (15)
+ : : +- VeloxResizeBatches (14)
+ : : +- ^ ProjectExecTransformer (12)
+ : : +- ^ FilterExecTransformer (11)
+ : : +- ^ ScanTransformer parquet (10)
+ : +- ^ ProjectExecTransformer (30)
+ : +- ^ FilterExecTransformer (29)
+ : +- ^ RegularHashAggregateExecTransformer (28)
+ : +- ^ InputIteratorTransformer (27)
+ : +- ShuffleQueryStage (25)
+ : +- ColumnarExchange (24)
+ : +- VeloxResizeBatches (23)
+ : +- ^ ProjectExecTransformer (21)
+ : +- ^ FlushableHashAggregateExecTransformer (20)
+ : +- ^ ScanTransformer parquet (19)
+ +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (63)
+ :- ^ InputIteratorTransformer (55)
+ : +- ShuffleQueryStage (53)
+ : +- ColumnarExchange (52)
+ : +- VeloxResizeBatches (51)
+ : +- ^ ProjectExecTransformer (49)
+ : +- ^ FilterExecTransformer (48)
+ : +- ^ ScanTransformer parquet (47)
+ +- ^ ProjectExecTransformer (62)
+ +- ^ FilterExecTransformer (61)
+ +- ^ RegularHashAggregateExecTransformer (60)
+ +- ^ InputIteratorTransformer (59)
+ +- ShuffleQueryStage (57)
+ +- ReusedExchange (56)
+- == Initial Plan ==
- TakeOrderedAndProject (109)
- +- HashAggregate (108)
- +- HashAggregate (107)
- +- Project (106)
- +- SortMergeJoin Inner (105)
- :- Sort (92)
- : +- Exchange (91)
- : +- Project (90)
- : +- SortMergeJoin Inner (89)
- : :- Sort (74)
- : : +- Exchange (73)
- : : +- Filter (72)
- : : +- Scan parquet (71)
- : +- Sort (88)
- : +- Exchange (87)
- : +- SortMergeJoin LeftSemi (86)
- : :- Sort (78)
- : : +- Exchange (77)
- : : +- Filter (76)
- : : +- Scan parquet (75)
- : +- Sort (85)
- : +- Project (84)
- : +- Filter (83)
- : +- HashAggregate (82)
- : +- Exchange (81)
- : +- HashAggregate (80)
- : +- Scan parquet (79)
- +- SortMergeJoin LeftSemi (104)
- :- Sort (96)
- : +- Exchange (95)
- : +- Filter (94)
- : +- Scan parquet (93)
- +- Sort (103)
- +- Project (102)
- +- Filter (101)
- +- HashAggregate (100)
- +- Exchange (99)
- +- HashAggregate (98)
- +- Scan parquet (97)
-
-
-(1) Scan parquet
+ TakeOrderedAndProject (108)
+ +- HashAggregate (107)
+ +- HashAggregate (106)
+ +- Project (105)
+ +- SortMergeJoin Inner (104)
+ :- Sort (91)
+ : +- Exchange (90)
+ : +- Project (89)
+ : +- SortMergeJoin Inner (88)
+ : :- Sort (73)
+ : : +- Exchange (72)
+ : : +- Filter (71)
+ : : +- Scan parquet (70)
+ : +- Sort (87)
+ : +- Exchange (86)
+ : +- SortMergeJoin LeftSemi (85)
+ : :- Sort (77)
+ : : +- Exchange (76)
+ : : +- Filter (75)
+ : : +- Scan parquet (74)
+ : +- Sort (84)
+ : +- Project (83)
+ : +- Filter (82)
+ : +- HashAggregate (81)
+ : +- Exchange (80)
+ : +- HashAggregate (79)
+ : +- Scan parquet (78)
+ +- SortMergeJoin LeftSemi (103)
+ :- Sort (95)
+ : +- Exchange (94)
+ : +- Filter (93)
+ : +- Scan parquet (92)
+ +- Sort (102)
+ +- Project (101)
+ +- Filter (100)
+ +- HashAggregate (99)
+ +- Exchange (98)
+ +- HashAggregate (97)
+ +- Scan parquet (96)
+
+
+(1) ScanTransformer parquet
Output [2]: [c_custkey#X, c_name#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -136,7 +135,7 @@ Input [2]: [c_custkey#X, c_name#X]
(9) InputIteratorTransformer
Input [2]: [c_custkey#X, c_name#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -173,7 +172,7 @@ Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X]
(18) InputIteratorTransformer
Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X]
-(19) Scan parquet
+(19) ScanTransformer parquet
Output [2]: [l_orderkey#X, l_quantity#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -289,7 +288,7 @@ Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X]
(46) InputIteratorTransformer
Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X]
-(47) Scan parquet
+(47) ScanTransformer parquet
Output [2]: [l_orderkey#X, l_quantity#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -371,219 +370,212 @@ Input [7]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X,
(66) RegularHashAggregateExecTransformer
Input [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_quantity#X]
Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X]
-Functions [1]: [partial_sum(l_quantity#X)]
-Aggregate Attributes [2]: [sum#X, isEmpty#X]
-Results [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X]
-
-(67) RegularHashAggregateExecTransformer
-Input [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X]
-Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X]
Functions [1]: [sum(l_quantity#X)]
Aggregate Attributes [1]: [sum(l_quantity#X)#X]
Results [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity#X)#X AS sum(l_quantity)#X]
-(68) WholeStageCodegenTransformer (X)
+(67) WholeStageCodegenTransformer (X)
Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X]
Arguments: false
-(69) TakeOrderedAndProjectExecTransformer
+(68) TakeOrderedAndProjectExecTransformer
Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X]
Arguments: X, [o_totalprice#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X], 0
-(70) VeloxColumnarToRow
+(69) VeloxColumnarToRow
Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X]
-(71) Scan parquet
+(70) Scan parquet
Output [2]: [c_custkey#X, c_name#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(c_custkey)]
ReadSchema: struct
-(72) Filter
+(71) Filter
Input [2]: [c_custkey#X, c_name#X]
Condition : isnotnull(c_custkey#X)
-(73) Exchange
+(72) Exchange
Input [2]: [c_custkey#X, c_name#X]
Arguments: hashpartitioning(c_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(74) Sort
+(73) Sort
Input [2]: [c_custkey#X, c_name#X]
Arguments: [c_custkey#X ASC NULLS FIRST], false, 0
-(75) Scan parquet
+(74) Scan parquet
Output [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(o_custkey), IsNotNull(o_orderkey)]
ReadSchema: struct
-(76) Filter
+(75) Filter
Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X]
Condition : (isnotnull(o_custkey#X) AND isnotnull(o_orderkey#X))
-(77) Exchange
+(76) Exchange
Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X]
Arguments: hashpartitioning(o_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(78) Sort
+(77) Sort
Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X]
Arguments: [o_orderkey#X ASC NULLS FIRST], false, 0
-(79) Scan parquet
+(78) Scan parquet
Output [2]: [l_orderkey#X, l_quantity#X]
Batched: true
Location: InMemoryFileIndex [*]
ReadSchema: struct
-(80) HashAggregate
+(79) HashAggregate
Input [2]: [l_orderkey#X, l_quantity#X]
Keys [1]: [l_orderkey#X]
Functions [1]: [partial_sum(l_quantity#X)]
Aggregate Attributes [2]: [sum#X, isEmpty#X]
Results [3]: [l_orderkey#X, sum#X, isEmpty#X]
-(81) Exchange
+(80) Exchange
Input [3]: [l_orderkey#X, sum#X, isEmpty#X]
Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(82) HashAggregate
+(81) HashAggregate
Input [3]: [l_orderkey#X, sum#X, isEmpty#X]
Keys [1]: [l_orderkey#X]
Functions [1]: [sum(l_quantity#X)]
Aggregate Attributes [1]: [sum(l_quantity#X)#X]
Results [2]: [l_orderkey#X, sum(l_quantity#X)#X AS sum(l_quantity#X)#X]
-(83) Filter
+(82) Filter
Input [2]: [l_orderkey#X, sum(l_quantity#X)#X]
Condition : (isnotnull(sum(l_quantity#X)#X) AND (sum(l_quantity#X)#X > 300.00))
-(84) Project
+(83) Project
Output [1]: [l_orderkey#X]
Input [2]: [l_orderkey#X, sum(l_quantity#X)#X]
-(85) Sort
+(84) Sort
Input [1]: [l_orderkey#X]
Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0
-(86) SortMergeJoin
+(85) SortMergeJoin
Left keys [1]: [o_orderkey#X]
Right keys [1]: [l_orderkey#X]
Join condition: None
-(87) Exchange
+(86) Exchange
Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X]
Arguments: hashpartitioning(o_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(88) Sort
+(87) Sort
Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X]
Arguments: [o_custkey#X ASC NULLS FIRST], false, 0
-(89) SortMergeJoin
+(88) SortMergeJoin
Left keys [1]: [c_custkey#X]
Right keys [1]: [o_custkey#X]
Join condition: None
-(90) Project
+(89) Project
Output [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X]
Input [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X]
-(91) Exchange
+(90) Exchange
Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X]
Arguments: hashpartitioning(o_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(92) Sort
+(91) Sort
Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X]
Arguments: [o_orderkey#X ASC NULLS FIRST], false, 0
-(93) Scan parquet
+(92) Scan parquet
Output [2]: [l_orderkey#X, l_quantity#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(l_orderkey)]
ReadSchema: struct
-(94) Filter
+(93) Filter
Input [2]: [l_orderkey#X, l_quantity#X]
Condition : isnotnull(l_orderkey#X)
-(95) Exchange
+(94) Exchange
Input [2]: [l_orderkey#X, l_quantity#X]
Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(96) Sort
+(95) Sort
Input [2]: [l_orderkey#X, l_quantity#X]
Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0
-(97) Scan parquet
+(96) Scan parquet
Output [2]: [l_orderkey#X, l_quantity#X]
Batched: true
Location: InMemoryFileIndex [*]
ReadSchema: struct
-(98) HashAggregate
+(97) HashAggregate
Input [2]: [l_orderkey#X, l_quantity#X]
Keys [1]: [l_orderkey#X]
Functions [1]: [partial_sum(l_quantity#X)]
Aggregate Attributes [2]: [sum#X, isEmpty#X]
Results [3]: [l_orderkey#X, sum#X, isEmpty#X]
-(99) Exchange
+(98) Exchange
Input [3]: [l_orderkey#X, sum#X, isEmpty#X]
Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(100) HashAggregate
+(99) HashAggregate
Input [3]: [l_orderkey#X, sum#X, isEmpty#X]
Keys [1]: [l_orderkey#X]
Functions [1]: [sum(l_quantity#X)]
Aggregate Attributes [1]: [sum(l_quantity#X)#X]
Results [2]: [l_orderkey#X, sum(l_quantity#X)#X AS sum(l_quantity#X)#X]
-(101) Filter
+(100) Filter
Input [2]: [l_orderkey#X, sum(l_quantity#X)#X]
Condition : (isnotnull(sum(l_quantity#X)#X) AND (sum(l_quantity#X)#X > 300.00))
-(102) Project
+(101) Project
Output [1]: [l_orderkey#X]
Input [2]: [l_orderkey#X, sum(l_quantity#X)#X]
-(103) Sort
+(102) Sort
Input [1]: [l_orderkey#X]
Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0
-(104) SortMergeJoin
+(103) SortMergeJoin
Left keys [1]: [l_orderkey#X]
Right keys [1]: [l_orderkey#X]
Join condition: None
-(105) SortMergeJoin
+(104) SortMergeJoin
Left keys [1]: [o_orderkey#X]
Right keys [1]: [l_orderkey#X]
Join condition: None
-(106) Project
+(105) Project
Output [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_quantity#X]
Input [7]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_orderkey#X, l_quantity#X]
-(107) HashAggregate
+(106) HashAggregate
Input [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_quantity#X]
Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X]
Functions [1]: [partial_sum(l_quantity#X)]
Aggregate Attributes [2]: [sum#X, isEmpty#X]
Results [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X]
-(108) HashAggregate
+(107) HashAggregate
Input [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X]
Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X]
Functions [1]: [sum(l_quantity#X)]
Aggregate Attributes [1]: [sum(l_quantity#X)#X]
Results [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity#X)#X AS sum(l_quantity)#X]
-(109) TakeOrderedAndProject
+(108) TakeOrderedAndProject
Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X]
Arguments: X, [o_totalprice#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X]
-(110) AdaptiveSparkPlan
+(109) AdaptiveSparkPlan
Output [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X]
Arguments: isFinalPlan=true
\ No newline at end of file
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 d06cdb0fecff..507b700e37e9 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
@@ -1,42 +1,41 @@
== Physical Plan ==
-AdaptiveSparkPlan (38)
+AdaptiveSparkPlan (37)
+- == Final Plan ==
- VeloxColumnarToRow (24)
- +- ^ RegularHashAggregateExecTransformer (22)
- +- ^ RegularHashAggregateExecTransformer (21)
- +- ^ ProjectExecTransformer (20)
- +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19)
- :- ^ InputIteratorTransformer (9)
- : +- ShuffleQueryStage (7)
- : +- ColumnarExchange (6)
- : +- VeloxResizeBatches (5)
- : +- ^ ProjectExecTransformer (3)
- : +- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
- +- ^ InputIteratorTransformer (18)
- +- ShuffleQueryStage (16)
- +- ColumnarExchange (15)
- +- VeloxResizeBatches (14)
- +- ^ ProjectExecTransformer (12)
- +- ^ FilterExecTransformer (11)
- +- ^ Scan parquet (10)
+ VeloxColumnarToRow (23)
+ +- ^ RegularHashAggregateExecTransformer (21)
+ +- ^ ProjectExecTransformer (20)
+ +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19)
+ :- ^ InputIteratorTransformer (9)
+ : +- ShuffleQueryStage (7)
+ : +- ColumnarExchange (6)
+ : +- VeloxResizeBatches (5)
+ : +- ^ ProjectExecTransformer (3)
+ : +- ^ FilterExecTransformer (2)
+ : +- ^ ScanTransformer parquet (1)
+ +- ^ InputIteratorTransformer (18)
+ +- ShuffleQueryStage (16)
+ +- ColumnarExchange (15)
+ +- VeloxResizeBatches (14)
+ +- ^ ProjectExecTransformer (12)
+ +- ^ FilterExecTransformer (11)
+ +- ^ ScanTransformer parquet (10)
+- == Initial Plan ==
- HashAggregate (37)
- +- HashAggregate (36)
- +- Project (35)
- +- SortMergeJoin Inner (34)
- :- Sort (29)
- : +- Exchange (28)
- : +- Project (27)
- : +- Filter (26)
- : +- Scan parquet (25)
- +- Sort (33)
- +- Exchange (32)
- +- Filter (31)
- +- Scan parquet (30)
-
-
-(1) Scan parquet
+ HashAggregate (36)
+ +- HashAggregate (35)
+ +- Project (34)
+ +- SortMergeJoin Inner (33)
+ :- Sort (28)
+ : +- Exchange (27)
+ : +- Project (26)
+ : +- Filter (25)
+ : +- Scan parquet (24)
+ +- Sort (32)
+ +- Exchange (31)
+ +- Filter (30)
+ +- Scan parquet (29)
+
+
+(1) ScanTransformer parquet
Output [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -73,7 +72,7 @@ Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X]
(9) InputIteratorTransformer
Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -122,89 +121,82 @@ Input [8]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, p_partke
(21) RegularHashAggregateExecTransformer
Input [3]: [l_extendedprice#X, l_discount#X, _pre_X#X]
Keys: []
-Functions [1]: [partial_sum(_pre_X#X)]
-Aggregate Attributes [2]: [sum#X, isEmpty#X]
-Results [2]: [sum#X, isEmpty#X]
-
-(22) RegularHashAggregateExecTransformer
-Input [2]: [sum#X, isEmpty#X]
-Keys: []
-Functions [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))]
+Functions [1]: [sum(_pre_X#X)]
Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X]
Results [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X AS revenue#X]
-(23) WholeStageCodegenTransformer (X)
+(22) WholeStageCodegenTransformer (X)
Input [1]: [revenue#X]
Arguments: false
-(24) VeloxColumnarToRow
+(23) VeloxColumnarToRow
Input [1]: [revenue#X]
-(25) Scan parquet
+(24) Scan parquet
Output [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(l_shipinstruct), In(l_shipmode, [AIR,AIR REG]), EqualTo(l_shipinstruct,DELIVER IN PERSON), IsNotNull(l_partkey), Or(Or(And(GreaterThanOrEqual(l_quantity,1.00),LessThanOrEqual(l_quantity,11.00)),And(GreaterThanOrEqual(l_quantity,10.00),LessThanOrEqual(l_quantity,20.00))),And(GreaterThanOrEqual(l_quantity,20.00),LessThanOrEqual(l_quantity,30.00)))]
ReadSchema: struct
-(26) Filter
+(25) Filter
Input [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X]
Condition : ((((isnotnull(l_shipinstruct#X) AND l_shipmode#X IN (AIR,AIR REG)) AND (l_shipinstruct#X = DELIVER IN PERSON)) AND isnotnull(l_partkey#X)) AND ((((l_quantity#X >= 1.00) AND (l_quantity#X <= 11.00)) OR ((l_quantity#X >= 10.00) AND (l_quantity#X <= 20.00))) OR ((l_quantity#X >= 20.00) AND (l_quantity#X <= 30.00))))
-(27) Project
+(26) Project
Output [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X]
Input [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X]
-(28) Exchange
+(27) Exchange
Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X]
Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(29) Sort
+(28) Sort
Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X]
Arguments: [l_partkey#X ASC NULLS FIRST], false, 0
-(30) Scan parquet
+(29) Scan parquet
Output [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(p_size), GreaterThanOrEqual(p_size,1), IsNotNull(p_partkey), Or(Or(And(And(EqualTo(p_brand,Brand#X),In(p_container, [SM BOX,SM CASE,SM PACK,SM PKG])),LessThanOrEqual(p_size,5)),And(And(EqualTo(p_brand,Brand#X),In(p_container, [MED BAG,MED BOX,MED PACK,MED PKG])),LessThanOrEqual(p_size,10))),And(And(EqualTo(p_brand,Brand#X),In(p_container, [LG BOX,LG CASE,LG PACK,LG PKG])),LessThanOrEqual(p_size,15)))]
ReadSchema: struct
-(31) Filter
+(30) Filter
Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X]
Condition : (((isnotnull(p_size#X) AND (p_size#X >= 1)) AND isnotnull(p_partkey#X)) AND (((((p_brand#X = Brand#X) AND p_container#X IN (SM CASE,SM BOX,SM PACK,SM PKG)) AND (p_size#X <= 5)) OR (((p_brand#X = Brand#X) AND p_container#X IN (MED BAG,MED BOX,MED PKG,MED PACK)) AND (p_size#X <= 10))) OR (((p_brand#X = Brand#X) AND p_container#X IN (LG CASE,LG BOX,LG PACK,LG PKG)) AND (p_size#X <= 15))))
-(32) Exchange
+(31) Exchange
Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X]
Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(33) Sort
+(32) Sort
Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X]
Arguments: [p_partkey#X ASC NULLS FIRST], false, 0
-(34) SortMergeJoin
+(33) SortMergeJoin
Left keys [1]: [l_partkey#X]
Right keys [1]: [p_partkey#X]
Join condition: (((((((p_brand#X = Brand#X) AND p_container#X IN (SM CASE,SM BOX,SM PACK,SM PKG)) AND (l_quantity#X >= 1.00)) AND (l_quantity#X <= 11.00)) AND (p_size#X <= 5)) OR (((((p_brand#X = Brand#X) AND p_container#X IN (MED BAG,MED BOX,MED PKG,MED PACK)) AND (l_quantity#X >= 10.00)) AND (l_quantity#X <= 20.00)) AND (p_size#X <= 10))) OR (((((p_brand#X = Brand#X) AND p_container#X IN (LG CASE,LG BOX,LG PACK,LG PKG)) AND (l_quantity#X >= 20.00)) AND (l_quantity#X <= 30.00)) AND (p_size#X <= 15)))
-(35) Project
+(34) Project
Output [2]: [l_extendedprice#X, l_discount#X]
Input [8]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, p_partkey#X, p_brand#X, p_size#X, p_container#X]
-(36) HashAggregate
+(35) HashAggregate
Input [2]: [l_extendedprice#X, l_discount#X]
Keys: []
Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))]
Aggregate Attributes [2]: [sum#X, isEmpty#X]
Results [2]: [sum#X, isEmpty#X]
-(37) HashAggregate
+(36) HashAggregate
Input [2]: [sum#X, isEmpty#X]
Keys: []
Functions [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))]
Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X]
Results [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X AS revenue#X]
-(38) AdaptiveSparkPlan
+(37) AdaptiveSparkPlan
Output [1]: [revenue#X]
Arguments: isFinalPlan=true
\ No newline at end of file
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 6fa1271a5a39..13c10a7bba29 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
@@ -1,33 +1,33 @@
== Physical Plan ==
-AdaptiveSparkPlan (146)
+AdaptiveSparkPlan (145)
+- == Final Plan ==
- VeloxColumnarToRow (96)
- +- ^ SortExecTransformer (94)
- +- ^ InputIteratorTransformer (93)
- +- ShuffleQueryStage (91)
- +- ColumnarExchange (90)
- +- VeloxResizeBatches (89)
- +- ^ ProjectExecTransformer (87)
- +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (86)
- :- ^ InputIteratorTransformer (76)
- : +- ShuffleQueryStage (74)
- : +- ColumnarExchange (73)
- : +- VeloxResizeBatches (72)
- : +- ^ ProjectExecTransformer (70)
- : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (69)
+ VeloxColumnarToRow (95)
+ +- ^ SortExecTransformer (93)
+ +- ^ InputIteratorTransformer (92)
+ +- ShuffleQueryStage (90)
+ +- ColumnarExchange (89)
+ +- VeloxResizeBatches (88)
+ +- ^ ProjectExecTransformer (86)
+ +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (85)
+ :- ^ InputIteratorTransformer (75)
+ : +- ShuffleQueryStage (73)
+ : +- ColumnarExchange (72)
+ : +- VeloxResizeBatches (71)
+ : +- ^ ProjectExecTransformer (69)
+ : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (68)
: :- ^ InputIteratorTransformer (9)
: : +- ShuffleQueryStage (7)
: : +- ColumnarExchange (6)
: : +- VeloxResizeBatches (5)
: : +- ^ ProjectExecTransformer (3)
: : +- ^ FilterExecTransformer (2)
- : : +- ^ Scan parquet (1)
- : +- ^ InputIteratorTransformer (68)
- : +- ShuffleQueryStage (66)
- : +- ColumnarExchange (65)
- : +- VeloxResizeBatches (64)
- : +- ^ ProjectExecTransformer (62)
- : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (61)
+ : : +- ^ ScanTransformer parquet (1)
+ : +- ^ InputIteratorTransformer (67)
+ : +- ShuffleQueryStage (65)
+ : +- ColumnarExchange (64)
+ : +- VeloxResizeBatches (63)
+ : +- ^ ProjectExecTransformer (61)
+ : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (60)
: :- ^ InputIteratorTransformer (35)
: : +- ShuffleQueryStage (33)
: : +- ColumnarExchange (32)
@@ -40,94 +40,93 @@ AdaptiveSparkPlan (146)
: : : +- VeloxResizeBatches (14)
: : : +- ^ ProjectExecTransformer (12)
: : : +- ^ FilterExecTransformer (11)
- : : : +- ^ Scan parquet (10)
+ : : : +- ^ ScanTransformer parquet (10)
: : +- ^ InputIteratorTransformer (27)
: : +- ShuffleQueryStage (25)
: : +- ColumnarExchange (24)
: : +- VeloxResizeBatches (23)
: : +- ^ ProjectExecTransformer (21)
: : +- ^ FilterExecTransformer (20)
- : : +- ^ Scan parquet (19)
- : +- ^ InputIteratorTransformer (60)
- : +- ShuffleQueryStage (58)
- : +- ColumnarExchange (57)
- : +- VeloxResizeBatches (56)
- : +- ^ ProjectExecTransformer (54)
- : +- ^ FilterExecTransformer (53)
- : +- ^ ProjectExecTransformer (52)
- : +- ^ RegularHashAggregateExecTransformer (51)
- : +- ^ RegularHashAggregateExecTransformer (50)
- : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (49)
- : :- ^ InputIteratorTransformer (44)
- : : +- ShuffleQueryStage (42)
- : : +- ColumnarExchange (41)
- : : +- VeloxResizeBatches (40)
- : : +- ^ ProjectExecTransformer (38)
- : : +- ^ FilterExecTransformer (37)
- : : +- ^ Scan parquet (36)
- : +- ^ InputIteratorTransformer (48)
- : +- ShuffleQueryStage (46)
- : +- ReusedExchange (45)
- +- ^ InputIteratorTransformer (85)
- +- ShuffleQueryStage (83)
- +- ColumnarExchange (82)
- +- VeloxResizeBatches (81)
- +- ^ ProjectExecTransformer (79)
- +- ^ FilterExecTransformer (78)
- +- ^ Scan parquet (77)
+ : : +- ^ ScanTransformer parquet (19)
+ : +- ^ InputIteratorTransformer (59)
+ : +- ShuffleQueryStage (57)
+ : +- ColumnarExchange (56)
+ : +- VeloxResizeBatches (55)
+ : +- ^ ProjectExecTransformer (53)
+ : +- ^ FilterExecTransformer (52)
+ : +- ^ ProjectExecTransformer (51)
+ : +- ^ RegularHashAggregateExecTransformer (50)
+ : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (49)
+ : :- ^ InputIteratorTransformer (44)
+ : : +- ShuffleQueryStage (42)
+ : : +- ColumnarExchange (41)
+ : : +- VeloxResizeBatches (40)
+ : : +- ^ ProjectExecTransformer (38)
+ : : +- ^ FilterExecTransformer (37)
+ : : +- ^ ScanTransformer parquet (36)
+ : +- ^ InputIteratorTransformer (48)
+ : +- ShuffleQueryStage (46)
+ : +- ReusedExchange (45)
+ +- ^ InputIteratorTransformer (84)
+ +- ShuffleQueryStage (82)
+ +- ColumnarExchange (81)
+ +- VeloxResizeBatches (80)
+ +- ^ ProjectExecTransformer (78)
+ +- ^ FilterExecTransformer (77)
+ +- ^ ScanTransformer parquet (76)
+- == Initial Plan ==
- Sort (145)
- +- Exchange (144)
- +- Project (143)
- +- SortMergeJoin Inner (142)
- :- Sort (136)
- : +- Exchange (135)
- : +- Project (134)
- : +- SortMergeJoin LeftSemi (133)
- : :- Sort (100)
- : : +- Exchange (99)
- : : +- Filter (98)
- : : +- Scan parquet (97)
- : +- Sort (132)
- : +- Exchange (131)
- : +- Project (130)
- : +- SortMergeJoin Inner (129)
- : :- Sort (112)
- : : +- Exchange (111)
- : : +- SortMergeJoin LeftSemi (110)
- : : :- Sort (104)
- : : : +- Exchange (103)
- : : : +- Filter (102)
- : : : +- Scan parquet (101)
- : : +- Sort (109)
- : : +- Exchange (108)
- : : +- Project (107)
- : : +- Filter (106)
- : : +- Scan parquet (105)
- : +- Sort (128)
- : +- Exchange (127)
- : +- Filter (126)
- : +- HashAggregate (125)
- : +- HashAggregate (124)
- : +- SortMergeJoin LeftSemi (123)
- : :- Sort (117)
- : : +- Exchange (116)
- : : +- Project (115)
- : : +- Filter (114)
- : : +- Scan parquet (113)
- : +- Sort (122)
- : +- Exchange (121)
- : +- Project (120)
- : +- Filter (119)
- : +- Scan parquet (118)
- +- Sort (141)
- +- Exchange (140)
- +- Project (139)
- +- Filter (138)
- +- Scan parquet (137)
-
-
-(1) Scan parquet
+ Sort (144)
+ +- Exchange (143)
+ +- Project (142)
+ +- SortMergeJoin Inner (141)
+ :- Sort (135)
+ : +- Exchange (134)
+ : +- Project (133)
+ : +- SortMergeJoin LeftSemi (132)
+ : :- Sort (99)
+ : : +- Exchange (98)
+ : : +- Filter (97)
+ : : +- Scan parquet (96)
+ : +- Sort (131)
+ : +- Exchange (130)
+ : +- Project (129)
+ : +- SortMergeJoin Inner (128)
+ : :- Sort (111)
+ : : +- Exchange (110)
+ : : +- SortMergeJoin LeftSemi (109)
+ : : :- Sort (103)
+ : : : +- Exchange (102)
+ : : : +- Filter (101)
+ : : : +- Scan parquet (100)
+ : : +- Sort (108)
+ : : +- Exchange (107)
+ : : +- Project (106)
+ : : +- Filter (105)
+ : : +- Scan parquet (104)
+ : +- Sort (127)
+ : +- Exchange (126)
+ : +- Filter (125)
+ : +- HashAggregate (124)
+ : +- HashAggregate (123)
+ : +- SortMergeJoin LeftSemi (122)
+ : :- Sort (116)
+ : : +- Exchange (115)
+ : : +- Project (114)
+ : : +- Filter (113)
+ : : +- Scan parquet (112)
+ : +- Sort (121)
+ : +- Exchange (120)
+ : +- Project (119)
+ : +- Filter (118)
+ : +- Scan parquet (117)
+ +- Sort (140)
+ +- Exchange (139)
+ +- Project (138)
+ +- Filter (137)
+ +- Scan parquet (136)
+
+
+(1) ScanTransformer parquet
Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -164,7 +163,7 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X]
(9) InputIteratorTransformer
Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -201,7 +200,7 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
(18) InputIteratorTransformer
Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
-(19) Scan parquet
+(19) ScanTransformer parquet
Output [2]: [p_partkey#X, p_name#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -269,7 +268,7 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
(35) InputIteratorTransformer
Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
-(36) Scan parquet
+(36) ScanTransformer parquet
Output [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -327,417 +326,410 @@ Join condition: None
(50) RegularHashAggregateExecTransformer
Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X]
Keys [2]: [l_partkey#X, l_suppkey#X]
-Functions [1]: [partial_sum(l_quantity#X)]
-Aggregate Attributes [2]: [sum#X, isEmpty#X]
-Results [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X]
-
-(51) RegularHashAggregateExecTransformer
-Input [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X]
-Keys [2]: [l_partkey#X, l_suppkey#X]
Functions [1]: [sum(l_quantity#X)]
Aggregate Attributes [1]: [sum(l_quantity#X)#X]
Results [3]: [l_partkey#X, l_suppkey#X, sum(l_quantity#X)#X]
-(52) ProjectExecTransformer
+(51) ProjectExecTransformer
Output [3]: [CheckOverflow((0.50 * promote_precision(sum(l_quantity#X)#X)), DecimalType(24,3), true) AS (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
Input [3]: [l_partkey#X, l_suppkey#X, sum(l_quantity#X)#X]
-(53) FilterExecTransformer
+(52) FilterExecTransformer
Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
Arguments: isnotnull((0.5 * sum(l_quantity))#X)
-(54) ProjectExecTransformer
+(53) ProjectExecTransformer
Output [4]: [hash(l_partkey#X, l_suppkey#X, 42) AS hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
-(55) WholeStageCodegenTransformer (X)
+(54) WholeStageCodegenTransformer (X)
Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
Arguments: false
-(56) VeloxResizeBatches
+(55) VeloxResizeBatches
Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
Arguments: X, X
-(57) ColumnarExchange
+(56) ColumnarExchange
Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
Arguments: hashpartitioning(l_partkey#X, l_suppkey#X, 1), ENSURE_REQUIREMENTS, [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X], [plan_id=X], [shuffle_writer_type=hash]
-(58) ShuffleQueryStage
+(57) ShuffleQueryStage
Output [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
Arguments: X
-(59) InputAdapter
+(58) InputAdapter
Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
-(60) InputIteratorTransformer
+(59) InputIteratorTransformer
Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
-(61) ShuffledHashJoinExecTransformer
+(60) ShuffledHashJoinExecTransformer
Left keys [2]: [ps_partkey#X, ps_suppkey#X]
Right keys [2]: [l_partkey#X, l_suppkey#X]
Join condition: (cast(cast(ps_availqty#X as decimal(10,0)) as decimal(24,3)) > (0.5 * sum(l_quantity))#X)
-(62) ProjectExecTransformer
+(61) ProjectExecTransformer
Output [2]: [hash(ps_suppkey#X, 42) AS hash_partition_key#X, ps_suppkey#X]
Input [6]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
-(63) WholeStageCodegenTransformer (X)
+(62) WholeStageCodegenTransformer (X)
Input [2]: [hash_partition_key#X, ps_suppkey#X]
Arguments: false
-(64) VeloxResizeBatches
+(63) VeloxResizeBatches
Input [2]: [hash_partition_key#X, ps_suppkey#X]
Arguments: X, X
-(65) ColumnarExchange
+(64) ColumnarExchange
Input [2]: [hash_partition_key#X, ps_suppkey#X]
Arguments: hashpartitioning(ps_suppkey#X, 1), ENSURE_REQUIREMENTS, [ps_suppkey#X], [plan_id=X], [shuffle_writer_type=hash]
-(66) ShuffleQueryStage
+(65) ShuffleQueryStage
Output [1]: [ps_suppkey#X]
Arguments: X
-(67) InputAdapter
+(66) InputAdapter
Input [1]: [ps_suppkey#X]
-(68) InputIteratorTransformer
+(67) InputIteratorTransformer
Input [1]: [ps_suppkey#X]
-(69) ShuffledHashJoinExecTransformer
+(68) ShuffledHashJoinExecTransformer
Left keys [1]: [s_suppkey#X]
Right keys [1]: [ps_suppkey#X]
Join condition: None
-(70) ProjectExecTransformer
+(69) ProjectExecTransformer
Output [4]: [hash(s_nationkey#X, 42) AS hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X]
Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X]
-(71) WholeStageCodegenTransformer (X)
+(70) WholeStageCodegenTransformer (X)
Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X]
Arguments: false
-(72) VeloxResizeBatches
+(71) VeloxResizeBatches
Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X]
Arguments: X, X
-(73) ColumnarExchange
+(72) ColumnarExchange
Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X]
Arguments: hashpartitioning(s_nationkey#X, 1), ENSURE_REQUIREMENTS, [s_name#X, s_address#X, s_nationkey#X], [plan_id=X], [shuffle_writer_type=hash]
-(74) ShuffleQueryStage
+(73) ShuffleQueryStage
Output [3]: [s_name#X, s_address#X, s_nationkey#X]
Arguments: X
-(75) InputAdapter
+(74) InputAdapter
Input [3]: [s_name#X, s_address#X, s_nationkey#X]
-(76) InputIteratorTransformer
+(75) InputIteratorTransformer
Input [3]: [s_name#X, s_address#X, s_nationkey#X]
-(77) Scan parquet
+(76) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(n_name), EqualTo(n_name,CANADA), IsNotNull(n_nationkey)]
ReadSchema: struct
-(78) FilterExecTransformer
+(77) FilterExecTransformer
Input [2]: [n_nationkey#X, n_name#X]
Arguments: ((isnotnull(n_name#X) AND (n_name#X = CANADA)) AND isnotnull(n_nationkey#X))
-(79) ProjectExecTransformer
+(78) ProjectExecTransformer
Output [2]: [hash(n_nationkey#X, 42) AS hash_partition_key#X, n_nationkey#X]
Input [2]: [n_nationkey#X, n_name#X]
-(80) WholeStageCodegenTransformer (X)
+(79) WholeStageCodegenTransformer (X)
Input [2]: [hash_partition_key#X, n_nationkey#X]
Arguments: false
-(81) VeloxResizeBatches
+(80) VeloxResizeBatches
Input [2]: [hash_partition_key#X, n_nationkey#X]
Arguments: X, X
-(82) ColumnarExchange
+(81) ColumnarExchange
Input [2]: [hash_partition_key#X, n_nationkey#X]
Arguments: hashpartitioning(n_nationkey#X, 1), ENSURE_REQUIREMENTS, [n_nationkey#X], [plan_id=X], [shuffle_writer_type=hash]
-(83) ShuffleQueryStage
+(82) ShuffleQueryStage
Output [1]: [n_nationkey#X]
Arguments: X
-(84) InputAdapter
+(83) InputAdapter
Input [1]: [n_nationkey#X]
-(85) InputIteratorTransformer
+(84) InputIteratorTransformer
Input [1]: [n_nationkey#X]
-(86) ShuffledHashJoinExecTransformer
+(85) ShuffledHashJoinExecTransformer
Left keys [1]: [s_nationkey#X]
Right keys [1]: [n_nationkey#X]
Join condition: None
-(87) ProjectExecTransformer
+(86) ProjectExecTransformer
Output [2]: [s_name#X, s_address#X]
Input [4]: [s_name#X, s_address#X, s_nationkey#X, n_nationkey#X]
-(88) WholeStageCodegenTransformer (X)
+(87) WholeStageCodegenTransformer (X)
Input [2]: [s_name#X, s_address#X]
Arguments: false
-(89) VeloxResizeBatches
+(88) VeloxResizeBatches
Input [2]: [s_name#X, s_address#X]
Arguments: X, X
-(90) ColumnarExchange
+(89) ColumnarExchange
Input [2]: [s_name#X, s_address#X]
Arguments: rangepartitioning(s_name#X ASC NULLS FIRST, 1), ENSURE_REQUIREMENTS, [plan_id=X], [shuffle_writer_type=hash]
-(91) ShuffleQueryStage
+(90) ShuffleQueryStage
Output [2]: [s_name#X, s_address#X]
Arguments: X
-(92) InputAdapter
+(91) InputAdapter
Input [2]: [s_name#X, s_address#X]
-(93) InputIteratorTransformer
+(92) InputIteratorTransformer
Input [2]: [s_name#X, s_address#X]
-(94) SortExecTransformer
+(93) SortExecTransformer
Input [2]: [s_name#X, s_address#X]
Arguments: [s_name#X ASC NULLS FIRST], true, 0
-(95) WholeStageCodegenTransformer (X)
+(94) WholeStageCodegenTransformer (X)
Input [2]: [s_name#X, s_address#X]
Arguments: false
-(96) VeloxColumnarToRow
+(95) VeloxColumnarToRow
Input [2]: [s_name#X, s_address#X]
-(97) Scan parquet
+(96) Scan parquet
Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(s_nationkey)]
ReadSchema: struct
-(98) Filter
+(97) Filter
Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X]
Condition : isnotnull(s_nationkey#X)
-(99) Exchange
+(98) Exchange
Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X]
Arguments: hashpartitioning(s_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(100) Sort
+(99) Sort
Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X]
Arguments: [s_suppkey#X ASC NULLS FIRST], false, 0
-(101) Scan parquet
+(100) Scan parquet
Output [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(ps_availqty), IsNotNull(ps_partkey), IsNotNull(ps_suppkey)]
ReadSchema: struct
-(102) Filter
+(101) Filter
Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
Condition : ((isnotnull(ps_availqty#X) AND isnotnull(ps_partkey#X)) AND isnotnull(ps_suppkey#X))
-(103) Exchange
+(102) Exchange
Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
Arguments: hashpartitioning(ps_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(104) Sort
+(103) Sort
Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
Arguments: [ps_partkey#X ASC NULLS FIRST], false, 0
-(105) Scan parquet
+(104) Scan parquet
Output [2]: [p_partkey#X, p_name#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(p_name), StringStartsWith(p_name,forest)]
ReadSchema: struct
-(106) Filter
+(105) Filter
Input [2]: [p_partkey#X, p_name#X]
Condition : (isnotnull(p_name#X) AND StartsWith(p_name#X, forest))
-(107) Project
+(106) Project
Output [1]: [p_partkey#X]
Input [2]: [p_partkey#X, p_name#X]
-(108) Exchange
+(107) Exchange
Input [1]: [p_partkey#X]
Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(109) Sort
+(108) Sort
Input [1]: [p_partkey#X]
Arguments: [p_partkey#X ASC NULLS FIRST], false, 0
-(110) SortMergeJoin
+(109) SortMergeJoin
Left keys [1]: [ps_partkey#X]
Right keys [1]: [p_partkey#X]
Join condition: None
-(111) Exchange
+(110) Exchange
Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
Arguments: hashpartitioning(ps_partkey#X, ps_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(112) Sort
+(111) Sort
Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
Arguments: [ps_partkey#X ASC NULLS FIRST, ps_suppkey#X ASC NULLS FIRST], false, 0
-(113) Scan parquet
+(112) Scan parquet
Output [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1994-01-01), LessThan(l_shipdate,1995-01-01), IsNotNull(l_partkey), IsNotNull(l_suppkey)]
ReadSchema: struct
-(114) Filter
+(113) Filter
Input [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X]
Condition : ((((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1994-01-01)) AND (l_shipdate#X < 1995-01-01)) AND isnotnull(l_partkey#X)) AND isnotnull(l_suppkey#X))
-(115) Project
+(114) Project
Output [3]: [l_partkey#X, l_suppkey#X, l_quantity#X]
Input [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X]
-(116) Exchange
+(115) Exchange
Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X]
Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(117) Sort
+(116) Sort
Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X]
Arguments: [l_partkey#X ASC NULLS FIRST], false, 0
-(118) Scan parquet
+(117) Scan parquet
Output [2]: [p_partkey#X, p_name#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(p_name), StringStartsWith(p_name,forest)]
ReadSchema: struct
-(119) Filter
+(118) Filter
Input [2]: [p_partkey#X, p_name#X]
Condition : (isnotnull(p_name#X) AND StartsWith(p_name#X, forest))
-(120) Project
+(119) Project
Output [1]: [p_partkey#X]
Input [2]: [p_partkey#X, p_name#X]
-(121) Exchange
+(120) Exchange
Input [1]: [p_partkey#X]
Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(122) Sort
+(121) Sort
Input [1]: [p_partkey#X]
Arguments: [p_partkey#X ASC NULLS FIRST], false, 0
-(123) SortMergeJoin
+(122) SortMergeJoin
Left keys [1]: [l_partkey#X]
Right keys [1]: [p_partkey#X]
Join condition: None
-(124) HashAggregate
+(123) HashAggregate
Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X]
Keys [2]: [l_partkey#X, l_suppkey#X]
Functions [1]: [partial_sum(l_quantity#X)]
Aggregate Attributes [2]: [sum#X, isEmpty#X]
Results [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X]
-(125) HashAggregate
+(124) HashAggregate
Input [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X]
Keys [2]: [l_partkey#X, l_suppkey#X]
Functions [1]: [sum(l_quantity#X)]
Aggregate Attributes [1]: [sum(l_quantity#X)#X]
Results [3]: [CheckOverflow((0.50 * promote_precision(sum(l_quantity#X)#X)), DecimalType(24,3), true) AS (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
-(126) Filter
+(125) Filter
Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
Condition : isnotnull((0.5 * sum(l_quantity))#X)
-(127) Exchange
+(126) Exchange
Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
Arguments: hashpartitioning(l_partkey#X, l_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(128) Sort
+(127) Sort
Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
Arguments: [l_partkey#X ASC NULLS FIRST, l_suppkey#X ASC NULLS FIRST], false, 0
-(129) SortMergeJoin
+(128) SortMergeJoin
Left keys [2]: [ps_partkey#X, ps_suppkey#X]
Right keys [2]: [l_partkey#X, l_suppkey#X]
Join condition: (cast(cast(ps_availqty#X as decimal(10,0)) as decimal(24,3)) > (0.5 * sum(l_quantity))#X)
-(130) Project
+(129) Project
Output [1]: [ps_suppkey#X]
Input [6]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
-(131) Exchange
+(130) Exchange
Input [1]: [ps_suppkey#X]
Arguments: hashpartitioning(ps_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(132) Sort
+(131) Sort
Input [1]: [ps_suppkey#X]
Arguments: [ps_suppkey#X ASC NULLS FIRST], false, 0
-(133) SortMergeJoin
+(132) SortMergeJoin
Left keys [1]: [s_suppkey#X]
Right keys [1]: [ps_suppkey#X]
Join condition: None
-(134) Project
+(133) Project
Output [3]: [s_name#X, s_address#X, s_nationkey#X]
Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X]
-(135) Exchange
+(134) Exchange
Input [3]: [s_name#X, s_address#X, s_nationkey#X]
Arguments: hashpartitioning(s_nationkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(136) Sort
+(135) Sort
Input [3]: [s_name#X, s_address#X, s_nationkey#X]
Arguments: [s_nationkey#X ASC NULLS FIRST], false, 0
-(137) Scan parquet
+(136) Scan parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(n_name), EqualTo(n_name,CANADA), IsNotNull(n_nationkey)]
ReadSchema: struct
-(138) Filter
+(137) Filter
Input [2]: [n_nationkey#X, n_name#X]
Condition : ((isnotnull(n_name#X) AND (n_name#X = CANADA)) AND isnotnull(n_nationkey#X))
-(139) Project
+(138) Project
Output [1]: [n_nationkey#X]
Input [2]: [n_nationkey#X, n_name#X]
-(140) Exchange
+(139) Exchange
Input [1]: [n_nationkey#X]
Arguments: hashpartitioning(n_nationkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(141) Sort
+(140) Sort
Input [1]: [n_nationkey#X]
Arguments: [n_nationkey#X ASC NULLS FIRST], false, 0
-(142) SortMergeJoin
+(141) SortMergeJoin
Left keys [1]: [s_nationkey#X]
Right keys [1]: [n_nationkey#X]
Join condition: None
-(143) Project
+(142) Project
Output [2]: [s_name#X, s_address#X]
Input [4]: [s_name#X, s_address#X, s_nationkey#X, n_nationkey#X]
-(144) Exchange
+(143) Exchange
Input [2]: [s_name#X, s_address#X]
Arguments: rangepartitioning(s_name#X ASC NULLS FIRST, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(145) Sort
+(144) Sort
Input [2]: [s_name#X, s_address#X]
Arguments: [s_name#X ASC NULLS FIRST], true, 0
-(146) AdaptiveSparkPlan
+(145) AdaptiveSparkPlan
Output [2]: [s_name#X, s_address#X]
Arguments: isFinalPlan=true
\ No newline at end of file
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 e70a145e9c1d..e72762dd9bfe 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
@@ -30,7 +30,7 @@ AdaptiveSparkPlan (138)
: : : +- VeloxResizeBatches (5)
: : : +- ^ ProjectExecTransformer (3)
: : : +- ^ FilterExecTransformer (2)
- : : : +- ^ Scan parquet (1)
+ : : : +- ^ ScanTransformer parquet (1)
: : +- ^ InputIteratorTransformer (44)
: : +- ShuffleQueryStage (42)
: : +- ColumnarExchange (41)
@@ -44,34 +44,34 @@ AdaptiveSparkPlan (138)
: : : : +- VeloxResizeBatches (14)
: : : : +- ^ ProjectExecTransformer (12)
: : : : +- ^ FilterExecTransformer (11)
- : : : : +- ^ Scan parquet (10)
+ : : : : +- ^ ScanTransformer parquet (10)
: : : +- ^ InputIteratorTransformer (26)
: : : +- ShuffleQueryStage (24)
: : : +- ColumnarExchange (23)
: : : +- VeloxResizeBatches (22)
: : : +- ^ ProjectExecTransformer (20)
- : : : +- ^ Scan parquet (19)
+ : : : +- ^ ScanTransformer parquet (19)
: : +- ^ InputIteratorTransformer (36)
: : +- ShuffleQueryStage (34)
: : +- ColumnarExchange (33)
: : +- VeloxResizeBatches (32)
: : +- ^ ProjectExecTransformer (30)
: : +- ^ FilterExecTransformer (29)
- : : +- ^ Scan parquet (28)
+ : : +- ^ ScanTransformer parquet (28)
: +- ^ InputIteratorTransformer (61)
: +- ShuffleQueryStage (59)
: +- ColumnarExchange (58)
: +- VeloxResizeBatches (57)
: +- ^ ProjectExecTransformer (55)
: +- ^ FilterExecTransformer (54)
- : +- ^ Scan parquet (53)
+ : +- ^ ScanTransformer parquet (53)
+- ^ InputIteratorTransformer (78)
+- ShuffleQueryStage (76)
+- ColumnarExchange (75)
+- VeloxResizeBatches (74)
+- ^ ProjectExecTransformer (72)
+- ^ FilterExecTransformer (71)
- +- ^ Scan parquet (70)
+ +- ^ ScanTransformer parquet (70)
+- == Initial Plan ==
TakeOrderedAndProject (137)
+- HashAggregate (136)
@@ -120,7 +120,7 @@ AdaptiveSparkPlan (138)
+- Scan parquet (127)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [3]: [s_suppkey#X, s_name#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -157,7 +157,7 @@ Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X]
(9) InputIteratorTransformer
Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -194,7 +194,7 @@ Input [2]: [l_orderkey#X, l_suppkey#X]
(18) InputIteratorTransformer
Input [2]: [l_orderkey#X, l_suppkey#X]
-(19) Scan parquet
+(19) ScanTransformer parquet
Output [2]: [l_orderkey#X, l_suppkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -231,7 +231,7 @@ Left keys [1]: [l_orderkey#X]
Right keys [1]: [l_orderkey#X]
Join condition: NOT (l_suppkey#X = l_suppkey#X)
-(28) Scan parquet
+(28) ScanTransformer parquet
Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -330,7 +330,7 @@ Input [3]: [s_name#X, s_nationkey#X, l_orderkey#X]
(52) InputIteratorTransformer
Input [3]: [s_name#X, s_nationkey#X, l_orderkey#X]
-(53) Scan parquet
+(53) ScanTransformer parquet
Output [2]: [o_orderkey#X, o_orderstatus#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -398,7 +398,7 @@ Input [2]: [s_name#X, s_nationkey#X]
(69) InputIteratorTransformer
Input [2]: [s_name#X, s_nationkey#X]
-(70) Scan parquet
+(70) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 674e0b96983f..81af91c0ee2a 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
@@ -22,13 +22,13 @@ AdaptiveSparkPlan (52)
: +- VeloxResizeBatches (5)
: +- ^ ProjectExecTransformer (3)
: +- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
+ : +- ^ ScanTransformer parquet (1)
+- ^ InputIteratorTransformer (17)
+- ShuffleQueryStage (15)
+- ColumnarExchange (14)
+- VeloxResizeBatches (13)
+- ^ ProjectExecTransformer (11)
- +- ^ Scan parquet (10)
+ +- ^ ScanTransformer parquet (10)
+- == Initial Plan ==
Sort (51)
+- Exchange (50)
@@ -46,7 +46,7 @@ AdaptiveSparkPlan (52)
+- Scan parquet (42)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [3]: [c_custkey#X, c_phone#X, c_acctbal#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -83,7 +83,7 @@ Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X]
(9) InputIteratorTransformer
Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [1]: [o_custkey#X]
Batched: true
Location: InMemoryFileIndex [*]
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 3b9b7fd93b96..2e427b6bf3ca 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
@@ -1,67 +1,66 @@
== Physical Plan ==
-AdaptiveSparkPlan (67)
+AdaptiveSparkPlan (66)
+- == Final Plan ==
- VeloxColumnarToRow (43)
- +- TakeOrderedAndProjectExecTransformer (42)
- +- ^ ProjectExecTransformer (40)
- +- ^ RegularHashAggregateExecTransformer (39)
- +- ^ RegularHashAggregateExecTransformer (38)
- +- ^ ProjectExecTransformer (37)
- +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (36)
- :- ^ InputIteratorTransformer (26)
- : +- ShuffleQueryStage (24)
- : +- ColumnarExchange (23)
- : +- VeloxResizeBatches (22)
- : +- ^ ProjectExecTransformer (20)
- : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19)
- : :- ^ InputIteratorTransformer (9)
- : : +- ShuffleQueryStage (7)
- : : +- ColumnarExchange (6)
- : : +- VeloxResizeBatches (5)
- : : +- ^ ProjectExecTransformer (3)
- : : +- ^ FilterExecTransformer (2)
- : : +- ^ Scan parquet (1)
- : +- ^ InputIteratorTransformer (18)
- : +- ShuffleQueryStage (16)
- : +- ColumnarExchange (15)
- : +- VeloxResizeBatches (14)
- : +- ^ ProjectExecTransformer (12)
- : +- ^ FilterExecTransformer (11)
- : +- ^ Scan parquet (10)
- +- ^ InputIteratorTransformer (35)
- +- ShuffleQueryStage (33)
- +- ColumnarExchange (32)
- +- VeloxResizeBatches (31)
- +- ^ ProjectExecTransformer (29)
- +- ^ FilterExecTransformer (28)
- +- ^ Scan parquet (27)
+ VeloxColumnarToRow (42)
+ +- TakeOrderedAndProjectExecTransformer (41)
+ +- ^ ProjectExecTransformer (39)
+ +- ^ RegularHashAggregateExecTransformer (38)
+ +- ^ ProjectExecTransformer (37)
+ +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (36)
+ :- ^ InputIteratorTransformer (26)
+ : +- ShuffleQueryStage (24)
+ : +- ColumnarExchange (23)
+ : +- VeloxResizeBatches (22)
+ : +- ^ ProjectExecTransformer (20)
+ : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19)
+ : :- ^ InputIteratorTransformer (9)
+ : : +- ShuffleQueryStage (7)
+ : : +- ColumnarExchange (6)
+ : : +- VeloxResizeBatches (5)
+ : : +- ^ ProjectExecTransformer (3)
+ : : +- ^ FilterExecTransformer (2)
+ : : +- ^ ScanTransformer parquet (1)
+ : +- ^ InputIteratorTransformer (18)
+ : +- ShuffleQueryStage (16)
+ : +- ColumnarExchange (15)
+ : +- VeloxResizeBatches (14)
+ : +- ^ ProjectExecTransformer (12)
+ : +- ^ FilterExecTransformer (11)
+ : +- ^ ScanTransformer parquet (10)
+ +- ^ InputIteratorTransformer (35)
+ +- ShuffleQueryStage (33)
+ +- ColumnarExchange (32)
+ +- VeloxResizeBatches (31)
+ +- ^ ProjectExecTransformer (29)
+ +- ^ FilterExecTransformer (28)
+ +- ^ ScanTransformer parquet (27)
+- == Initial Plan ==
- TakeOrderedAndProject (66)
- +- HashAggregate (65)
- +- HashAggregate (64)
- +- Project (63)
- +- SortMergeJoin Inner (62)
- :- Sort (56)
- : +- Exchange (55)
- : +- Project (54)
- : +- SortMergeJoin Inner (53)
- : :- Sort (48)
- : : +- Exchange (47)
- : : +- Project (46)
- : : +- Filter (45)
- : : +- Scan parquet (44)
- : +- Sort (52)
- : +- Exchange (51)
- : +- Filter (50)
- : +- Scan parquet (49)
- +- Sort (61)
- +- Exchange (60)
- +- Project (59)
- +- Filter (58)
- +- Scan parquet (57)
-
-
-(1) Scan parquet
+ TakeOrderedAndProject (65)
+ +- HashAggregate (64)
+ +- HashAggregate (63)
+ +- Project (62)
+ +- SortMergeJoin Inner (61)
+ :- Sort (55)
+ : +- Exchange (54)
+ : +- Project (53)
+ : +- SortMergeJoin Inner (52)
+ : :- Sort (47)
+ : : +- Exchange (46)
+ : : +- Project (45)
+ : : +- Filter (44)
+ : : +- Scan parquet (43)
+ : +- Sort (51)
+ : +- Exchange (50)
+ : +- Filter (49)
+ : +- Scan parquet (48)
+ +- Sort (60)
+ +- Exchange (59)
+ +- Project (58)
+ +- Filter (57)
+ +- Scan parquet (56)
+
+
+(1) ScanTransformer parquet
Output [2]: [c_custkey#X, c_mktsegment#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -98,7 +97,7 @@ Input [1]: [c_custkey#X]
(9) InputIteratorTransformer
Input [1]: [c_custkey#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -166,7 +165,7 @@ Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X]
(26) InputIteratorTransformer
Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X]
-(27) Scan parquet
+(27) ScanTransformer parquet
Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -215,141 +214,134 @@ Input [6]: [o_orderkey#X, o_orderdate#X, o_shippriority#X, l_orderkey#X, l_exten
(38) RegularHashAggregateExecTransformer
Input [6]: [o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X, _pre_X#X]
Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X]
-Functions [1]: [partial_sum(_pre_X#X)]
-Aggregate Attributes [2]: [sum#X, isEmpty#X]
-Results [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X]
-
-(39) RegularHashAggregateExecTransformer
-Input [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X]
-Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X]
-Functions [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))]
+Functions [1]: [sum(_pre_X#X)]
Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X]
Results [4]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X]
-(40) ProjectExecTransformer
+(39) ProjectExecTransformer
Output [4]: [l_orderkey#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X AS revenue#X, o_orderdate#X, o_shippriority#X]
Input [4]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X]
-(41) WholeStageCodegenTransformer (X)
+(40) WholeStageCodegenTransformer (X)
Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X]
Arguments: false
-(42) TakeOrderedAndProjectExecTransformer
+(41) TakeOrderedAndProjectExecTransformer
Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X]
Arguments: X, [revenue#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X], 0
-(43) VeloxColumnarToRow
+(42) VeloxColumnarToRow
Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X]
-(44) Scan parquet
+(43) Scan parquet
Output [2]: [c_custkey#X, c_mktsegment#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(c_mktsegment), EqualTo(c_mktsegment,BUILDING), IsNotNull(c_custkey)]
ReadSchema: struct
-(45) Filter
+(44) Filter
Input [2]: [c_custkey#X, c_mktsegment#X]
Condition : ((isnotnull(c_mktsegment#X) AND (c_mktsegment#X = BUILDING)) AND isnotnull(c_custkey#X))
-(46) Project
+(45) Project
Output [1]: [c_custkey#X]
Input [2]: [c_custkey#X, c_mktsegment#X]
-(47) Exchange
+(46) Exchange
Input [1]: [c_custkey#X]
Arguments: hashpartitioning(c_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(48) Sort
+(47) Sort
Input [1]: [c_custkey#X]
Arguments: [c_custkey#X ASC NULLS FIRST], false, 0
-(49) Scan parquet
+(48) Scan parquet
Output [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(o_orderdate), LessThan(o_orderdate,1995-03-15), IsNotNull(o_custkey), IsNotNull(o_orderkey)]
ReadSchema: struct
-(50) Filter
+(49) Filter
Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X]
Condition : (((isnotnull(o_orderdate#X) AND (o_orderdate#X < 1995-03-15)) AND isnotnull(o_custkey#X)) AND isnotnull(o_orderkey#X))
-(51) Exchange
+(50) Exchange
Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X]
Arguments: hashpartitioning(o_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(52) Sort
+(51) Sort
Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X]
Arguments: [o_custkey#X ASC NULLS FIRST], false, 0
-(53) SortMergeJoin
+(52) SortMergeJoin
Left keys [1]: [c_custkey#X]
Right keys [1]: [o_custkey#X]
Join condition: None
-(54) Project
+(53) Project
Output [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X]
Input [5]: [c_custkey#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X]
-(55) Exchange
+(54) Exchange
Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X]
Arguments: hashpartitioning(o_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(56) Sort
+(55) Sort
Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X]
Arguments: [o_orderkey#X ASC NULLS FIRST], false, 0
-(57) Scan parquet
+(56) Scan parquet
Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(l_shipdate), GreaterThan(l_shipdate,1995-03-15), IsNotNull(l_orderkey)]
ReadSchema: struct
-(58) Filter
+(57) Filter
Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Condition : ((isnotnull(l_shipdate#X) AND (l_shipdate#X > 1995-03-15)) AND isnotnull(l_orderkey#X))
-(59) Project
+(58) Project
Output [3]: [l_orderkey#X, l_extendedprice#X, l_discount#X]
Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
-(60) Exchange
+(59) Exchange
Input [3]: [l_orderkey#X, l_extendedprice#X, l_discount#X]
Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(61) Sort
+(60) Sort
Input [3]: [l_orderkey#X, l_extendedprice#X, l_discount#X]
Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0
-(62) SortMergeJoin
+(61) SortMergeJoin
Left keys [1]: [o_orderkey#X]
Right keys [1]: [l_orderkey#X]
Join condition: None
-(63) Project
+(62) Project
Output [5]: [o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X]
Input [6]: [o_orderkey#X, o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X]
-(64) HashAggregate
+(63) HashAggregate
Input [5]: [o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X]
Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X]
Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))]
Aggregate Attributes [2]: [sum#X, isEmpty#X]
Results [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X]
-(65) HashAggregate
+(64) HashAggregate
Input [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X]
Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X]
Functions [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))]
Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X]
Results [4]: [l_orderkey#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X AS revenue#X, o_orderdate#X, o_shippriority#X]
-(66) TakeOrderedAndProject
+(65) TakeOrderedAndProject
Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X]
Arguments: X, [revenue#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X]
-(67) AdaptiveSparkPlan
+(66) AdaptiveSparkPlan
Output [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X]
Arguments: isFinalPlan=true
\ No newline at end of file
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 ffb35d1813ae..40cfa020ce73 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
@@ -22,14 +22,14 @@ AdaptiveSparkPlan (56)
: +- VeloxResizeBatches (5)
: +- ^ ProjectExecTransformer (3)
: +- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
+ : +- ^ ScanTransformer parquet (1)
+- ^ InputIteratorTransformer (18)
+- ShuffleQueryStage (16)
+- ColumnarExchange (15)
+- VeloxResizeBatches (14)
+- ^ ProjectExecTransformer (12)
+- ^ FilterExecTransformer (11)
- +- ^ Scan parquet (10)
+ +- ^ ScanTransformer parquet (10)
+- == Initial Plan ==
Sort (55)
+- Exchange (54)
@@ -50,7 +50,7 @@ AdaptiveSparkPlan (56)
+- Scan parquet (44)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -87,7 +87,7 @@ Input [2]: [o_orderkey#X, o_orderpriority#X]
(9) InputIteratorTransformer
Input [2]: [o_orderkey#X, o_orderpriority#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [3]: [l_orderkey#X, l_commitdate#X, l_receiptdate#X]
Batched: true
Location: InMemoryFileIndex [*]
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 0e48f73c7795..37faa880d8af 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
@@ -46,42 +46,42 @@ AdaptiveSparkPlan (156)
: : : : : +- VeloxResizeBatches (5)
: : : : : +- ^ ProjectExecTransformer (3)
: : : : : +- ^ FilterExecTransformer (2)
- : : : : : +- ^ Scan parquet (1)
+ : : : : : +- ^ ScanTransformer parquet (1)
: : : : +- ^ InputIteratorTransformer (18)
: : : : +- ShuffleQueryStage (16)
: : : : +- ColumnarExchange (15)
: : : : +- VeloxResizeBatches (14)
: : : : +- ^ ProjectExecTransformer (12)
: : : : +- ^ FilterExecTransformer (11)
- : : : : +- ^ Scan parquet (10)
+ : : : : +- ^ ScanTransformer parquet (10)
: : : +- ^ InputIteratorTransformer (35)
: : : +- ShuffleQueryStage (33)
: : : +- ColumnarExchange (32)
: : : +- VeloxResizeBatches (31)
: : : +- ^ ProjectExecTransformer (29)
: : : +- ^ FilterExecTransformer (28)
- : : : +- ^ Scan parquet (27)
+ : : : +- ^ ScanTransformer parquet (27)
: : +- ^ InputIteratorTransformer (52)
: : +- ShuffleQueryStage (50)
: : +- ColumnarExchange (49)
: : +- VeloxResizeBatches (48)
: : +- ^ ProjectExecTransformer (46)
: : +- ^ FilterExecTransformer (45)
- : : +- ^ Scan parquet (44)
+ : : +- ^ ScanTransformer parquet (44)
: +- ^ InputIteratorTransformer (69)
: +- ShuffleQueryStage (67)
: +- ColumnarExchange (66)
: +- VeloxResizeBatches (65)
: +- ^ ProjectExecTransformer (63)
: +- ^ FilterExecTransformer (62)
- : +- ^ Scan parquet (61)
+ : +- ^ ScanTransformer parquet (61)
+- ^ InputIteratorTransformer (86)
+- ShuffleQueryStage (84)
+- ColumnarExchange (83)
+- VeloxResizeBatches (82)
+- ^ ProjectExecTransformer (80)
+- ^ FilterExecTransformer (79)
- +- ^ Scan parquet (78)
+ +- ^ ScanTransformer parquet (78)
+- == Initial Plan ==
Sort (155)
+- Exchange (154)
@@ -134,7 +134,7 @@ AdaptiveSparkPlan (156)
+- Scan parquet (144)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [c_custkey#X, c_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -171,7 +171,7 @@ Input [2]: [c_custkey#X, c_nationkey#X]
(9) InputIteratorTransformer
Input [2]: [c_custkey#X, c_nationkey#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -239,7 +239,7 @@ Input [2]: [c_nationkey#X, o_orderkey#X]
(26) InputIteratorTransformer
Input [2]: [c_nationkey#X, o_orderkey#X]
-(27) Scan parquet
+(27) ScanTransformer parquet
Output [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -307,7 +307,7 @@ Input [4]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
(43) InputIteratorTransformer
Input [4]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
-(44) Scan parquet
+(44) ScanTransformer parquet
Output [2]: [s_suppkey#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -375,7 +375,7 @@ Input [3]: [l_extendedprice#X, l_discount#X, s_nationkey#X]
(60) InputIteratorTransformer
Input [3]: [l_extendedprice#X, l_discount#X, s_nationkey#X]
-(61) Scan parquet
+(61) ScanTransformer parquet
Output [3]: [n_nationkey#X, n_name#X, n_regionkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -443,7 +443,7 @@ Input [4]: [l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X]
(77) InputIteratorTransformer
Input [4]: [l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X]
-(78) Scan parquet
+(78) ScanTransformer parquet
Output [2]: [r_regionkey#X, r_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 a1830678d0de..73d9d734e78a 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
@@ -10,7 +10,7 @@ AdaptiveSparkPlan (20)
+- ^ FlushableHashAggregateExecTransformer (4)
+- ^ ProjectExecTransformer (3)
+- ^ FilterExecTransformer (2)
- +- ^ Scan parquet (1)
+ +- ^ ScanTransformer parquet (1)
+- == Initial Plan ==
HashAggregate (19)
+- Exchange (18)
@@ -20,7 +20,7 @@ AdaptiveSparkPlan (20)
+- Scan parquet (14)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [4]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
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 f089996aa4e3..f3ec7ef3a620 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
@@ -46,35 +46,35 @@ AdaptiveSparkPlan (149)
: : : : : +- VeloxResizeBatches (5)
: : : : : +- ^ ProjectExecTransformer (3)
: : : : : +- ^ FilterExecTransformer (2)
- : : : : : +- ^ Scan parquet (1)
+ : : : : : +- ^ ScanTransformer parquet (1)
: : : : +- ^ InputIteratorTransformer (18)
: : : : +- ShuffleQueryStage (16)
: : : : +- ColumnarExchange (15)
: : : : +- VeloxResizeBatches (14)
: : : : +- ^ ProjectExecTransformer (12)
: : : : +- ^ FilterExecTransformer (11)
- : : : : +- ^ Scan parquet (10)
+ : : : : +- ^ ScanTransformer parquet (10)
: : : +- ^ InputIteratorTransformer (35)
: : : +- ShuffleQueryStage (33)
: : : +- ColumnarExchange (32)
: : : +- VeloxResizeBatches (31)
: : : +- ^ ProjectExecTransformer (29)
: : : +- ^ FilterExecTransformer (28)
- : : : +- ^ Scan parquet (27)
+ : : : +- ^ ScanTransformer parquet (27)
: : +- ^ InputIteratorTransformer (52)
: : +- ShuffleQueryStage (50)
: : +- ColumnarExchange (49)
: : +- VeloxResizeBatches (48)
: : +- ^ ProjectExecTransformer (46)
: : +- ^ FilterExecTransformer (45)
- : : +- ^ Scan parquet (44)
+ : : +- ^ ScanTransformer parquet (44)
: +- ^ InputIteratorTransformer (69)
: +- ShuffleQueryStage (67)
: +- ColumnarExchange (66)
: +- VeloxResizeBatches (65)
: +- ^ ProjectExecTransformer (63)
: +- ^ FilterExecTransformer (62)
- : +- ^ Scan parquet (61)
+ : +- ^ ScanTransformer parquet (61)
+- ^ InputIteratorTransformer (81)
+- ShuffleQueryStage (79)
+- ReusedExchange (78)
@@ -128,7 +128,7 @@ AdaptiveSparkPlan (149)
+- Scan parquet (138)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [s_suppkey#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -165,7 +165,7 @@ Input [2]: [s_suppkey#X, s_nationkey#X]
(9) InputIteratorTransformer
Input [2]: [s_suppkey#X, s_nationkey#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [5]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -233,7 +233,7 @@ Input [5]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_ship
(26) InputIteratorTransformer
Input [5]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
-(27) Scan parquet
+(27) ScanTransformer parquet
Output [2]: [o_orderkey#X, o_custkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -301,7 +301,7 @@ Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_cust
(43) InputIteratorTransformer
Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X]
-(44) Scan parquet
+(44) ScanTransformer parquet
Output [2]: [c_custkey#X, c_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -369,7 +369,7 @@ Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nati
(60) InputIteratorTransformer
Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X]
-(61) Scan parquet
+(61) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 80b0d02a6cb2..8bd2945e4859 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
@@ -59,56 +59,56 @@ AdaptiveSparkPlan (207)
: : : : : : : +- VeloxResizeBatches (5)
: : : : : : : +- ^ ProjectExecTransformer (3)
: : : : : : : +- ^ FilterExecTransformer (2)
- : : : : : : : +- ^ Scan parquet (1)
+ : : : : : : : +- ^ ScanTransformer parquet (1)
: : : : : : +- ^ InputIteratorTransformer (18)
: : : : : : +- ShuffleQueryStage (16)
: : : : : : +- ColumnarExchange (15)
: : : : : : +- VeloxResizeBatches (14)
: : : : : : +- ^ ProjectExecTransformer (12)
: : : : : : +- ^ FilterExecTransformer (11)
- : : : : : : +- ^ Scan parquet (10)
+ : : : : : : +- ^ ScanTransformer parquet (10)
: : : : : +- ^ InputIteratorTransformer (35)
: : : : : +- ShuffleQueryStage (33)
: : : : : +- ColumnarExchange (32)
: : : : : +- VeloxResizeBatches (31)
: : : : : +- ^ ProjectExecTransformer (29)
: : : : : +- ^ FilterExecTransformer (28)
- : : : : : +- ^ Scan parquet (27)
+ : : : : : +- ^ ScanTransformer parquet (27)
: : : : +- ^ InputIteratorTransformer (52)
: : : : +- ShuffleQueryStage (50)
: : : : +- ColumnarExchange (49)
: : : : +- VeloxResizeBatches (48)
: : : : +- ^ ProjectExecTransformer (46)
: : : : +- ^ FilterExecTransformer (45)
- : : : : +- ^ Scan parquet (44)
+ : : : : +- ^ ScanTransformer parquet (44)
: : : +- ^ InputIteratorTransformer (69)
: : : +- ShuffleQueryStage (67)
: : : +- ColumnarExchange (66)
: : : +- VeloxResizeBatches (65)
: : : +- ^ ProjectExecTransformer (63)
: : : +- ^ FilterExecTransformer (62)
- : : : +- ^ Scan parquet (61)
+ : : : +- ^ ScanTransformer parquet (61)
: : +- ^ InputIteratorTransformer (86)
: : +- ShuffleQueryStage (84)
: : +- ColumnarExchange (83)
: : +- VeloxResizeBatches (82)
: : +- ^ ProjectExecTransformer (80)
: : +- ^ FilterExecTransformer (79)
- : : +- ^ Scan parquet (78)
+ : : +- ^ ScanTransformer parquet (78)
: +- ^ InputIteratorTransformer (103)
: +- ShuffleQueryStage (101)
: +- ColumnarExchange (100)
: +- VeloxResizeBatches (99)
: +- ^ ProjectExecTransformer (97)
: +- ^ FilterExecTransformer (96)
- : +- ^ Scan parquet (95)
+ : +- ^ ScanTransformer parquet (95)
+- ^ InputIteratorTransformer (120)
+- ShuffleQueryStage (118)
+- ColumnarExchange (117)
+- VeloxResizeBatches (116)
+- ^ ProjectExecTransformer (114)
+- ^ FilterExecTransformer (113)
- +- ^ Scan parquet (112)
+ +- ^ ScanTransformer parquet (112)
+- == Initial Plan ==
Sort (206)
+- Exchange (205)
@@ -177,7 +177,7 @@ AdaptiveSparkPlan (207)
+- Scan parquet (195)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [p_partkey#X, p_type#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -214,7 +214,7 @@ Input [1]: [p_partkey#X]
(9) InputIteratorTransformer
Input [1]: [p_partkey#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [5]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -282,7 +282,7 @@ Input [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
(26) InputIteratorTransformer
Input [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
-(27) Scan parquet
+(27) ScanTransformer parquet
Output [2]: [s_suppkey#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -350,7 +350,7 @@ Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X]
(43) InputIteratorTransformer
Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X]
-(44) Scan parquet
+(44) ScanTransformer parquet
Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -418,7 +418,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_order
(60) InputIteratorTransformer
Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X]
-(61) Scan parquet
+(61) ScanTransformer parquet
Output [2]: [c_custkey#X, c_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -486,7 +486,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nat
(77) InputIteratorTransformer
Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X]
-(78) Scan parquet
+(78) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_regionkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -554,7 +554,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_reg
(94) InputIteratorTransformer
Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X]
-(95) Scan parquet
+(95) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -622,7 +622,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_nam
(111) InputIteratorTransformer
Input [5]: [l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X]
-(112) Scan parquet
+(112) ScanTransformer parquet
Output [2]: [r_regionkey#X, r_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 a41822a60a19..6cca8ba5a479 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
@@ -46,42 +46,42 @@ AdaptiveSparkPlan (155)
: : : : : +- VeloxResizeBatches (5)
: : : : : +- ^ ProjectExecTransformer (3)
: : : : : +- ^ FilterExecTransformer (2)
- : : : : : +- ^ Scan parquet (1)
+ : : : : : +- ^ ScanTransformer parquet (1)
: : : : +- ^ InputIteratorTransformer (18)
: : : : +- ShuffleQueryStage (16)
: : : : +- ColumnarExchange (15)
: : : : +- VeloxResizeBatches (14)
: : : : +- ^ ProjectExecTransformer (12)
: : : : +- ^ FilterExecTransformer (11)
- : : : : +- ^ Scan parquet (10)
+ : : : : +- ^ ScanTransformer parquet (10)
: : : +- ^ InputIteratorTransformer (35)
: : : +- ShuffleQueryStage (33)
: : : +- ColumnarExchange (32)
: : : +- VeloxResizeBatches (31)
: : : +- ^ ProjectExecTransformer (29)
: : : +- ^ FilterExecTransformer (28)
- : : : +- ^ Scan parquet (27)
+ : : : +- ^ ScanTransformer parquet (27)
: : +- ^ InputIteratorTransformer (52)
: : +- ShuffleQueryStage (50)
: : +- ColumnarExchange (49)
: : +- VeloxResizeBatches (48)
: : +- ^ ProjectExecTransformer (46)
: : +- ^ FilterExecTransformer (45)
- : : +- ^ Scan parquet (44)
+ : : +- ^ ScanTransformer parquet (44)
: +- ^ InputIteratorTransformer (69)
: +- ShuffleQueryStage (67)
: +- ColumnarExchange (66)
: +- VeloxResizeBatches (65)
: +- ^ ProjectExecTransformer (63)
: +- ^ FilterExecTransformer (62)
- : +- ^ Scan parquet (61)
+ : +- ^ ScanTransformer parquet (61)
+- ^ InputIteratorTransformer (86)
+- ShuffleQueryStage (84)
+- ColumnarExchange (83)
+- VeloxResizeBatches (82)
+- ^ ProjectExecTransformer (80)
+- ^ FilterExecTransformer (79)
- +- ^ Scan parquet (78)
+ +- ^ ScanTransformer parquet (78)
+- == Initial Plan ==
Sort (154)
+- Exchange (153)
@@ -133,7 +133,7 @@ AdaptiveSparkPlan (155)
+- Scan parquet (144)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [p_partkey#X, p_name#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -170,7 +170,7 @@ Input [1]: [p_partkey#X]
(9) InputIteratorTransformer
Input [1]: [p_partkey#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -238,7 +238,7 @@ Input [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpric
(26) InputIteratorTransformer
Input [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X]
-(27) Scan parquet
+(27) ScanTransformer parquet
Output [2]: [s_suppkey#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -306,7 +306,7 @@ Input [7]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpric
(43) InputIteratorTransformer
Input [7]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X]
-(44) Scan parquet
+(44) ScanTransformer parquet
Output [3]: [ps_partkey#X, ps_suppkey#X, ps_supplycost#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -374,7 +374,7 @@ Input [6]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_natio
(60) InputIteratorTransformer
Input [6]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X]
-(61) Scan parquet
+(61) ScanTransformer parquet
Output [2]: [o_orderkey#X, o_orderdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -442,7 +442,7 @@ Input [6]: [l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_sup
(77) InputIteratorTransformer
Input [6]: [l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X]
-(78) Scan parquet
+(78) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 0851fa2447ea..ca3b50ab05c5 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
@@ -16,7 +16,7 @@ AdaptiveSparkPlan (30)
+- ^ FlushableHashAggregateExecTransformer (4)
+- ^ ProjectExecTransformer (3)
+- ^ FilterExecTransformer (2)
- +- ^ Scan parquet (1)
+ +- ^ ScanTransformer parquet (1)
+- == Initial Plan ==
Sort (29)
+- Exchange (28)
@@ -28,7 +28,7 @@ AdaptiveSparkPlan (30)
+- Scan parquet (22)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [7]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_tax#X, l_returnflag#X, l_linestatus#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
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 1b89e64860f3..a14969cbc44b 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
@@ -31,28 +31,28 @@ AdaptiveSparkPlan (100)
: : : +- VeloxResizeBatches (5)
: : : +- ^ ProjectExecTransformer (3)
: : : +- ^ FilterExecTransformer (2)
- : : : +- ^ Scan parquet (1)
+ : : : +- ^ ScanTransformer parquet (1)
: : +- ^ InputIteratorTransformer (18)
: : +- ShuffleQueryStage (16), Statistics(X)
: : +- ColumnarExchange (15)
: : +- VeloxResizeBatches (14)
: : +- ^ ProjectExecTransformer (12)
: : +- ^ FilterExecTransformer (11)
- : : +- ^ Scan parquet (10)
+ : : +- ^ ScanTransformer parquet (10)
: +- ^ InputIteratorTransformer (35)
: +- ShuffleQueryStage (33), Statistics(X)
: +- ColumnarExchange (32)
: +- VeloxResizeBatches (31)
: +- ^ ProjectExecTransformer (29)
: +- ^ FilterExecTransformer (28)
- : +- ^ Scan parquet (27)
+ : +- ^ ScanTransformer parquet (27)
+- ^ InputIteratorTransformer (52)
+- ShuffleQueryStage (50), Statistics(X)
+- ColumnarExchange (49)
+- VeloxResizeBatches (48)
+- ^ ProjectExecTransformer (46)
+- ^ FilterExecTransformer (45)
- +- ^ Scan parquet (44)
+ +- ^ ScanTransformer parquet (44)
+- == Initial Plan ==
TakeOrderedAndProject (99)
+- HashAggregate (98)
@@ -88,7 +88,7 @@ AdaptiveSparkPlan (100)
+- Scan parquet (90)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -125,7 +125,7 @@ Input [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct
(9) InputIteratorTransformer
Input [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -193,7 +193,7 @@ Input [8]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct
(26) InputIteratorTransformer
Input [8]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, o_orderkey#X]
-(27) Scan parquet
+(27) ScanTransformer parquet
Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_returnflag#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -261,7 +261,7 @@ Input [9]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct
(43) InputIteratorTransformer
Input [9]: [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]
-(44) Scan parquet
+(44) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 6ca992b70988..55d544b898c6 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
@@ -29,21 +29,21 @@ AdaptiveSparkPlan (82)
: : +- VeloxResizeBatches (5)
: : +- ^ ProjectExecTransformer (3)
: : +- ^ FilterExecTransformer (2)
- : : +- ^ Scan parquet (1)
+ : : +- ^ ScanTransformer parquet (1)
: +- ^ InputIteratorTransformer (18)
: +- ShuffleQueryStage (16), Statistics(X)
: +- ColumnarExchange (15)
: +- VeloxResizeBatches (14)
: +- ^ ProjectExecTransformer (12)
: +- ^ FilterExecTransformer (11)
- : +- ^ Scan parquet (10)
+ : +- ^ ScanTransformer parquet (10)
+- ^ InputIteratorTransformer (35)
+- ShuffleQueryStage (33), Statistics(X)
+- ColumnarExchange (32)
+- VeloxResizeBatches (31)
+- ^ ProjectExecTransformer (29)
+- ^ FilterExecTransformer (28)
- +- ^ Scan parquet (27)
+ +- ^ ScanTransformer parquet (27)
+- == Initial Plan ==
Sort (81)
+- Exchange (80)
@@ -72,7 +72,7 @@ AdaptiveSparkPlan (82)
+- Scan parquet (69)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -109,7 +109,7 @@ Input [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X]
(9) InputIteratorTransformer
Input [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [2]: [s_suppkey#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -177,7 +177,7 @@ Input [4]: [ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X]
(26) InputIteratorTransformer
Input [4]: [ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X]
-(27) Scan parquet
+(27) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -424,58 +424,57 @@ Arguments: isFinalPlan=true
===== Subqueries =====
Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery subquery#X, [id=#X]
-AdaptiveSparkPlan (136)
+AdaptiveSparkPlan (135)
+- == Final Plan ==
- VeloxColumnarToRow (114)
- +- ^ ProjectExecTransformer (112)
- +- ^ RegularHashAggregateExecTransformer (111)
- +- ^ RegularHashAggregateExecTransformer (110)
- +- ^ ProjectExecTransformer (109)
- +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (108)
- :- ^ InputIteratorTransformer (103)
- : +- ShuffleQueryStage (101), Statistics(X)
- : +- ColumnarExchange (100)
- : +- VeloxResizeBatches (99)
- : +- ^ ProjectExecTransformer (97)
- : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (96)
- : :- ^ InputIteratorTransformer (91)
- : : +- ShuffleQueryStage (89), Statistics(X)
- : : +- ColumnarExchange (88)
- : : +- VeloxResizeBatches (87)
- : : +- ^ ProjectExecTransformer (85)
- : : +- ^ FilterExecTransformer (84)
- : : +- ^ Scan parquet (83)
- : +- ^ InputIteratorTransformer (95)
- : +- ShuffleQueryStage (93), Statistics(X)
- : +- ReusedExchange (92)
- +- ^ InputIteratorTransformer (107)
- +- ShuffleQueryStage (105), Statistics(X)
- +- ReusedExchange (104)
+ VeloxColumnarToRow (113)
+ +- ^ ProjectExecTransformer (111)
+ +- ^ RegularHashAggregateExecTransformer (110)
+ +- ^ ProjectExecTransformer (109)
+ +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (108)
+ :- ^ InputIteratorTransformer (103)
+ : +- ShuffleQueryStage (101), Statistics(X)
+ : +- ColumnarExchange (100)
+ : +- VeloxResizeBatches (99)
+ : +- ^ ProjectExecTransformer (97)
+ : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (96)
+ : :- ^ InputIteratorTransformer (91)
+ : : +- ShuffleQueryStage (89), Statistics(X)
+ : : +- ColumnarExchange (88)
+ : : +- VeloxResizeBatches (87)
+ : : +- ^ ProjectExecTransformer (85)
+ : : +- ^ FilterExecTransformer (84)
+ : : +- ^ ScanTransformer parquet (83)
+ : +- ^ InputIteratorTransformer (95)
+ : +- ShuffleQueryStage (93), Statistics(X)
+ : +- ReusedExchange (92)
+ +- ^ InputIteratorTransformer (107)
+ +- ShuffleQueryStage (105), Statistics(X)
+ +- ReusedExchange (104)
+- == Initial Plan ==
- HashAggregate (135)
- +- HashAggregate (134)
- +- Project (133)
- +- SortMergeJoin Inner (132)
- :- Sort (126)
- : +- Exchange (125)
- : +- Project (124)
- : +- SortMergeJoin Inner (123)
- : :- Sort (118)
- : : +- Exchange (117)
- : : +- Filter (116)
- : : +- Scan parquet (115)
- : +- Sort (122)
- : +- Exchange (121)
- : +- Filter (120)
- : +- Scan parquet (119)
- +- Sort (131)
- +- Exchange (130)
- +- Project (129)
- +- Filter (128)
- +- Scan parquet (127)
-
-
-(83) Scan parquet
+ HashAggregate (134)
+ +- HashAggregate (133)
+ +- Project (132)
+ +- SortMergeJoin Inner (131)
+ :- Sort (125)
+ : +- Exchange (124)
+ : +- Project (123)
+ : +- SortMergeJoin Inner (122)
+ : :- Sort (117)
+ : : +- Exchange (116)
+ : : +- Filter (115)
+ : : +- Scan parquet (114)
+ : +- Sort (121)
+ : +- Exchange (120)
+ : +- Filter (119)
+ : +- Scan parquet (118)
+ +- Sort (130)
+ +- Exchange (129)
+ +- Project (128)
+ +- Filter (127)
+ +- Scan parquet (126)
+
+
+(83) ScanTransformer parquet
Output [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -581,129 +580,122 @@ Input [4]: [ps_availqty#X, ps_supplycost#X, s_nationkey#X, n_nationkey#X]
(110) RegularHashAggregateExecTransformer
Input [3]: [ps_availqty#X, ps_supplycost#X, _pre_X#X]
Keys: []
-Functions [1]: [partial_sum(_pre_X#X)]
-Aggregate Attributes [2]: [sum#X, isEmpty#X]
-Results [2]: [sum#X, isEmpty#X]
-
-(111) RegularHashAggregateExecTransformer
-Input [2]: [sum#X, isEmpty#X]
-Keys: []
-Functions [1]: [sum(CheckOverflow((promote_precision(ps_supplycost#X) * promote_precision(cast(ps_availqty#X as decimal(12,2)))), DecimalType(23,2)))]
+Functions [1]: [sum(_pre_X#X)]
Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(ps_supplycost#X) * promote_precision(cast(ps_availqty#X as decimal(12,2)))), DecimalType(23,2)))#X]
Results [1]: [sum(CheckOverflow((promote_precision(ps_supplycost#X) * promote_precision(cast(ps_availqty#X as decimal(12,2)))), DecimalType(23,2)))#X]
-(112) ProjectExecTransformer
+(111) ProjectExecTransformer
Output [1]: [CheckOverflow((promote_precision(cast(sum(CheckOverflow((promote_precision(ps_supplycost#X) * promote_precision(cast(ps_availqty#X as decimal(12,2)))), DecimalType(23,2)))#X as decimal(38,10))) * 0.0001000000), DecimalType(38,6)) AS (sum((ps_supplycost * ps_availqty)) * 0.0001000000)#X]
Input [1]: [sum(CheckOverflow((promote_precision(ps_supplycost#X) * promote_precision(cast(ps_availqty#X as decimal(12,2)))), DecimalType(23,2)))#X]
-(113) WholeStageCodegenTransformer (X)
+(112) WholeStageCodegenTransformer (X)
Input [1]: [(sum((ps_supplycost * ps_availqty)) * 0.0001000000)#X]
Arguments: false
-(114) VeloxColumnarToRow
+(113) VeloxColumnarToRow
Input [1]: [(sum((ps_supplycost * ps_availqty)) * 0.0001000000)#X]
-(115) Scan parquet
+(114) Scan parquet
Output [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(ps_suppkey)]
ReadSchema: struct
-(116) Filter
+(115) Filter
Input [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X]
Condition : isnotnull(ps_suppkey#X)
-(117) Exchange
+(116) Exchange
Input [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X]
Arguments: hashpartitioning(ps_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(118) Sort
+(117) Sort
Input [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X]
Arguments: [ps_suppkey#X ASC NULLS FIRST], false, 0
-(119) Scan parquet
+(118) Scan parquet
Output [2]: [s_suppkey#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(s_suppkey), IsNotNull(s_nationkey)]
ReadSchema: struct
-(120) Filter
+(119) Filter
Input [2]: [s_suppkey#X, s_nationkey#X]
Condition : (isnotnull(s_suppkey#X) AND isnotnull(s_nationkey#X))
-(121) Exchange
+(120) Exchange
Input [2]: [s_suppkey#X, s_nationkey#X]
Arguments: hashpartitioning(s_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(122) Sort
+(121) Sort
Input [2]: [s_suppkey#X, s_nationkey#X]
Arguments: [s_suppkey#X ASC NULLS FIRST], false, 0
-(123) SortMergeJoin
+(122) SortMergeJoin
Left keys [1]: [ps_suppkey#X]
Right keys [1]: [s_suppkey#X]
Join condition: None
-(124) Project
+(123) Project
Output [3]: [ps_availqty#X, ps_supplycost#X, s_nationkey#X]
Input [5]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X, s_suppkey#X, s_nationkey#X]
-(125) Exchange
+(124) Exchange
Input [3]: [ps_availqty#X, ps_supplycost#X, s_nationkey#X]
Arguments: hashpartitioning(s_nationkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(126) Sort
+(125) Sort
Input [3]: [ps_availqty#X, ps_supplycost#X, s_nationkey#X]
Arguments: [s_nationkey#X ASC NULLS FIRST], false, 0
-(127) Scan parquet
+(126) Scan parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(n_name), EqualTo(n_name,GERMANY), IsNotNull(n_nationkey)]
ReadSchema: struct
-(128) Filter
+(127) Filter
Input [2]: [n_nationkey#X, n_name#X]
Condition : ((isnotnull(n_name#X) AND (n_name#X = GERMANY)) AND isnotnull(n_nationkey#X))
-(129) Project
+(128) Project
Output [1]: [n_nationkey#X]
Input [2]: [n_nationkey#X, n_name#X]
-(130) Exchange
+(129) Exchange
Input [1]: [n_nationkey#X]
Arguments: hashpartitioning(n_nationkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(131) Sort
+(130) Sort
Input [1]: [n_nationkey#X]
Arguments: [n_nationkey#X ASC NULLS FIRST], false, 0
-(132) SortMergeJoin
+(131) SortMergeJoin
Left keys [1]: [s_nationkey#X]
Right keys [1]: [n_nationkey#X]
Join condition: None
-(133) Project
+(132) Project
Output [2]: [ps_availqty#X, ps_supplycost#X]
Input [4]: [ps_availqty#X, ps_supplycost#X, s_nationkey#X, n_nationkey#X]
-(134) HashAggregate
+(133) HashAggregate
Input [2]: [ps_availqty#X, ps_supplycost#X]
Keys: []
Functions [1]: [partial_sum(CheckOverflow((promote_precision(ps_supplycost#X) * promote_precision(cast(ps_availqty#X as decimal(12,2)))), DecimalType(23,2)))]
Aggregate Attributes [2]: [sum#X, isEmpty#X]
Results [2]: [sum#X, isEmpty#X]
-(135) HashAggregate
+(134) HashAggregate
Input [2]: [sum#X, isEmpty#X]
Keys: []
Functions [1]: [sum(CheckOverflow((promote_precision(ps_supplycost#X) * promote_precision(cast(ps_availqty#X as decimal(12,2)))), DecimalType(23,2)))]
Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(ps_supplycost#X) * promote_precision(cast(ps_availqty#X as decimal(12,2)))), DecimalType(23,2)))#X]
Results [1]: [CheckOverflow((promote_precision(cast(sum(CheckOverflow((promote_precision(ps_supplycost#X) * promote_precision(cast(ps_availqty#X as decimal(12,2)))), DecimalType(23,2)))#X as decimal(38,10))) * 0.0001000000), DecimalType(38,6)) AS (sum((ps_supplycost * ps_availqty)) * 0.0001000000)#X]
-(136) AdaptiveSparkPlan
+(135) AdaptiveSparkPlan
Output [1]: [(sum((ps_supplycost * ps_availqty)) * 0.0001000000)#X]
Arguments: isFinalPlan=true
\ No newline at end of file
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 346a6866342e..f998361cddc5 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
@@ -22,14 +22,14 @@ AdaptiveSparkPlan (55)
: +- VeloxResizeBatches (5)
: +- ^ ProjectExecTransformer (3)
: +- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
+ : +- ^ ScanTransformer parquet (1)
+- ^ InputIteratorTransformer (18)
+- ShuffleQueryStage (16), Statistics(X)
+- ColumnarExchange (15)
+- VeloxResizeBatches (14)
+- ^ ProjectExecTransformer (12)
+- ^ FilterExecTransformer (11)
- +- ^ Scan parquet (10)
+ +- ^ ScanTransformer parquet (10)
+- == Initial Plan ==
Sort (54)
+- Exchange (53)
@@ -49,7 +49,7 @@ AdaptiveSparkPlan (55)
+- Scan parquet (43)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [o_orderkey#X, o_orderpriority#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -86,7 +86,7 @@ Input [2]: [o_orderkey#X, o_orderpriority#X]
(9) InputIteratorTransformer
Input [2]: [o_orderkey#X, o_orderpriority#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [5]: [l_orderkey#X, l_shipdate#X, l_commitdate#X, l_receiptdate#X, l_shipmode#X]
Batched: true
Location: InMemoryFileIndex [*]
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 9e927782f6c3..7c3509c3e726 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
@@ -1,58 +1,57 @@
== Physical Plan ==
-AdaptiveSparkPlan (58)
+AdaptiveSparkPlan (57)
+- == Final Plan ==
- VeloxColumnarToRow (40)
- +- ^ SortExecTransformer (38)
- +- ^ InputIteratorTransformer (37)
- +- ShuffleQueryStage (35), Statistics(X)
- +- ColumnarExchange (34)
- +- VeloxResizeBatches (33)
- +- ^ RegularHashAggregateExecTransformer (31)
- +- ^ InputIteratorTransformer (30)
- +- ShuffleQueryStage (28), Statistics(X)
- +- ColumnarExchange (27)
- +- VeloxResizeBatches (26)
- +- ^ ProjectExecTransformer (24)
- +- ^ FlushableHashAggregateExecTransformer (23)
- +- ^ ProjectExecTransformer (22)
- +- ^ RegularHashAggregateExecTransformer (21)
- +- ^ RegularHashAggregateExecTransformer (20)
- +- ^ ProjectExecTransformer (19)
- +- ^ ShuffledHashJoinExecTransformer LeftOuter BuildLeft (18)
- :- ^ InputIteratorTransformer (8)
- : +- ShuffleQueryStage (6), Statistics(X)
- : +- ColumnarExchange (5)
- : +- VeloxResizeBatches (4)
- : +- ^ ProjectExecTransformer (2)
- : +- ^ Scan parquet (1)
- +- ^ InputIteratorTransformer (17)
- +- ShuffleQueryStage (15), Statistics(X)
- +- ColumnarExchange (14)
- +- VeloxResizeBatches (13)
- +- ^ ProjectExecTransformer (11)
- +- ^ FilterExecTransformer (10)
- +- ^ Scan parquet (9)
+ VeloxColumnarToRow (39)
+ +- ^ SortExecTransformer (37)
+ +- ^ InputIteratorTransformer (36)
+ +- ShuffleQueryStage (34), Statistics(X)
+ +- ColumnarExchange (33)
+ +- VeloxResizeBatches (32)
+ +- ^ RegularHashAggregateExecTransformer (30)
+ +- ^ InputIteratorTransformer (29)
+ +- ShuffleQueryStage (27), Statistics(X)
+ +- ColumnarExchange (26)
+ +- VeloxResizeBatches (25)
+ +- ^ ProjectExecTransformer (23)
+ +- ^ FlushableHashAggregateExecTransformer (22)
+ +- ^ ProjectExecTransformer (21)
+ +- ^ RegularHashAggregateExecTransformer (20)
+ +- ^ ProjectExecTransformer (19)
+ +- ^ ShuffledHashJoinExecTransformer LeftOuter BuildLeft (18)
+ :- ^ InputIteratorTransformer (8)
+ : +- ShuffleQueryStage (6), Statistics(X)
+ : +- ColumnarExchange (5)
+ : +- VeloxResizeBatches (4)
+ : +- ^ ProjectExecTransformer (2)
+ : +- ^ ScanTransformer parquet (1)
+ +- ^ InputIteratorTransformer (17)
+ +- ShuffleQueryStage (15), Statistics(X)
+ +- ColumnarExchange (14)
+ +- VeloxResizeBatches (13)
+ +- ^ ProjectExecTransformer (11)
+ +- ^ FilterExecTransformer (10)
+ +- ^ ScanTransformer parquet (9)
+- == Initial Plan ==
- Sort (57)
- +- Exchange (56)
- +- HashAggregate (55)
- +- Exchange (54)
- +- HashAggregate (53)
- +- HashAggregate (52)
- +- HashAggregate (51)
- +- Project (50)
- +- SortMergeJoin LeftOuter (49)
- :- Sort (43)
- : +- Exchange (42)
- : +- Scan parquet (41)
- +- Sort (48)
- +- Exchange (47)
- +- Project (46)
- +- Filter (45)
- +- Scan parquet (44)
-
-
-(1) Scan parquet
+ Sort (56)
+ +- Exchange (55)
+ +- HashAggregate (54)
+ +- Exchange (53)
+ +- HashAggregate (52)
+ +- HashAggregate (51)
+ +- HashAggregate (50)
+ +- Project (49)
+ +- SortMergeJoin LeftOuter (48)
+ :- Sort (42)
+ : +- Exchange (41)
+ : +- Scan parquet (40)
+ +- Sort (47)
+ +- Exchange (46)
+ +- Project (45)
+ +- Filter (44)
+ +- Scan parquet (43)
+
+
+(1) ScanTransformer parquet
Output [1]: [c_custkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -84,7 +83,7 @@ Input [1]: [c_custkey#X]
(8) InputIteratorTransformer
Input [1]: [c_custkey#X]
-(9) Scan parquet
+(9) ScanTransformer parquet
Output [3]: [o_orderkey#X, o_custkey#X, o_comment#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -133,180 +132,173 @@ Input [3]: [c_custkey#X, o_orderkey#X, o_custkey#X]
(20) RegularHashAggregateExecTransformer
Input [2]: [c_custkey#X, o_orderkey#X]
Keys [1]: [c_custkey#X]
-Functions [1]: [partial_count(o_orderkey#X)]
-Aggregate Attributes [1]: [count#X]
-Results [2]: [c_custkey#X, count#X]
-
-(21) RegularHashAggregateExecTransformer
-Input [2]: [c_custkey#X, count#X]
-Keys [1]: [c_custkey#X]
Functions [1]: [count(o_orderkey#X)]
Aggregate Attributes [1]: [count(o_orderkey#X)#X]
Results [2]: [c_custkey#X, count(o_orderkey#X)#X]
-(22) ProjectExecTransformer
+(21) ProjectExecTransformer
Output [1]: [count(o_orderkey#X)#X AS c_count#X]
Input [2]: [c_custkey#X, count(o_orderkey#X)#X]
-(23) FlushableHashAggregateExecTransformer
+(22) FlushableHashAggregateExecTransformer
Input [1]: [c_count#X]
Keys [1]: [c_count#X]
Functions [1]: [partial_count(1)]
Aggregate Attributes [1]: [count#X]
Results [2]: [c_count#X, count#X]
-(24) ProjectExecTransformer
+(23) ProjectExecTransformer
Output [3]: [hash(c_count#X, 42) AS hash_partition_key#X, c_count#X, count#X]
Input [2]: [c_count#X, count#X]
-(25) WholeStageCodegenTransformer (X)
+(24) WholeStageCodegenTransformer (X)
Input [3]: [hash_partition_key#X, c_count#X, count#X]
Arguments: false
-(26) VeloxResizeBatches
+(25) VeloxResizeBatches
Input [3]: [hash_partition_key#X, c_count#X, count#X]
Arguments: X, X
-(27) ColumnarExchange
+(26) ColumnarExchange
Input [3]: [hash_partition_key#X, c_count#X, count#X]
Arguments: hashpartitioning(c_count#X, 1), ENSURE_REQUIREMENTS, [c_count#X, count#X], [plan_id=X], [shuffle_writer_type=hash]
-(28) ShuffleQueryStage
+(27) ShuffleQueryStage
Output [2]: [c_count#X, count#X]
Arguments: X
-(29) InputAdapter
+(28) InputAdapter
Input [2]: [c_count#X, count#X]
-(30) InputIteratorTransformer
+(29) InputIteratorTransformer
Input [2]: [c_count#X, count#X]
-(31) RegularHashAggregateExecTransformer
+(30) RegularHashAggregateExecTransformer
Input [2]: [c_count#X, count#X]
Keys [1]: [c_count#X]
Functions [1]: [count(1)]
Aggregate Attributes [1]: [count(1)#X]
Results [2]: [c_count#X, count(1)#X AS custdist#X]
-(32) WholeStageCodegenTransformer (X)
+(31) WholeStageCodegenTransformer (X)
Input [2]: [c_count#X, custdist#X]
Arguments: false
-(33) VeloxResizeBatches
+(32) VeloxResizeBatches
Input [2]: [c_count#X, custdist#X]
Arguments: X, X
-(34) ColumnarExchange
+(33) ColumnarExchange
Input [2]: [c_count#X, custdist#X]
Arguments: rangepartitioning(custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST, 1), ENSURE_REQUIREMENTS, [plan_id=X], [shuffle_writer_type=hash]
-(35) ShuffleQueryStage
+(34) ShuffleQueryStage
Output [2]: [c_count#X, custdist#X]
Arguments: X
-(36) InputAdapter
+(35) InputAdapter
Input [2]: [c_count#X, custdist#X]
-(37) InputIteratorTransformer
+(36) InputIteratorTransformer
Input [2]: [c_count#X, custdist#X]
-(38) SortExecTransformer
+(37) SortExecTransformer
Input [2]: [c_count#X, custdist#X]
Arguments: [custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST], true, 0
-(39) WholeStageCodegenTransformer (X)
+(38) WholeStageCodegenTransformer (X)
Input [2]: [c_count#X, custdist#X]
Arguments: false
-(40) VeloxColumnarToRow
+(39) VeloxColumnarToRow
Input [2]: [c_count#X, custdist#X]
-(41) Scan parquet
+(40) Scan parquet
Output [1]: [c_custkey#X]
Batched: true
Location: InMemoryFileIndex [*]
ReadSchema: struct
-(42) Exchange
+(41) Exchange
Input [1]: [c_custkey#X]
Arguments: hashpartitioning(c_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(43) Sort
+(42) Sort
Input [1]: [c_custkey#X]
Arguments: [c_custkey#X ASC NULLS FIRST], false, 0
-(44) Scan parquet
+(43) Scan parquet
Output [3]: [o_orderkey#X, o_custkey#X, o_comment#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(o_comment), IsNotNull(o_custkey)]
ReadSchema: struct
-(45) Filter
+(44) Filter
Input [3]: [o_orderkey#X, o_custkey#X, o_comment#X]
Condition : ((isnotnull(o_comment#X) AND NOT o_comment#X LIKE %special%requests%) AND isnotnull(o_custkey#X))
-(46) Project
+(45) Project
Output [2]: [o_orderkey#X, o_custkey#X]
Input [3]: [o_orderkey#X, o_custkey#X, o_comment#X]
-(47) Exchange
+(46) Exchange
Input [2]: [o_orderkey#X, o_custkey#X]
Arguments: hashpartitioning(o_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(48) Sort
+(47) Sort
Input [2]: [o_orderkey#X, o_custkey#X]
Arguments: [o_custkey#X ASC NULLS FIRST], false, 0
-(49) SortMergeJoin
+(48) SortMergeJoin
Left keys [1]: [c_custkey#X]
Right keys [1]: [o_custkey#X]
Join condition: None
-(50) Project
+(49) Project
Output [2]: [c_custkey#X, o_orderkey#X]
Input [3]: [c_custkey#X, o_orderkey#X, o_custkey#X]
-(51) HashAggregate
+(50) HashAggregate
Input [2]: [c_custkey#X, o_orderkey#X]
Keys [1]: [c_custkey#X]
Functions [1]: [partial_count(o_orderkey#X)]
Aggregate Attributes [1]: [count#X]
Results [2]: [c_custkey#X, count#X]
-(52) HashAggregate
+(51) HashAggregate
Input [2]: [c_custkey#X, count#X]
Keys [1]: [c_custkey#X]
Functions [1]: [count(o_orderkey#X)]
Aggregate Attributes [1]: [count(o_orderkey#X)#X]
Results [1]: [count(o_orderkey#X)#X AS c_count#X]
-(53) HashAggregate
+(52) HashAggregate
Input [1]: [c_count#X]
Keys [1]: [c_count#X]
Functions [1]: [partial_count(1)]
Aggregate Attributes [1]: [count#X]
Results [2]: [c_count#X, count#X]
-(54) Exchange
+(53) Exchange
Input [2]: [c_count#X, count#X]
Arguments: hashpartitioning(c_count#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(55) HashAggregate
+(54) HashAggregate
Input [2]: [c_count#X, count#X]
Keys [1]: [c_count#X]
Functions [1]: [count(1)]
Aggregate Attributes [1]: [count(1)#X]
Results [2]: [c_count#X, count(1)#X AS custdist#X]
-(56) Exchange
+(55) Exchange
Input [2]: [c_count#X, custdist#X]
Arguments: rangepartitioning(custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(57) Sort
+(56) Sort
Input [2]: [c_count#X, custdist#X]
Arguments: [custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST], true, 0
-(58) AdaptiveSparkPlan
+(57) AdaptiveSparkPlan
Output [2]: [c_count#X, custdist#X]
Arguments: isFinalPlan=true
\ No newline at end of file
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 d82e4f2b21db..1161b32937bc 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
@@ -1,43 +1,42 @@
== Physical Plan ==
-AdaptiveSparkPlan (39)
+AdaptiveSparkPlan (38)
+- == Final Plan ==
- VeloxColumnarToRow (25)
- +- ^ ProjectExecTransformer (23)
- +- ^ RegularHashAggregateExecTransformer (22)
- +- ^ RegularHashAggregateExecTransformer (21)
- +- ^ ProjectExecTransformer (20)
- +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19)
- :- ^ InputIteratorTransformer (9)
- : +- ShuffleQueryStage (7), Statistics(X)
- : +- ColumnarExchange (6)
- : +- VeloxResizeBatches (5)
- : +- ^ ProjectExecTransformer (3)
- : +- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
- +- ^ InputIteratorTransformer (18)
- +- ShuffleQueryStage (16), Statistics(X)
- +- ColumnarExchange (15)
- +- VeloxResizeBatches (14)
- +- ^ ProjectExecTransformer (12)
- +- ^ FilterExecTransformer (11)
- +- ^ Scan parquet (10)
+ VeloxColumnarToRow (24)
+ +- ^ ProjectExecTransformer (22)
+ +- ^ RegularHashAggregateExecTransformer (21)
+ +- ^ ProjectExecTransformer (20)
+ +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19)
+ :- ^ InputIteratorTransformer (9)
+ : +- ShuffleQueryStage (7), Statistics(X)
+ : +- ColumnarExchange (6)
+ : +- VeloxResizeBatches (5)
+ : +- ^ ProjectExecTransformer (3)
+ : +- ^ FilterExecTransformer (2)
+ : +- ^ ScanTransformer parquet (1)
+ +- ^ InputIteratorTransformer (18)
+ +- ShuffleQueryStage (16), Statistics(X)
+ +- ColumnarExchange (15)
+ +- VeloxResizeBatches (14)
+ +- ^ ProjectExecTransformer (12)
+ +- ^ FilterExecTransformer (11)
+ +- ^ ScanTransformer parquet (10)
+- == Initial Plan ==
- HashAggregate (38)
- +- HashAggregate (37)
- +- Project (36)
- +- SortMergeJoin Inner (35)
- :- Sort (30)
- : +- Exchange (29)
- : +- Project (28)
- : +- Filter (27)
- : +- Scan parquet (26)
- +- Sort (34)
- +- Exchange (33)
- +- Filter (32)
- +- Scan parquet (31)
-
-
-(1) Scan parquet
+ HashAggregate (37)
+ +- HashAggregate (36)
+ +- Project (35)
+ +- SortMergeJoin Inner (34)
+ :- Sort (29)
+ : +- Exchange (28)
+ : +- Project (27)
+ : +- Filter (26)
+ : +- Scan parquet (25)
+ +- Sort (33)
+ +- Exchange (32)
+ +- Filter (31)
+ +- Scan parquet (30)
+
+
+(1) ScanTransformer parquet
Output [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -74,7 +73,7 @@ Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X]
(9) InputIteratorTransformer
Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [2]: [p_partkey#X, p_type#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -123,93 +122,86 @@ Input [5]: [l_partkey#X, l_extendedprice#X, l_discount#X, p_partkey#X, p_type#X]
(21) RegularHashAggregateExecTransformer
Input [5]: [l_extendedprice#X, l_discount#X, p_type#X, _pre_X#X, _pre_X#X]
Keys: []
-Functions [2]: [partial_sum(_pre_X#X), partial_sum(_pre_X#X)]
-Aggregate Attributes [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X]
-Results [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X]
-
-(22) RegularHashAggregateExecTransformer
-Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X]
-Keys: []
-Functions [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)) ELSE 0.0000 END), sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))]
+Functions [2]: [sum(_pre_X#X), sum(_pre_X#X)]
Aggregate Attributes [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)) ELSE 0.0000 END)#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X]
Results [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)) ELSE 0.0000 END)#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X]
-(23) ProjectExecTransformer
+(22) ProjectExecTransformer
Output [1]: [CheckOverflow((promote_precision(CheckOverflow((100.0000 * promote_precision(sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)) ELSE 0.0000 END)#X)), DecimalType(38,6))) / promote_precision(cast(sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X as decimal(38,6)))), DecimalType(38,6)) AS promo_revenue#X]
Input [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)) ELSE 0.0000 END)#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X]
-(24) WholeStageCodegenTransformer (X)
+(23) WholeStageCodegenTransformer (X)
Input [1]: [promo_revenue#X]
Arguments: false
-(25) VeloxColumnarToRow
+(24) VeloxColumnarToRow
Input [1]: [promo_revenue#X]
-(26) Scan parquet
+(25) Scan parquet
Output [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1995-09-01), LessThan(l_shipdate,1995-10-01), IsNotNull(l_partkey)]
ReadSchema: struct
-(27) Filter
+(26) Filter
Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Condition : (((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1995-09-01)) AND (l_shipdate#X < 1995-10-01)) AND isnotnull(l_partkey#X))
-(28) Project
+(27) Project
Output [3]: [l_partkey#X, l_extendedprice#X, l_discount#X]
Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
-(29) Exchange
+(28) Exchange
Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X]
Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(30) Sort
+(29) Sort
Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X]
Arguments: [l_partkey#X ASC NULLS FIRST], false, 0
-(31) Scan parquet
+(30) Scan parquet
Output [2]: [p_partkey#X, p_type#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(p_partkey)]
ReadSchema: struct
-(32) Filter
+(31) Filter
Input [2]: [p_partkey#X, p_type#X]
Condition : isnotnull(p_partkey#X)
-(33) Exchange
+(32) Exchange
Input [2]: [p_partkey#X, p_type#X]
Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(34) Sort
+(33) Sort
Input [2]: [p_partkey#X, p_type#X]
Arguments: [p_partkey#X ASC NULLS FIRST], false, 0
-(35) SortMergeJoin
+(34) SortMergeJoin
Left keys [1]: [l_partkey#X]
Right keys [1]: [p_partkey#X]
Join condition: None
-(36) Project
+(35) Project
Output [3]: [l_extendedprice#X, l_discount#X, p_type#X]
Input [5]: [l_partkey#X, l_extendedprice#X, l_discount#X, p_partkey#X, p_type#X]
-(37) HashAggregate
+(36) HashAggregate
Input [3]: [l_extendedprice#X, l_discount#X, p_type#X]
Keys: []
Functions [2]: [partial_sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)) ELSE 0.0000 END), partial_sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))]
Aggregate Attributes [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X]
Results [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X]
-(38) HashAggregate
+(37) HashAggregate
Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X]
Keys: []
Functions [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)) ELSE 0.0000 END), sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))]
Aggregate Attributes [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)) ELSE 0.0000 END)#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X]
Results [1]: [CheckOverflow((promote_precision(CheckOverflow((100.0000 * promote_precision(sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)) ELSE 0.0000 END)#X)), DecimalType(38,6))) / promote_precision(cast(sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X as decimal(38,6)))), DecimalType(38,6)) AS promo_revenue#X]
-(39) AdaptiveSparkPlan
+(38) AdaptiveSparkPlan
Output [1]: [promo_revenue#X]
Arguments: isFinalPlan=true
\ No newline at end of file
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 950441af70e9..398cc1f5b8b9 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
@@ -14,7 +14,7 @@ AdaptiveSparkPlan (47)
: +- VeloxResizeBatches (5)
: +- ^ ProjectExecTransformer (3)
: +- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
+ : +- ^ ScanTransformer parquet (1)
+- ^ FilterExecTransformer (22)
+- ^ RegularHashAggregateExecTransformer (21)
+- ^ InputIteratorTransformer (20)
@@ -25,7 +25,7 @@ AdaptiveSparkPlan (47)
+- ^ FlushableHashAggregateExecTransformer (13)
+- ^ ProjectExecTransformer (12)
+- ^ FilterExecTransformer (11)
- +- ^ Scan parquet (10)
+ +- ^ ScanTransformer parquet (10)
+- == Initial Plan ==
Sort (46)
+- Exchange (45)
@@ -45,7 +45,7 @@ AdaptiveSparkPlan (47)
+- Scan parquet (35)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -82,7 +82,7 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X]
(9) InputIteratorTransformer
Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -257,34 +257,33 @@ Arguments: isFinalPlan=true
===== Subqueries =====
Subquery:1 Hosting operator id = 22 Hosting Expression = Subquery subquery#X, [id=#X]
-AdaptiveSparkPlan (73)
+AdaptiveSparkPlan (72)
+- == Final Plan ==
- VeloxColumnarToRow (64)
- +- ^ RegularHashAggregateExecTransformer (62)
- +- ^ RegularHashAggregateExecTransformer (61)
- +- ^ ProjectExecTransformer (60)
- +- ^ RegularHashAggregateExecTransformer (59)
- +- ^ InputIteratorTransformer (58)
- +- ShuffleQueryStage (56), Statistics(X)
- +- ColumnarExchange (55)
- +- VeloxResizeBatches (54)
- +- ^ ProjectExecTransformer (52)
- +- ^ FlushableHashAggregateExecTransformer (51)
- +- ^ ProjectExecTransformer (50)
- +- ^ FilterExecTransformer (49)
- +- ^ Scan parquet (48)
+ VeloxColumnarToRow (63)
+ +- ^ RegularHashAggregateExecTransformer (61)
+ +- ^ ProjectExecTransformer (60)
+ +- ^ RegularHashAggregateExecTransformer (59)
+ +- ^ InputIteratorTransformer (58)
+ +- ShuffleQueryStage (56), Statistics(X)
+ +- ColumnarExchange (55)
+ +- VeloxResizeBatches (54)
+ +- ^ ProjectExecTransformer (52)
+ +- ^ FlushableHashAggregateExecTransformer (51)
+ +- ^ ProjectExecTransformer (50)
+ +- ^ FilterExecTransformer (49)
+ +- ^ ScanTransformer parquet (48)
+- == Initial Plan ==
- HashAggregate (72)
- +- HashAggregate (71)
- +- HashAggregate (70)
- +- Exchange (69)
- +- HashAggregate (68)
- +- Project (67)
- +- Filter (66)
- +- Scan parquet (65)
+ HashAggregate (71)
+ +- HashAggregate (70)
+ +- HashAggregate (69)
+ +- Exchange (68)
+ +- HashAggregate (67)
+ +- Project (66)
+ +- Filter (65)
+ +- Scan parquet (64)
-(48) Scan parquet
+(48) ScanTransformer parquet
Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -346,71 +345,64 @@ Input [2]: [l_suppkey#X, sum(CheckOverflow((promote_precision(cast(l_extendedpri
(61) RegularHashAggregateExecTransformer
Input [1]: [total_revenue#X]
Keys: []
-Functions [1]: [partial_max(total_revenue#X)]
-Aggregate Attributes [1]: [max#X]
-Results [1]: [max#X]
-
-(62) RegularHashAggregateExecTransformer
-Input [1]: [max#X]
-Keys: []
Functions [1]: [max(total_revenue#X)]
Aggregate Attributes [1]: [max(total_revenue#X)#X]
Results [1]: [max(total_revenue#X)#X AS max(total_revenue)#X]
-(63) WholeStageCodegenTransformer (X)
+(62) WholeStageCodegenTransformer (X)
Input [1]: [max(total_revenue)#X]
Arguments: false
-(64) VeloxColumnarToRow
+(63) VeloxColumnarToRow
Input [1]: [max(total_revenue)#X]
-(65) Scan parquet
+(64) Scan parquet
Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1996-01-01), LessThan(l_shipdate,1996-04-01)]
ReadSchema: struct
-(66) Filter
+(65) Filter
Input [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Condition : ((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1996-01-01)) AND (l_shipdate#X < 1996-04-01))
-(67) Project
+(66) Project
Output [3]: [l_suppkey#X, l_extendedprice#X, l_discount#X]
Input [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
-(68) HashAggregate
+(67) HashAggregate
Input [3]: [l_suppkey#X, l_extendedprice#X, l_discount#X]
Keys [1]: [l_suppkey#X]
Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))]
Aggregate Attributes [2]: [sum#X, isEmpty#X]
Results [3]: [l_suppkey#X, sum#X, isEmpty#X]
-(69) Exchange
+(68) Exchange
Input [3]: [l_suppkey#X, sum#X, isEmpty#X]
Arguments: hashpartitioning(l_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(70) HashAggregate
+(69) HashAggregate
Input [3]: [l_suppkey#X, sum#X, isEmpty#X]
Keys [1]: [l_suppkey#X]
Functions [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))]
Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X]
Results [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X AS total_revenue#X]
-(71) HashAggregate
+(70) HashAggregate
Input [1]: [total_revenue#X]
Keys: []
Functions [1]: [partial_max(total_revenue#X)]
Aggregate Attributes [1]: [max#X]
Results [1]: [max#X]
-(72) HashAggregate
+(71) HashAggregate
Input [1]: [max#X]
Keys: []
Functions [1]: [max(total_revenue#X)]
Aggregate Attributes [1]: [max(total_revenue#X)#X]
Results [1]: [max(total_revenue#X)#X AS max(total_revenue)#X]
-(73) AdaptiveSparkPlan
+(72) AdaptiveSparkPlan
Output [1]: [max(total_revenue)#X]
Arguments: isFinalPlan=true
\ No newline at end of file
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 aa7aa005a9b2..cac61db4a6bd 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
@@ -29,14 +29,14 @@ AdaptiveSparkPlan (71)
: +- VeloxResizeBatches (5)
: +- ^ ProjectExecTransformer (3)
: +- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
+ : +- ^ ScanTransformer parquet (1)
+- ^ InputIteratorTransformer (18)
+- ShuffleQueryStage (16), Statistics(X)
+- ColumnarExchange (15)
+- VeloxResizeBatches (14)
+- ^ ProjectExecTransformer (12)
+- ^ FilterExecTransformer (11)
- +- ^ Scan parquet (10)
+ +- ^ ScanTransformer parquet (10)
+- == Initial Plan ==
Sort (70)
+- Exchange (69)
@@ -63,7 +63,7 @@ AdaptiveSparkPlan (71)
+- Scan parquet (57)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [ps_partkey#X, ps_suppkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -100,7 +100,7 @@ Input [2]: [ps_partkey#X, ps_suppkey#X]
(9) InputIteratorTransformer
Input [2]: [ps_partkey#X, ps_suppkey#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [4]: [p_partkey#X, p_brand#X, p_type#X, p_size#X]
Batched: true
Location: InMemoryFileIndex [*]
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 e48379edc9fa..843ce9b39eb6 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
@@ -1,65 +1,64 @@
== Physical Plan ==
-AdaptiveSparkPlan (63)
+AdaptiveSparkPlan (62)
+- == Final Plan ==
- VeloxColumnarToRow (40)
- +- ^ ProjectExecTransformer (38)
- +- ^ RegularHashAggregateExecTransformer (37)
- +- ^ RegularHashAggregateExecTransformer (36)
- +- ^ ProjectExecTransformer (35)
- +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (34)
- :- ^ ProjectExecTransformer (20)
- : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19)
- : :- ^ InputIteratorTransformer (9)
- : : +- ShuffleQueryStage (7), Statistics(X)
- : : +- ColumnarExchange (6)
- : : +- VeloxResizeBatches (5)
- : : +- ^ ProjectExecTransformer (3)
- : : +- ^ FilterExecTransformer (2)
- : : +- ^ Scan parquet (1)
- : +- ^ InputIteratorTransformer (18)
- : +- ShuffleQueryStage (16), Statistics(X)
- : +- ColumnarExchange (15)
- : +- VeloxResizeBatches (14)
- : +- ^ ProjectExecTransformer (12)
- : +- ^ FilterExecTransformer (11)
- : +- ^ Scan parquet (10)
- +- ^ FilterExecTransformer (33)
- +- ^ ProjectExecTransformer (32)
- +- ^ RegularHashAggregateExecTransformer (31)
- +- ^ InputIteratorTransformer (30)
- +- ShuffleQueryStage (28), Statistics(X)
- +- ColumnarExchange (27)
- +- VeloxResizeBatches (26)
- +- ^ ProjectExecTransformer (24)
- +- ^ FlushableHashAggregateExecTransformer (23)
- +- ^ FilterExecTransformer (22)
- +- ^ Scan parquet (21)
+ VeloxColumnarToRow (39)
+ +- ^ ProjectExecTransformer (37)
+ +- ^ RegularHashAggregateExecTransformer (36)
+ +- ^ ProjectExecTransformer (35)
+ +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (34)
+ :- ^ ProjectExecTransformer (20)
+ : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19)
+ : :- ^ InputIteratorTransformer (9)
+ : : +- ShuffleQueryStage (7), Statistics(X)
+ : : +- ColumnarExchange (6)
+ : : +- VeloxResizeBatches (5)
+ : : +- ^ ProjectExecTransformer (3)
+ : : +- ^ FilterExecTransformer (2)
+ : : +- ^ ScanTransformer parquet (1)
+ : +- ^ InputIteratorTransformer (18)
+ : +- ShuffleQueryStage (16), Statistics(X)
+ : +- ColumnarExchange (15)
+ : +- VeloxResizeBatches (14)
+ : +- ^ ProjectExecTransformer (12)
+ : +- ^ FilterExecTransformer (11)
+ : +- ^ ScanTransformer parquet (10)
+ +- ^ FilterExecTransformer (33)
+ +- ^ ProjectExecTransformer (32)
+ +- ^ RegularHashAggregateExecTransformer (31)
+ +- ^ InputIteratorTransformer (30)
+ +- ShuffleQueryStage (28), Statistics(X)
+ +- ColumnarExchange (27)
+ +- VeloxResizeBatches (26)
+ +- ^ ProjectExecTransformer (24)
+ +- ^ FlushableHashAggregateExecTransformer (23)
+ +- ^ FilterExecTransformer (22)
+ +- ^ ScanTransformer parquet (21)
+- == Initial Plan ==
- HashAggregate (62)
- +- HashAggregate (61)
- +- Project (60)
- +- SortMergeJoin Inner (59)
- :- Project (51)
- : +- SortMergeJoin Inner (50)
- : :- Sort (44)
- : : +- Exchange (43)
- : : +- Filter (42)
- : : +- Scan parquet (41)
- : +- Sort (49)
- : +- Exchange (48)
- : +- Project (47)
- : +- Filter (46)
- : +- Scan parquet (45)
- +- Sort (58)
- +- Filter (57)
- +- HashAggregate (56)
- +- Exchange (55)
- +- HashAggregate (54)
- +- Filter (53)
- +- Scan parquet (52)
-
-
-(1) Scan parquet
+ HashAggregate (61)
+ +- HashAggregate (60)
+ +- Project (59)
+ +- SortMergeJoin Inner (58)
+ :- Project (50)
+ : +- SortMergeJoin Inner (49)
+ : :- Sort (43)
+ : : +- Exchange (42)
+ : : +- Filter (41)
+ : : +- Scan parquet (40)
+ : +- Sort (48)
+ : +- Exchange (47)
+ : +- Project (46)
+ : +- Filter (45)
+ : +- Scan parquet (44)
+ +- Sort (57)
+ +- Filter (56)
+ +- HashAggregate (55)
+ +- Exchange (54)
+ +- HashAggregate (53)
+ +- Filter (52)
+ +- Scan parquet (51)
+
+
+(1) ScanTransformer parquet
Output [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -96,7 +95,7 @@ Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X]
(9) InputIteratorTransformer
Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [3]: [p_partkey#X, p_brand#X, p_container#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -142,7 +141,7 @@ Join condition: None
Output [3]: [l_quantity#X, l_extendedprice#X, p_partkey#X]
Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, p_partkey#X]
-(21) Scan parquet
+(21) ScanTransformer parquet
Output [2]: [l_partkey#X, l_quantity#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -213,139 +212,132 @@ Input [5]: [l_quantity#X, l_extendedprice#X, p_partkey#X, (0.2 * avg(l_quantity)
(36) RegularHashAggregateExecTransformer
Input [1]: [l_extendedprice#X]
Keys: []
-Functions [1]: [partial_sum(l_extendedprice#X)]
-Aggregate Attributes [2]: [sum#X, isEmpty#X]
-Results [2]: [sum#X, isEmpty#X]
-
-(37) RegularHashAggregateExecTransformer
-Input [2]: [sum#X, isEmpty#X]
-Keys: []
Functions [1]: [sum(l_extendedprice#X)]
Aggregate Attributes [1]: [sum(l_extendedprice#X)#X]
Results [1]: [sum(l_extendedprice#X)#X]
-(38) ProjectExecTransformer
+(37) ProjectExecTransformer
Output [1]: [CheckOverflow((promote_precision(sum(l_extendedprice#X)#X) / 7.00), DecimalType(27,6)) AS avg_yearly#X]
Input [1]: [sum(l_extendedprice#X)#X]
-(39) WholeStageCodegenTransformer (X)
+(38) WholeStageCodegenTransformer (X)
Input [1]: [avg_yearly#X]
Arguments: false
-(40) VeloxColumnarToRow
+(39) VeloxColumnarToRow
Input [1]: [avg_yearly#X]
-(41) Scan parquet
+(40) Scan parquet
Output [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(l_partkey), IsNotNull(l_quantity)]
ReadSchema: struct
-(42) Filter
+(41) Filter
Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X]
Condition : (isnotnull(l_partkey#X) AND isnotnull(l_quantity#X))
-(43) Exchange
+(42) Exchange
Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X]
Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(44) Sort
+(43) Sort
Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X]
Arguments: [l_partkey#X ASC NULLS FIRST], false, 0
-(45) Scan parquet
+(44) Scan parquet
Output [3]: [p_partkey#X, p_brand#X, p_container#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(p_brand), IsNotNull(p_container), EqualTo(p_brand,Brand#X), EqualTo(p_container,MED BOX), IsNotNull(p_partkey)]
ReadSchema: struct
-(46) Filter
+(45) Filter
Input [3]: [p_partkey#X, p_brand#X, p_container#X]
Condition : ((((isnotnull(p_brand#X) AND isnotnull(p_container#X)) AND (p_brand#X = Brand#X)) AND (p_container#X = MED BOX)) AND isnotnull(p_partkey#X))
-(47) Project
+(46) Project
Output [1]: [p_partkey#X]
Input [3]: [p_partkey#X, p_brand#X, p_container#X]
-(48) Exchange
+(47) Exchange
Input [1]: [p_partkey#X]
Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(49) Sort
+(48) Sort
Input [1]: [p_partkey#X]
Arguments: [p_partkey#X ASC NULLS FIRST], false, 0
-(50) SortMergeJoin
+(49) SortMergeJoin
Left keys [1]: [l_partkey#X]
Right keys [1]: [p_partkey#X]
Join condition: None
-(51) Project
+(50) Project
Output [3]: [l_quantity#X, l_extendedprice#X, p_partkey#X]
Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, p_partkey#X]
-(52) Scan parquet
+(51) Scan parquet
Output [2]: [l_partkey#X, l_quantity#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(l_partkey)]
ReadSchema: struct
-(53) Filter
+(52) Filter
Input [2]: [l_partkey#X, l_quantity#X]
Condition : isnotnull(l_partkey#X)
-(54) HashAggregate
+(53) HashAggregate
Input [2]: [l_partkey#X, l_quantity#X]
Keys [1]: [l_partkey#X]
Functions [1]: [partial_avg(l_quantity#X)]
Aggregate Attributes [2]: [sum#X, count#X]
Results [3]: [l_partkey#X, sum#X, count#X]
-(55) Exchange
+(54) Exchange
Input [3]: [l_partkey#X, sum#X, count#X]
Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(56) HashAggregate
+(55) HashAggregate
Input [3]: [l_partkey#X, sum#X, count#X]
Keys [1]: [l_partkey#X]
Functions [1]: [avg(l_quantity#X)]
Aggregate Attributes [1]: [avg(l_quantity#X)#X]
Results [2]: [CheckOverflow((0.200000 * promote_precision(avg(l_quantity#X)#X)), DecimalType(18,7)) AS (0.2 * avg(l_quantity))#X, l_partkey#X]
-(57) Filter
+(56) Filter
Input [2]: [(0.2 * avg(l_quantity))#X, l_partkey#X]
Condition : isnotnull((0.2 * avg(l_quantity))#X)
-(58) Sort
+(57) Sort
Input [2]: [(0.2 * avg(l_quantity))#X, l_partkey#X]
Arguments: [l_partkey#X ASC NULLS FIRST], false, 0
-(59) SortMergeJoin
+(58) SortMergeJoin
Left keys [1]: [p_partkey#X]
Right keys [1]: [l_partkey#X]
Join condition: (cast(l_quantity#X as decimal(18,7)) < (0.2 * avg(l_quantity))#X)
-(60) Project
+(59) Project
Output [1]: [l_extendedprice#X]
Input [5]: [l_quantity#X, l_extendedprice#X, p_partkey#X, (0.2 * avg(l_quantity))#X, l_partkey#X]
-(61) HashAggregate
+(60) HashAggregate
Input [1]: [l_extendedprice#X]
Keys: []
Functions [1]: [partial_sum(l_extendedprice#X)]
Aggregate Attributes [2]: [sum#X, isEmpty#X]
Results [2]: [sum#X, isEmpty#X]
-(62) HashAggregate
+(61) HashAggregate
Input [2]: [sum#X, isEmpty#X]
Keys: []
Functions [1]: [sum(l_extendedprice#X)]
Aggregate Attributes [1]: [sum(l_extendedprice#X)#X]
Results [1]: [CheckOverflow((promote_precision(sum(l_extendedprice#X)#X) / 7.00), DecimalType(27,6)) AS avg_yearly#X]
-(63) AdaptiveSparkPlan
+(62) AdaptiveSparkPlan
Output [1]: [avg_yearly#X]
Arguments: isFinalPlan=true
\ No newline at end of file
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 8dbf9429961a..1119ddcf80be 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
@@ -1,105 +1,104 @@
== Physical Plan ==
-AdaptiveSparkPlan (110)
+AdaptiveSparkPlan (109)
+- == Final Plan ==
- VeloxColumnarToRow (70)
- +- TakeOrderedAndProjectExecTransformer (69)
- +- ^ RegularHashAggregateExecTransformer (67)
- +- ^ RegularHashAggregateExecTransformer (66)
- +- ^ ProjectExecTransformer (65)
- +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (64)
- :- ^ InputIteratorTransformer (46)
- : +- ShuffleQueryStage (44), Statistics(X)
- : +- ColumnarExchange (43)
- : +- VeloxResizeBatches (42)
- : +- ^ ProjectExecTransformer (40)
- : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (39)
- : :- ^ InputIteratorTransformer (9)
- : : +- ShuffleQueryStage (7), Statistics(X)
- : : +- ColumnarExchange (6)
- : : +- VeloxResizeBatches (5)
- : : +- ^ ProjectExecTransformer (3)
- : : +- ^ FilterExecTransformer (2)
- : : +- ^ Scan parquet (1)
- : +- ^ InputIteratorTransformer (38)
- : +- ShuffleQueryStage (36), Statistics(X)
- : +- ColumnarExchange (35)
- : +- VeloxResizeBatches (34)
- : +- ^ ProjectExecTransformer (32)
- : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (31)
- : :- ^ InputIteratorTransformer (18)
- : : +- ShuffleQueryStage (16), Statistics(X)
- : : +- ColumnarExchange (15)
- : : +- VeloxResizeBatches (14)
- : : +- ^ ProjectExecTransformer (12)
- : : +- ^ FilterExecTransformer (11)
- : : +- ^ Scan parquet (10)
- : +- ^ ProjectExecTransformer (30)
- : +- ^ FilterExecTransformer (29)
- : +- ^ RegularHashAggregateExecTransformer (28)
- : +- ^ InputIteratorTransformer (27)
- : +- ShuffleQueryStage (25), Statistics(X)
- : +- ColumnarExchange (24)
- : +- VeloxResizeBatches (23)
- : +- ^ ProjectExecTransformer (21)
- : +- ^ FlushableHashAggregateExecTransformer (20)
- : +- ^ Scan parquet (19)
- +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (63)
- :- ^ InputIteratorTransformer (55)
- : +- ShuffleQueryStage (53), Statistics(X)
- : +- ColumnarExchange (52)
- : +- VeloxResizeBatches (51)
- : +- ^ ProjectExecTransformer (49)
- : +- ^ FilterExecTransformer (48)
- : +- ^ Scan parquet (47)
- +- ^ ProjectExecTransformer (62)
- +- ^ FilterExecTransformer (61)
- +- ^ RegularHashAggregateExecTransformer (60)
- +- ^ InputIteratorTransformer (59)
- +- ShuffleQueryStage (57), Statistics(X)
- +- ReusedExchange (56)
+ VeloxColumnarToRow (69)
+ +- TakeOrderedAndProjectExecTransformer (68)
+ +- ^ RegularHashAggregateExecTransformer (66)
+ +- ^ ProjectExecTransformer (65)
+ +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (64)
+ :- ^ InputIteratorTransformer (46)
+ : +- ShuffleQueryStage (44), Statistics(X)
+ : +- ColumnarExchange (43)
+ : +- VeloxResizeBatches (42)
+ : +- ^ ProjectExecTransformer (40)
+ : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (39)
+ : :- ^ InputIteratorTransformer (9)
+ : : +- ShuffleQueryStage (7), Statistics(X)
+ : : +- ColumnarExchange (6)
+ : : +- VeloxResizeBatches (5)
+ : : +- ^ ProjectExecTransformer (3)
+ : : +- ^ FilterExecTransformer (2)
+ : : +- ^ ScanTransformer parquet (1)
+ : +- ^ InputIteratorTransformer (38)
+ : +- ShuffleQueryStage (36), Statistics(X)
+ : +- ColumnarExchange (35)
+ : +- VeloxResizeBatches (34)
+ : +- ^ ProjectExecTransformer (32)
+ : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (31)
+ : :- ^ InputIteratorTransformer (18)
+ : : +- ShuffleQueryStage (16), Statistics(X)
+ : : +- ColumnarExchange (15)
+ : : +- VeloxResizeBatches (14)
+ : : +- ^ ProjectExecTransformer (12)
+ : : +- ^ FilterExecTransformer (11)
+ : : +- ^ ScanTransformer parquet (10)
+ : +- ^ ProjectExecTransformer (30)
+ : +- ^ FilterExecTransformer (29)
+ : +- ^ RegularHashAggregateExecTransformer (28)
+ : +- ^ InputIteratorTransformer (27)
+ : +- ShuffleQueryStage (25), Statistics(X)
+ : +- ColumnarExchange (24)
+ : +- VeloxResizeBatches (23)
+ : +- ^ ProjectExecTransformer (21)
+ : +- ^ FlushableHashAggregateExecTransformer (20)
+ : +- ^ ScanTransformer parquet (19)
+ +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (63)
+ :- ^ InputIteratorTransformer (55)
+ : +- ShuffleQueryStage (53), Statistics(X)
+ : +- ColumnarExchange (52)
+ : +- VeloxResizeBatches (51)
+ : +- ^ ProjectExecTransformer (49)
+ : +- ^ FilterExecTransformer (48)
+ : +- ^ ScanTransformer parquet (47)
+ +- ^ ProjectExecTransformer (62)
+ +- ^ FilterExecTransformer (61)
+ +- ^ RegularHashAggregateExecTransformer (60)
+ +- ^ InputIteratorTransformer (59)
+ +- ShuffleQueryStage (57), Statistics(X)
+ +- ReusedExchange (56)
+- == Initial Plan ==
- TakeOrderedAndProject (109)
- +- HashAggregate (108)
- +- HashAggregate (107)
- +- Project (106)
- +- SortMergeJoin Inner (105)
- :- Sort (92)
- : +- Exchange (91)
- : +- Project (90)
- : +- SortMergeJoin Inner (89)
- : :- Sort (74)
- : : +- Exchange (73)
- : : +- Filter (72)
- : : +- Scan parquet (71)
- : +- Sort (88)
- : +- Exchange (87)
- : +- SortMergeJoin LeftSemi (86)
- : :- Sort (78)
- : : +- Exchange (77)
- : : +- Filter (76)
- : : +- Scan parquet (75)
- : +- Sort (85)
- : +- Project (84)
- : +- Filter (83)
- : +- HashAggregate (82)
- : +- Exchange (81)
- : +- HashAggregate (80)
- : +- Scan parquet (79)
- +- SortMergeJoin LeftSemi (104)
- :- Sort (96)
- : +- Exchange (95)
- : +- Filter (94)
- : +- Scan parquet (93)
- +- Sort (103)
- +- Project (102)
- +- Filter (101)
- +- HashAggregate (100)
- +- Exchange (99)
- +- HashAggregate (98)
- +- Scan parquet (97)
-
-
-(1) Scan parquet
+ TakeOrderedAndProject (108)
+ +- HashAggregate (107)
+ +- HashAggregate (106)
+ +- Project (105)
+ +- SortMergeJoin Inner (104)
+ :- Sort (91)
+ : +- Exchange (90)
+ : +- Project (89)
+ : +- SortMergeJoin Inner (88)
+ : :- Sort (73)
+ : : +- Exchange (72)
+ : : +- Filter (71)
+ : : +- Scan parquet (70)
+ : +- Sort (87)
+ : +- Exchange (86)
+ : +- SortMergeJoin LeftSemi (85)
+ : :- Sort (77)
+ : : +- Exchange (76)
+ : : +- Filter (75)
+ : : +- Scan parquet (74)
+ : +- Sort (84)
+ : +- Project (83)
+ : +- Filter (82)
+ : +- HashAggregate (81)
+ : +- Exchange (80)
+ : +- HashAggregate (79)
+ : +- Scan parquet (78)
+ +- SortMergeJoin LeftSemi (103)
+ :- Sort (95)
+ : +- Exchange (94)
+ : +- Filter (93)
+ : +- Scan parquet (92)
+ +- Sort (102)
+ +- Project (101)
+ +- Filter (100)
+ +- HashAggregate (99)
+ +- Exchange (98)
+ +- HashAggregate (97)
+ +- Scan parquet (96)
+
+
+(1) ScanTransformer parquet
Output [2]: [c_custkey#X, c_name#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -136,7 +135,7 @@ Input [2]: [c_custkey#X, c_name#X]
(9) InputIteratorTransformer
Input [2]: [c_custkey#X, c_name#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -173,7 +172,7 @@ Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X]
(18) InputIteratorTransformer
Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X]
-(19) Scan parquet
+(19) ScanTransformer parquet
Output [2]: [l_orderkey#X, l_quantity#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -289,7 +288,7 @@ Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X]
(46) InputIteratorTransformer
Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X]
-(47) Scan parquet
+(47) ScanTransformer parquet
Output [2]: [l_orderkey#X, l_quantity#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -371,219 +370,212 @@ Input [7]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X,
(66) RegularHashAggregateExecTransformer
Input [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_quantity#X]
Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X]
-Functions [1]: [partial_sum(l_quantity#X)]
-Aggregate Attributes [2]: [sum#X, isEmpty#X]
-Results [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X]
-
-(67) RegularHashAggregateExecTransformer
-Input [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X]
-Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X]
Functions [1]: [sum(l_quantity#X)]
Aggregate Attributes [1]: [sum(l_quantity#X)#X]
Results [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity#X)#X AS sum(l_quantity)#X]
-(68) WholeStageCodegenTransformer (X)
+(67) WholeStageCodegenTransformer (X)
Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X]
Arguments: false
-(69) TakeOrderedAndProjectExecTransformer
+(68) TakeOrderedAndProjectExecTransformer
Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X]
Arguments: X, [o_totalprice#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X], 0
-(70) VeloxColumnarToRow
+(69) VeloxColumnarToRow
Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X]
-(71) Scan parquet
+(70) Scan parquet
Output [2]: [c_custkey#X, c_name#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(c_custkey)]
ReadSchema: struct
-(72) Filter
+(71) Filter
Input [2]: [c_custkey#X, c_name#X]
Condition : isnotnull(c_custkey#X)
-(73) Exchange
+(72) Exchange
Input [2]: [c_custkey#X, c_name#X]
Arguments: hashpartitioning(c_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(74) Sort
+(73) Sort
Input [2]: [c_custkey#X, c_name#X]
Arguments: [c_custkey#X ASC NULLS FIRST], false, 0
-(75) Scan parquet
+(74) Scan parquet
Output [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(o_custkey), IsNotNull(o_orderkey)]
ReadSchema: struct
-(76) Filter
+(75) Filter
Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X]
Condition : (isnotnull(o_custkey#X) AND isnotnull(o_orderkey#X))
-(77) Exchange
+(76) Exchange
Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X]
Arguments: hashpartitioning(o_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(78) Sort
+(77) Sort
Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X]
Arguments: [o_orderkey#X ASC NULLS FIRST], false, 0
-(79) Scan parquet
+(78) Scan parquet
Output [2]: [l_orderkey#X, l_quantity#X]
Batched: true
Location: InMemoryFileIndex [*]
ReadSchema: struct
-(80) HashAggregate
+(79) HashAggregate
Input [2]: [l_orderkey#X, l_quantity#X]
Keys [1]: [l_orderkey#X]
Functions [1]: [partial_sum(l_quantity#X)]
Aggregate Attributes [2]: [sum#X, isEmpty#X]
Results [3]: [l_orderkey#X, sum#X, isEmpty#X]
-(81) Exchange
+(80) Exchange
Input [3]: [l_orderkey#X, sum#X, isEmpty#X]
Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(82) HashAggregate
+(81) HashAggregate
Input [3]: [l_orderkey#X, sum#X, isEmpty#X]
Keys [1]: [l_orderkey#X]
Functions [1]: [sum(l_quantity#X)]
Aggregate Attributes [1]: [sum(l_quantity#X)#X]
Results [2]: [l_orderkey#X, sum(l_quantity#X)#X AS sum(l_quantity#X)#X]
-(83) Filter
+(82) Filter
Input [2]: [l_orderkey#X, sum(l_quantity#X)#X]
Condition : (isnotnull(sum(l_quantity#X)#X) AND (sum(l_quantity#X)#X > 300.00))
-(84) Project
+(83) Project
Output [1]: [l_orderkey#X]
Input [2]: [l_orderkey#X, sum(l_quantity#X)#X]
-(85) Sort
+(84) Sort
Input [1]: [l_orderkey#X]
Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0
-(86) SortMergeJoin
+(85) SortMergeJoin
Left keys [1]: [o_orderkey#X]
Right keys [1]: [l_orderkey#X]
Join condition: None
-(87) Exchange
+(86) Exchange
Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X]
Arguments: hashpartitioning(o_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(88) Sort
+(87) Sort
Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X]
Arguments: [o_custkey#X ASC NULLS FIRST], false, 0
-(89) SortMergeJoin
+(88) SortMergeJoin
Left keys [1]: [c_custkey#X]
Right keys [1]: [o_custkey#X]
Join condition: None
-(90) Project
+(89) Project
Output [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X]
Input [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X]
-(91) Exchange
+(90) Exchange
Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X]
Arguments: hashpartitioning(o_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(92) Sort
+(91) Sort
Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X]
Arguments: [o_orderkey#X ASC NULLS FIRST], false, 0
-(93) Scan parquet
+(92) Scan parquet
Output [2]: [l_orderkey#X, l_quantity#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(l_orderkey)]
ReadSchema: struct
-(94) Filter
+(93) Filter
Input [2]: [l_orderkey#X, l_quantity#X]
Condition : isnotnull(l_orderkey#X)
-(95) Exchange
+(94) Exchange
Input [2]: [l_orderkey#X, l_quantity#X]
Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(96) Sort
+(95) Sort
Input [2]: [l_orderkey#X, l_quantity#X]
Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0
-(97) Scan parquet
+(96) Scan parquet
Output [2]: [l_orderkey#X, l_quantity#X]
Batched: true
Location: InMemoryFileIndex [*]
ReadSchema: struct
-(98) HashAggregate
+(97) HashAggregate
Input [2]: [l_orderkey#X, l_quantity#X]
Keys [1]: [l_orderkey#X]
Functions [1]: [partial_sum(l_quantity#X)]
Aggregate Attributes [2]: [sum#X, isEmpty#X]
Results [3]: [l_orderkey#X, sum#X, isEmpty#X]
-(99) Exchange
+(98) Exchange
Input [3]: [l_orderkey#X, sum#X, isEmpty#X]
Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(100) HashAggregate
+(99) HashAggregate
Input [3]: [l_orderkey#X, sum#X, isEmpty#X]
Keys [1]: [l_orderkey#X]
Functions [1]: [sum(l_quantity#X)]
Aggregate Attributes [1]: [sum(l_quantity#X)#X]
Results [2]: [l_orderkey#X, sum(l_quantity#X)#X AS sum(l_quantity#X)#X]
-(101) Filter
+(100) Filter
Input [2]: [l_orderkey#X, sum(l_quantity#X)#X]
Condition : (isnotnull(sum(l_quantity#X)#X) AND (sum(l_quantity#X)#X > 300.00))
-(102) Project
+(101) Project
Output [1]: [l_orderkey#X]
Input [2]: [l_orderkey#X, sum(l_quantity#X)#X]
-(103) Sort
+(102) Sort
Input [1]: [l_orderkey#X]
Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0
-(104) SortMergeJoin
+(103) SortMergeJoin
Left keys [1]: [l_orderkey#X]
Right keys [1]: [l_orderkey#X]
Join condition: None
-(105) SortMergeJoin
+(104) SortMergeJoin
Left keys [1]: [o_orderkey#X]
Right keys [1]: [l_orderkey#X]
Join condition: None
-(106) Project
+(105) Project
Output [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_quantity#X]
Input [7]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_orderkey#X, l_quantity#X]
-(107) HashAggregate
+(106) HashAggregate
Input [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_quantity#X]
Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X]
Functions [1]: [partial_sum(l_quantity#X)]
Aggregate Attributes [2]: [sum#X, isEmpty#X]
Results [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X]
-(108) HashAggregate
+(107) HashAggregate
Input [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X]
Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X]
Functions [1]: [sum(l_quantity#X)]
Aggregate Attributes [1]: [sum(l_quantity#X)#X]
Results [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity#X)#X AS sum(l_quantity)#X]
-(109) TakeOrderedAndProject
+(108) TakeOrderedAndProject
Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X]
Arguments: X, [o_totalprice#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X]
-(110) AdaptiveSparkPlan
+(109) AdaptiveSparkPlan
Output [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X]
Arguments: isFinalPlan=true
\ No newline at end of file
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 41da1d28e987..31341db339e1 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
@@ -1,42 +1,41 @@
== Physical Plan ==
-AdaptiveSparkPlan (38)
+AdaptiveSparkPlan (37)
+- == Final Plan ==
- VeloxColumnarToRow (24)
- +- ^ RegularHashAggregateExecTransformer (22)
- +- ^ RegularHashAggregateExecTransformer (21)
- +- ^ ProjectExecTransformer (20)
- +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19)
- :- ^ InputIteratorTransformer (9)
- : +- ShuffleQueryStage (7), Statistics(X)
- : +- ColumnarExchange (6)
- : +- VeloxResizeBatches (5)
- : +- ^ ProjectExecTransformer (3)
- : +- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
- +- ^ InputIteratorTransformer (18)
- +- ShuffleQueryStage (16), Statistics(X)
- +- ColumnarExchange (15)
- +- VeloxResizeBatches (14)
- +- ^ ProjectExecTransformer (12)
- +- ^ FilterExecTransformer (11)
- +- ^ Scan parquet (10)
+ VeloxColumnarToRow (23)
+ +- ^ RegularHashAggregateExecTransformer (21)
+ +- ^ ProjectExecTransformer (20)
+ +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19)
+ :- ^ InputIteratorTransformer (9)
+ : +- ShuffleQueryStage (7), Statistics(X)
+ : +- ColumnarExchange (6)
+ : +- VeloxResizeBatches (5)
+ : +- ^ ProjectExecTransformer (3)
+ : +- ^ FilterExecTransformer (2)
+ : +- ^ ScanTransformer parquet (1)
+ +- ^ InputIteratorTransformer (18)
+ +- ShuffleQueryStage (16), Statistics(X)
+ +- ColumnarExchange (15)
+ +- VeloxResizeBatches (14)
+ +- ^ ProjectExecTransformer (12)
+ +- ^ FilterExecTransformer (11)
+ +- ^ ScanTransformer parquet (10)
+- == Initial Plan ==
- HashAggregate (37)
- +- HashAggregate (36)
- +- Project (35)
- +- SortMergeJoin Inner (34)
- :- Sort (29)
- : +- Exchange (28)
- : +- Project (27)
- : +- Filter (26)
- : +- Scan parquet (25)
- +- Sort (33)
- +- Exchange (32)
- +- Filter (31)
- +- Scan parquet (30)
-
-
-(1) Scan parquet
+ HashAggregate (36)
+ +- HashAggregate (35)
+ +- Project (34)
+ +- SortMergeJoin Inner (33)
+ :- Sort (28)
+ : +- Exchange (27)
+ : +- Project (26)
+ : +- Filter (25)
+ : +- Scan parquet (24)
+ +- Sort (32)
+ +- Exchange (31)
+ +- Filter (30)
+ +- Scan parquet (29)
+
+
+(1) ScanTransformer parquet
Output [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -73,7 +72,7 @@ Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X]
(9) InputIteratorTransformer
Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -122,89 +121,82 @@ Input [8]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, p_partke
(21) RegularHashAggregateExecTransformer
Input [3]: [l_extendedprice#X, l_discount#X, _pre_X#X]
Keys: []
-Functions [1]: [partial_sum(_pre_X#X)]
-Aggregate Attributes [2]: [sum#X, isEmpty#X]
-Results [2]: [sum#X, isEmpty#X]
-
-(22) RegularHashAggregateExecTransformer
-Input [2]: [sum#X, isEmpty#X]
-Keys: []
-Functions [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))]
+Functions [1]: [sum(_pre_X#X)]
Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X]
Results [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X AS revenue#X]
-(23) WholeStageCodegenTransformer (X)
+(22) WholeStageCodegenTransformer (X)
Input [1]: [revenue#X]
Arguments: false
-(24) VeloxColumnarToRow
+(23) VeloxColumnarToRow
Input [1]: [revenue#X]
-(25) Scan parquet
+(24) Scan parquet
Output [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(l_shipinstruct), In(l_shipmode, [AIR,AIR REG]), EqualTo(l_shipinstruct,DELIVER IN PERSON), IsNotNull(l_partkey), Or(Or(And(GreaterThanOrEqual(l_quantity,1.00),LessThanOrEqual(l_quantity,11.00)),And(GreaterThanOrEqual(l_quantity,10.00),LessThanOrEqual(l_quantity,20.00))),And(GreaterThanOrEqual(l_quantity,20.00),LessThanOrEqual(l_quantity,30.00)))]
ReadSchema: struct
-(26) Filter
+(25) Filter
Input [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X]
Condition : ((((isnotnull(l_shipinstruct#X) AND l_shipmode#X IN (AIR,AIR REG)) AND (l_shipinstruct#X = DELIVER IN PERSON)) AND isnotnull(l_partkey#X)) AND ((((l_quantity#X >= 1.00) AND (l_quantity#X <= 11.00)) OR ((l_quantity#X >= 10.00) AND (l_quantity#X <= 20.00))) OR ((l_quantity#X >= 20.00) AND (l_quantity#X <= 30.00))))
-(27) Project
+(26) Project
Output [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X]
Input [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X]
-(28) Exchange
+(27) Exchange
Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X]
Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(29) Sort
+(28) Sort
Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X]
Arguments: [l_partkey#X ASC NULLS FIRST], false, 0
-(30) Scan parquet
+(29) Scan parquet
Output [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(p_size), GreaterThanOrEqual(p_size,1), IsNotNull(p_partkey), Or(Or(And(And(EqualTo(p_brand,Brand#X),In(p_container, [SM BOX,SM CASE,SM PACK,SM PKG])),LessThanOrEqual(p_size,5)),And(And(EqualTo(p_brand,Brand#X),In(p_container, [MED BAG,MED BOX,MED PACK,MED PKG])),LessThanOrEqual(p_size,10))),And(And(EqualTo(p_brand,Brand#X),In(p_container, [LG BOX,LG CASE,LG PACK,LG PKG])),LessThanOrEqual(p_size,15)))]
ReadSchema: struct
-(31) Filter
+(30) Filter
Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X]
Condition : (((isnotnull(p_size#X) AND (p_size#X >= 1)) AND isnotnull(p_partkey#X)) AND (((((p_brand#X = Brand#X) AND p_container#X IN (SM CASE,SM BOX,SM PACK,SM PKG)) AND (p_size#X <= 5)) OR (((p_brand#X = Brand#X) AND p_container#X IN (MED BAG,MED BOX,MED PKG,MED PACK)) AND (p_size#X <= 10))) OR (((p_brand#X = Brand#X) AND p_container#X IN (LG CASE,LG BOX,LG PACK,LG PKG)) AND (p_size#X <= 15))))
-(32) Exchange
+(31) Exchange
Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X]
Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(33) Sort
+(32) Sort
Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X]
Arguments: [p_partkey#X ASC NULLS FIRST], false, 0
-(34) SortMergeJoin
+(33) SortMergeJoin
Left keys [1]: [l_partkey#X]
Right keys [1]: [p_partkey#X]
Join condition: (((((((p_brand#X = Brand#X) AND p_container#X IN (SM CASE,SM BOX,SM PACK,SM PKG)) AND (l_quantity#X >= 1.00)) AND (l_quantity#X <= 11.00)) AND (p_size#X <= 5)) OR (((((p_brand#X = Brand#X) AND p_container#X IN (MED BAG,MED BOX,MED PKG,MED PACK)) AND (l_quantity#X >= 10.00)) AND (l_quantity#X <= 20.00)) AND (p_size#X <= 10))) OR (((((p_brand#X = Brand#X) AND p_container#X IN (LG CASE,LG BOX,LG PACK,LG PKG)) AND (l_quantity#X >= 20.00)) AND (l_quantity#X <= 30.00)) AND (p_size#X <= 15)))
-(35) Project
+(34) Project
Output [2]: [l_extendedprice#X, l_discount#X]
Input [8]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, p_partkey#X, p_brand#X, p_size#X, p_container#X]
-(36) HashAggregate
+(35) HashAggregate
Input [2]: [l_extendedprice#X, l_discount#X]
Keys: []
Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))]
Aggregate Attributes [2]: [sum#X, isEmpty#X]
Results [2]: [sum#X, isEmpty#X]
-(37) HashAggregate
+(36) HashAggregate
Input [2]: [sum#X, isEmpty#X]
Keys: []
Functions [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))]
Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X]
Results [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X AS revenue#X]
-(38) AdaptiveSparkPlan
+(37) AdaptiveSparkPlan
Output [1]: [revenue#X]
Arguments: isFinalPlan=true
\ No newline at end of file
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 1fad7160bf64..9a2c56040b38 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
@@ -1,32 +1,32 @@
== Physical Plan ==
-AdaptiveSparkPlan (143)
+AdaptiveSparkPlan (142)
+- == Final Plan ==
- VeloxColumnarToRow (93)
- +- AQEShuffleRead (92)
- +- ShuffleQueryStage (91), Statistics(X)
- +- ColumnarExchange (90)
- +- VeloxResizeBatches (89)
- +- ^ ProjectExecTransformer (87)
- +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (86)
- :- ^ InputIteratorTransformer (76)
- : +- ShuffleQueryStage (74), Statistics(X)
- : +- ColumnarExchange (73)
- : +- VeloxResizeBatches (72)
- : +- ^ ProjectExecTransformer (70)
- : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (69)
+ VeloxColumnarToRow (92)
+ +- AQEShuffleRead (91)
+ +- ShuffleQueryStage (90), Statistics(X)
+ +- ColumnarExchange (89)
+ +- VeloxResizeBatches (88)
+ +- ^ ProjectExecTransformer (86)
+ +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (85)
+ :- ^ InputIteratorTransformer (75)
+ : +- ShuffleQueryStage (73), Statistics(X)
+ : +- ColumnarExchange (72)
+ : +- VeloxResizeBatches (71)
+ : +- ^ ProjectExecTransformer (69)
+ : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (68)
: :- ^ InputIteratorTransformer (9)
: : +- ShuffleQueryStage (7), Statistics(X)
: : +- ColumnarExchange (6)
: : +- VeloxResizeBatches (5)
: : +- ^ ProjectExecTransformer (3)
: : +- ^ FilterExecTransformer (2)
- : : +- ^ Scan parquet (1)
- : +- ^ InputIteratorTransformer (68)
- : +- ShuffleQueryStage (66), Statistics(X)
- : +- ColumnarExchange (65)
- : +- VeloxResizeBatches (64)
- : +- ^ ProjectExecTransformer (62)
- : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (61)
+ : : +- ^ ScanTransformer parquet (1)
+ : +- ^ InputIteratorTransformer (67)
+ : +- ShuffleQueryStage (65), Statistics(X)
+ : +- ColumnarExchange (64)
+ : +- VeloxResizeBatches (63)
+ : +- ^ ProjectExecTransformer (61)
+ : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (60)
: :- ^ InputIteratorTransformer (35)
: : +- ShuffleQueryStage (33), Statistics(X)
: : +- ColumnarExchange (32)
@@ -39,94 +39,93 @@ AdaptiveSparkPlan (143)
: : : +- VeloxResizeBatches (14)
: : : +- ^ ProjectExecTransformer (12)
: : : +- ^ FilterExecTransformer (11)
- : : : +- ^ Scan parquet (10)
+ : : : +- ^ ScanTransformer parquet (10)
: : +- ^ InputIteratorTransformer (27)
: : +- ShuffleQueryStage (25), Statistics(X)
: : +- ColumnarExchange (24)
: : +- VeloxResizeBatches (23)
: : +- ^ ProjectExecTransformer (21)
: : +- ^ FilterExecTransformer (20)
- : : +- ^ Scan parquet (19)
- : +- ^ InputIteratorTransformer (60)
- : +- ShuffleQueryStage (58), Statistics(X)
- : +- ColumnarExchange (57)
- : +- VeloxResizeBatches (56)
- : +- ^ ProjectExecTransformer (54)
- : +- ^ FilterExecTransformer (53)
- : +- ^ ProjectExecTransformer (52)
- : +- ^ RegularHashAggregateExecTransformer (51)
- : +- ^ RegularHashAggregateExecTransformer (50)
- : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (49)
- : :- ^ InputIteratorTransformer (44)
- : : +- ShuffleQueryStage (42), Statistics(X)
- : : +- ColumnarExchange (41)
- : : +- VeloxResizeBatches (40)
- : : +- ^ ProjectExecTransformer (38)
- : : +- ^ FilterExecTransformer (37)
- : : +- ^ Scan parquet (36)
- : +- ^ InputIteratorTransformer (48)
- : +- ShuffleQueryStage (46), Statistics(X)
- : +- ReusedExchange (45)
- +- ^ InputIteratorTransformer (85)
- +- ShuffleQueryStage (83), Statistics(X)
- +- ColumnarExchange (82)
- +- VeloxResizeBatches (81)
- +- ^ ProjectExecTransformer (79)
- +- ^ FilterExecTransformer (78)
- +- ^ Scan parquet (77)
+ : : +- ^ ScanTransformer parquet (19)
+ : +- ^ InputIteratorTransformer (59)
+ : +- ShuffleQueryStage (57), Statistics(X)
+ : +- ColumnarExchange (56)
+ : +- VeloxResizeBatches (55)
+ : +- ^ ProjectExecTransformer (53)
+ : +- ^ FilterExecTransformer (52)
+ : +- ^ ProjectExecTransformer (51)
+ : +- ^ RegularHashAggregateExecTransformer (50)
+ : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (49)
+ : :- ^ InputIteratorTransformer (44)
+ : : +- ShuffleQueryStage (42), Statistics(X)
+ : : +- ColumnarExchange (41)
+ : : +- VeloxResizeBatches (40)
+ : : +- ^ ProjectExecTransformer (38)
+ : : +- ^ FilterExecTransformer (37)
+ : : +- ^ ScanTransformer parquet (36)
+ : +- ^ InputIteratorTransformer (48)
+ : +- ShuffleQueryStage (46), Statistics(X)
+ : +- ReusedExchange (45)
+ +- ^ InputIteratorTransformer (84)
+ +- ShuffleQueryStage (82), Statistics(X)
+ +- ColumnarExchange (81)
+ +- VeloxResizeBatches (80)
+ +- ^ ProjectExecTransformer (78)
+ +- ^ FilterExecTransformer (77)
+ +- ^ ScanTransformer parquet (76)
+- == Initial Plan ==
- Sort (142)
- +- Exchange (141)
- +- Project (140)
- +- SortMergeJoin Inner (139)
- :- Sort (133)
- : +- Exchange (132)
- : +- Project (131)
- : +- SortMergeJoin LeftSemi (130)
- : :- Sort (97)
- : : +- Exchange (96)
- : : +- Filter (95)
- : : +- Scan parquet (94)
- : +- Sort (129)
- : +- Exchange (128)
- : +- Project (127)
- : +- SortMergeJoin Inner (126)
- : :- Sort (109)
- : : +- Exchange (108)
- : : +- SortMergeJoin LeftSemi (107)
- : : :- Sort (101)
- : : : +- Exchange (100)
- : : : +- Filter (99)
- : : : +- Scan parquet (98)
- : : +- Sort (106)
- : : +- Exchange (105)
- : : +- Project (104)
- : : +- Filter (103)
- : : +- Scan parquet (102)
- : +- Sort (125)
- : +- Exchange (124)
- : +- Filter (123)
- : +- HashAggregate (122)
- : +- HashAggregate (121)
- : +- SortMergeJoin LeftSemi (120)
- : :- Sort (114)
- : : +- Exchange (113)
- : : +- Project (112)
- : : +- Filter (111)
- : : +- Scan parquet (110)
- : +- Sort (119)
- : +- Exchange (118)
- : +- Project (117)
- : +- Filter (116)
- : +- Scan parquet (115)
- +- Sort (138)
- +- Exchange (137)
- +- Project (136)
- +- Filter (135)
- +- Scan parquet (134)
-
-
-(1) Scan parquet
+ Sort (141)
+ +- Exchange (140)
+ +- Project (139)
+ +- SortMergeJoin Inner (138)
+ :- Sort (132)
+ : +- Exchange (131)
+ : +- Project (130)
+ : +- SortMergeJoin LeftSemi (129)
+ : :- Sort (96)
+ : : +- Exchange (95)
+ : : +- Filter (94)
+ : : +- Scan parquet (93)
+ : +- Sort (128)
+ : +- Exchange (127)
+ : +- Project (126)
+ : +- SortMergeJoin Inner (125)
+ : :- Sort (108)
+ : : +- Exchange (107)
+ : : +- SortMergeJoin LeftSemi (106)
+ : : :- Sort (100)
+ : : : +- Exchange (99)
+ : : : +- Filter (98)
+ : : : +- Scan parquet (97)
+ : : +- Sort (105)
+ : : +- Exchange (104)
+ : : +- Project (103)
+ : : +- Filter (102)
+ : : +- Scan parquet (101)
+ : +- Sort (124)
+ : +- Exchange (123)
+ : +- Filter (122)
+ : +- HashAggregate (121)
+ : +- HashAggregate (120)
+ : +- SortMergeJoin LeftSemi (119)
+ : :- Sort (113)
+ : : +- Exchange (112)
+ : : +- Project (111)
+ : : +- Filter (110)
+ : : +- Scan parquet (109)
+ : +- Sort (118)
+ : +- Exchange (117)
+ : +- Project (116)
+ : +- Filter (115)
+ : +- Scan parquet (114)
+ +- Sort (137)
+ +- Exchange (136)
+ +- Project (135)
+ +- Filter (134)
+ +- Scan parquet (133)
+
+
+(1) ScanTransformer parquet
Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -163,7 +162,7 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X]
(9) InputIteratorTransformer
Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -200,7 +199,7 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
(18) InputIteratorTransformer
Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
-(19) Scan parquet
+(19) ScanTransformer parquet
Output [2]: [p_partkey#X, p_name#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -268,7 +267,7 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
(35) InputIteratorTransformer
Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
-(36) Scan parquet
+(36) ScanTransformer parquet
Output [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -326,407 +325,400 @@ Join condition: None
(50) RegularHashAggregateExecTransformer
Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X]
Keys [2]: [l_partkey#X, l_suppkey#X]
-Functions [1]: [partial_sum(l_quantity#X)]
-Aggregate Attributes [2]: [sum#X, isEmpty#X]
-Results [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X]
-
-(51) RegularHashAggregateExecTransformer
-Input [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X]
-Keys [2]: [l_partkey#X, l_suppkey#X]
Functions [1]: [sum(l_quantity#X)]
Aggregate Attributes [1]: [sum(l_quantity#X)#X]
Results [3]: [l_partkey#X, l_suppkey#X, sum(l_quantity#X)#X]
-(52) ProjectExecTransformer
+(51) ProjectExecTransformer
Output [3]: [CheckOverflow((0.50 * promote_precision(sum(l_quantity#X)#X)), DecimalType(24,3)) AS (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
Input [3]: [l_partkey#X, l_suppkey#X, sum(l_quantity#X)#X]
-(53) FilterExecTransformer
+(52) FilterExecTransformer
Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
Arguments: isnotnull((0.5 * sum(l_quantity))#X)
-(54) ProjectExecTransformer
+(53) ProjectExecTransformer
Output [4]: [hash(l_partkey#X, l_suppkey#X, 42) AS hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
-(55) WholeStageCodegenTransformer (X)
+(54) WholeStageCodegenTransformer (X)
Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
Arguments: false
-(56) VeloxResizeBatches
+(55) VeloxResizeBatches
Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
Arguments: X, X
-(57) ColumnarExchange
+(56) ColumnarExchange
Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
Arguments: hashpartitioning(l_partkey#X, l_suppkey#X, 1), ENSURE_REQUIREMENTS, [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X], [plan_id=X], [shuffle_writer_type=hash]
-(58) ShuffleQueryStage
+(57) ShuffleQueryStage
Output [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
Arguments: X
-(59) InputAdapter
+(58) InputAdapter
Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
-(60) InputIteratorTransformer
+(59) InputIteratorTransformer
Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
-(61) ShuffledHashJoinExecTransformer
+(60) ShuffledHashJoinExecTransformer
Left keys [2]: [ps_partkey#X, ps_suppkey#X]
Right keys [2]: [l_partkey#X, l_suppkey#X]
Join condition: (cast(ps_availqty#X as decimal(24,3)) > (0.5 * sum(l_quantity))#X)
-(62) ProjectExecTransformer
+(61) ProjectExecTransformer
Output [2]: [hash(ps_suppkey#X, 42) AS hash_partition_key#X, ps_suppkey#X]
Input [6]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
-(63) WholeStageCodegenTransformer (X)
+(62) WholeStageCodegenTransformer (X)
Input [2]: [hash_partition_key#X, ps_suppkey#X]
Arguments: false
-(64) VeloxResizeBatches
+(63) VeloxResizeBatches
Input [2]: [hash_partition_key#X, ps_suppkey#X]
Arguments: X, X
-(65) ColumnarExchange
+(64) ColumnarExchange
Input [2]: [hash_partition_key#X, ps_suppkey#X]
Arguments: hashpartitioning(ps_suppkey#X, 1), ENSURE_REQUIREMENTS, [ps_suppkey#X], [plan_id=X], [shuffle_writer_type=hash]
-(66) ShuffleQueryStage
+(65) ShuffleQueryStage
Output [1]: [ps_suppkey#X]
Arguments: X
-(67) InputAdapter
+(66) InputAdapter
Input [1]: [ps_suppkey#X]
-(68) InputIteratorTransformer
+(67) InputIteratorTransformer
Input [1]: [ps_suppkey#X]
-(69) ShuffledHashJoinExecTransformer
+(68) ShuffledHashJoinExecTransformer
Left keys [1]: [s_suppkey#X]
Right keys [1]: [ps_suppkey#X]
Join condition: None
-(70) ProjectExecTransformer
+(69) ProjectExecTransformer
Output [4]: [hash(s_nationkey#X, 42) AS hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X]
Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X]
-(71) WholeStageCodegenTransformer (X)
+(70) WholeStageCodegenTransformer (X)
Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X]
Arguments: false
-(72) VeloxResizeBatches
+(71) VeloxResizeBatches
Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X]
Arguments: X, X
-(73) ColumnarExchange
+(72) ColumnarExchange
Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X]
Arguments: hashpartitioning(s_nationkey#X, 1), ENSURE_REQUIREMENTS, [s_name#X, s_address#X, s_nationkey#X], [plan_id=X], [shuffle_writer_type=hash]
-(74) ShuffleQueryStage
+(73) ShuffleQueryStage
Output [3]: [s_name#X, s_address#X, s_nationkey#X]
Arguments: X
-(75) InputAdapter
+(74) InputAdapter
Input [3]: [s_name#X, s_address#X, s_nationkey#X]
-(76) InputIteratorTransformer
+(75) InputIteratorTransformer
Input [3]: [s_name#X, s_address#X, s_nationkey#X]
-(77) Scan parquet
+(76) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(n_name), EqualTo(n_name,CANADA), IsNotNull(n_nationkey)]
ReadSchema: struct
-(78) FilterExecTransformer
+(77) FilterExecTransformer
Input [2]: [n_nationkey#X, n_name#X]
Arguments: ((isnotnull(n_name#X) AND (n_name#X = CANADA)) AND isnotnull(n_nationkey#X))
-(79) ProjectExecTransformer
+(78) ProjectExecTransformer
Output [2]: [hash(n_nationkey#X, 42) AS hash_partition_key#X, n_nationkey#X]
Input [2]: [n_nationkey#X, n_name#X]
-(80) WholeStageCodegenTransformer (X)
+(79) WholeStageCodegenTransformer (X)
Input [2]: [hash_partition_key#X, n_nationkey#X]
Arguments: false
-(81) VeloxResizeBatches
+(80) VeloxResizeBatches
Input [2]: [hash_partition_key#X, n_nationkey#X]
Arguments: X, X
-(82) ColumnarExchange
+(81) ColumnarExchange
Input [2]: [hash_partition_key#X, n_nationkey#X]
Arguments: hashpartitioning(n_nationkey#X, 1), ENSURE_REQUIREMENTS, [n_nationkey#X], [plan_id=X], [shuffle_writer_type=hash]
-(83) ShuffleQueryStage
+(82) ShuffleQueryStage
Output [1]: [n_nationkey#X]
Arguments: X
-(84) InputAdapter
+(83) InputAdapter
Input [1]: [n_nationkey#X]
-(85) InputIteratorTransformer
+(84) InputIteratorTransformer
Input [1]: [n_nationkey#X]
-(86) ShuffledHashJoinExecTransformer
+(85) ShuffledHashJoinExecTransformer
Left keys [1]: [s_nationkey#X]
Right keys [1]: [n_nationkey#X]
Join condition: None
-(87) ProjectExecTransformer
+(86) ProjectExecTransformer
Output [2]: [s_name#X, s_address#X]
Input [4]: [s_name#X, s_address#X, s_nationkey#X, n_nationkey#X]
-(88) WholeStageCodegenTransformer (X)
+(87) WholeStageCodegenTransformer (X)
Input [2]: [s_name#X, s_address#X]
Arguments: false
-(89) VeloxResizeBatches
+(88) VeloxResizeBatches
Input [2]: [s_name#X, s_address#X]
Arguments: X, X
-(90) ColumnarExchange
+(89) ColumnarExchange
Input [2]: [s_name#X, s_address#X]
Arguments: rangepartitioning(s_name#X ASC NULLS FIRST, 1), ENSURE_REQUIREMENTS, [plan_id=X], [shuffle_writer_type=hash]
-(91) ShuffleQueryStage
+(90) ShuffleQueryStage
Output [2]: [s_name#X, s_address#X]
Arguments: X
-(92) AQEShuffleRead
+(91) AQEShuffleRead
Input [2]: [s_name#X, s_address#X]
Arguments: local
-(93) VeloxColumnarToRow
+(92) VeloxColumnarToRow
Input [2]: [s_name#X, s_address#X]
-(94) Scan parquet
+(93) Scan parquet
Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(s_nationkey)]
ReadSchema: struct
-(95) Filter
+(94) Filter
Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X]
Condition : isnotnull(s_nationkey#X)
-(96) Exchange
+(95) Exchange
Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X]
Arguments: hashpartitioning(s_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(97) Sort
+(96) Sort
Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X]
Arguments: [s_suppkey#X ASC NULLS FIRST], false, 0
-(98) Scan parquet
+(97) Scan parquet
Output [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(ps_availqty), IsNotNull(ps_partkey), IsNotNull(ps_suppkey)]
ReadSchema: struct
-(99) Filter
+(98) Filter
Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
Condition : ((isnotnull(ps_availqty#X) AND isnotnull(ps_partkey#X)) AND isnotnull(ps_suppkey#X))
-(100) Exchange
+(99) Exchange
Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
Arguments: hashpartitioning(ps_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(101) Sort
+(100) Sort
Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
Arguments: [ps_partkey#X ASC NULLS FIRST], false, 0
-(102) Scan parquet
+(101) Scan parquet
Output [2]: [p_partkey#X, p_name#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(p_name), StringStartsWith(p_name,forest)]
ReadSchema: struct
-(103) Filter
+(102) Filter
Input [2]: [p_partkey#X, p_name#X]
Condition : (isnotnull(p_name#X) AND StartsWith(p_name#X, forest))
-(104) Project
+(103) Project
Output [1]: [p_partkey#X]
Input [2]: [p_partkey#X, p_name#X]
-(105) Exchange
+(104) Exchange
Input [1]: [p_partkey#X]
Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(106) Sort
+(105) Sort
Input [1]: [p_partkey#X]
Arguments: [p_partkey#X ASC NULLS FIRST], false, 0
-(107) SortMergeJoin
+(106) SortMergeJoin
Left keys [1]: [ps_partkey#X]
Right keys [1]: [p_partkey#X]
Join condition: None
-(108) Exchange
+(107) Exchange
Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
Arguments: hashpartitioning(ps_partkey#X, ps_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(109) Sort
+(108) Sort
Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
Arguments: [ps_partkey#X ASC NULLS FIRST, ps_suppkey#X ASC NULLS FIRST], false, 0
-(110) Scan parquet
+(109) Scan parquet
Output [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1994-01-01), LessThan(l_shipdate,1995-01-01), IsNotNull(l_partkey), IsNotNull(l_suppkey)]
ReadSchema: struct
-(111) Filter
+(110) Filter
Input [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X]
Condition : ((((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1994-01-01)) AND (l_shipdate#X < 1995-01-01)) AND isnotnull(l_partkey#X)) AND isnotnull(l_suppkey#X))
-(112) Project
+(111) Project
Output [3]: [l_partkey#X, l_suppkey#X, l_quantity#X]
Input [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X]
-(113) Exchange
+(112) Exchange
Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X]
Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(114) Sort
+(113) Sort
Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X]
Arguments: [l_partkey#X ASC NULLS FIRST], false, 0
-(115) Scan parquet
+(114) Scan parquet
Output [2]: [p_partkey#X, p_name#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(p_name), StringStartsWith(p_name,forest)]
ReadSchema: struct
-(116) Filter
+(115) Filter
Input [2]: [p_partkey#X, p_name#X]
Condition : (isnotnull(p_name#X) AND StartsWith(p_name#X, forest))
-(117) Project
+(116) Project
Output [1]: [p_partkey#X]
Input [2]: [p_partkey#X, p_name#X]
-(118) Exchange
+(117) Exchange
Input [1]: [p_partkey#X]
Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(119) Sort
+(118) Sort
Input [1]: [p_partkey#X]
Arguments: [p_partkey#X ASC NULLS FIRST], false, 0
-(120) SortMergeJoin
+(119) SortMergeJoin
Left keys [1]: [l_partkey#X]
Right keys [1]: [p_partkey#X]
Join condition: None
-(121) HashAggregate
+(120) HashAggregate
Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X]
Keys [2]: [l_partkey#X, l_suppkey#X]
Functions [1]: [partial_sum(l_quantity#X)]
Aggregate Attributes [2]: [sum#X, isEmpty#X]
Results [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X]
-(122) HashAggregate
+(121) HashAggregate
Input [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X]
Keys [2]: [l_partkey#X, l_suppkey#X]
Functions [1]: [sum(l_quantity#X)]
Aggregate Attributes [1]: [sum(l_quantity#X)#X]
Results [3]: [CheckOverflow((0.50 * promote_precision(sum(l_quantity#X)#X)), DecimalType(24,3)) AS (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
-(123) Filter
+(122) Filter
Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
Condition : isnotnull((0.5 * sum(l_quantity))#X)
-(124) Exchange
+(123) Exchange
Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
Arguments: hashpartitioning(l_partkey#X, l_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(125) Sort
+(124) Sort
Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
Arguments: [l_partkey#X ASC NULLS FIRST, l_suppkey#X ASC NULLS FIRST], false, 0
-(126) SortMergeJoin
+(125) SortMergeJoin
Left keys [2]: [ps_partkey#X, ps_suppkey#X]
Right keys [2]: [l_partkey#X, l_suppkey#X]
Join condition: (cast(ps_availqty#X as decimal(24,3)) > (0.5 * sum(l_quantity))#X)
-(127) Project
+(126) Project
Output [1]: [ps_suppkey#X]
Input [6]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
-(128) Exchange
+(127) Exchange
Input [1]: [ps_suppkey#X]
Arguments: hashpartitioning(ps_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(129) Sort
+(128) Sort
Input [1]: [ps_suppkey#X]
Arguments: [ps_suppkey#X ASC NULLS FIRST], false, 0
-(130) SortMergeJoin
+(129) SortMergeJoin
Left keys [1]: [s_suppkey#X]
Right keys [1]: [ps_suppkey#X]
Join condition: None
-(131) Project
+(130) Project
Output [3]: [s_name#X, s_address#X, s_nationkey#X]
Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X]
-(132) Exchange
+(131) Exchange
Input [3]: [s_name#X, s_address#X, s_nationkey#X]
Arguments: hashpartitioning(s_nationkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(133) Sort
+(132) Sort
Input [3]: [s_name#X, s_address#X, s_nationkey#X]
Arguments: [s_nationkey#X ASC NULLS FIRST], false, 0
-(134) Scan parquet
+(133) Scan parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(n_name), EqualTo(n_name,CANADA), IsNotNull(n_nationkey)]
ReadSchema: struct
-(135) Filter
+(134) Filter
Input [2]: [n_nationkey#X, n_name#X]
Condition : ((isnotnull(n_name#X) AND (n_name#X = CANADA)) AND isnotnull(n_nationkey#X))
-(136) Project
+(135) Project
Output [1]: [n_nationkey#X]
Input [2]: [n_nationkey#X, n_name#X]
-(137) Exchange
+(136) Exchange
Input [1]: [n_nationkey#X]
Arguments: hashpartitioning(n_nationkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(138) Sort
+(137) Sort
Input [1]: [n_nationkey#X]
Arguments: [n_nationkey#X ASC NULLS FIRST], false, 0
-(139) SortMergeJoin
+(138) SortMergeJoin
Left keys [1]: [s_nationkey#X]
Right keys [1]: [n_nationkey#X]
Join condition: None
-(140) Project
+(139) Project
Output [2]: [s_name#X, s_address#X]
Input [4]: [s_name#X, s_address#X, s_nationkey#X, n_nationkey#X]
-(141) Exchange
+(140) Exchange
Input [2]: [s_name#X, s_address#X]
Arguments: rangepartitioning(s_name#X ASC NULLS FIRST, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(142) Sort
+(141) Sort
Input [2]: [s_name#X, s_address#X]
Arguments: [s_name#X ASC NULLS FIRST], true, 0
-(143) AdaptiveSparkPlan
+(142) AdaptiveSparkPlan
Output [2]: [s_name#X, s_address#X]
Arguments: isFinalPlan=true
\ No newline at end of file
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 fd61236faaf8..a675841a475a 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
@@ -29,7 +29,7 @@ AdaptiveSparkPlan (137)
: : : +- VeloxResizeBatches (5)
: : : +- ^ ProjectExecTransformer (3)
: : : +- ^ FilterExecTransformer (2)
- : : : +- ^ Scan parquet (1)
+ : : : +- ^ ScanTransformer parquet (1)
: : +- ^ InputIteratorTransformer (44)
: : +- ShuffleQueryStage (42), Statistics(X)
: : +- ColumnarExchange (41)
@@ -43,34 +43,34 @@ AdaptiveSparkPlan (137)
: : : : +- VeloxResizeBatches (14)
: : : : +- ^ ProjectExecTransformer (12)
: : : : +- ^ FilterExecTransformer (11)
- : : : : +- ^ Scan parquet (10)
+ : : : : +- ^ ScanTransformer parquet (10)
: : : +- ^ InputIteratorTransformer (26)
: : : +- ShuffleQueryStage (24), Statistics(X)
: : : +- ColumnarExchange (23)
: : : +- VeloxResizeBatches (22)
: : : +- ^ ProjectExecTransformer (20)
- : : : +- ^ Scan parquet (19)
+ : : : +- ^ ScanTransformer parquet (19)
: : +- ^ InputIteratorTransformer (36)
: : +- ShuffleQueryStage (34), Statistics(X)
: : +- ColumnarExchange (33)
: : +- VeloxResizeBatches (32)
: : +- ^ ProjectExecTransformer (30)
: : +- ^ FilterExecTransformer (29)
- : : +- ^ Scan parquet (28)
+ : : +- ^ ScanTransformer parquet (28)
: +- ^ InputIteratorTransformer (61)
: +- ShuffleQueryStage (59), Statistics(X)
: +- ColumnarExchange (58)
: +- VeloxResizeBatches (57)
: +- ^ ProjectExecTransformer (55)
: +- ^ FilterExecTransformer (54)
- : +- ^ Scan parquet (53)
+ : +- ^ ScanTransformer parquet (53)
+- ^ InputIteratorTransformer (78)
+- ShuffleQueryStage (76), Statistics(X)
+- ColumnarExchange (75)
+- VeloxResizeBatches (74)
+- ^ ProjectExecTransformer (72)
+- ^ FilterExecTransformer (71)
- +- ^ Scan parquet (70)
+ +- ^ ScanTransformer parquet (70)
+- == Initial Plan ==
TakeOrderedAndProject (136)
+- HashAggregate (135)
@@ -119,7 +119,7 @@ AdaptiveSparkPlan (137)
+- Scan parquet (126)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [3]: [s_suppkey#X, s_name#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -156,7 +156,7 @@ Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X]
(9) InputIteratorTransformer
Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -193,7 +193,7 @@ Input [2]: [l_orderkey#X, l_suppkey#X]
(18) InputIteratorTransformer
Input [2]: [l_orderkey#X, l_suppkey#X]
-(19) Scan parquet
+(19) ScanTransformer parquet
Output [2]: [l_orderkey#X, l_suppkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -230,7 +230,7 @@ Left keys [1]: [l_orderkey#X]
Right keys [1]: [l_orderkey#X]
Join condition: NOT (l_suppkey#X = l_suppkey#X)
-(28) Scan parquet
+(28) ScanTransformer parquet
Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -329,7 +329,7 @@ Input [3]: [s_name#X, s_nationkey#X, l_orderkey#X]
(52) InputIteratorTransformer
Input [3]: [s_name#X, s_nationkey#X, l_orderkey#X]
-(53) Scan parquet
+(53) ScanTransformer parquet
Output [2]: [o_orderkey#X, o_orderstatus#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -397,7 +397,7 @@ Input [2]: [s_name#X, s_nationkey#X]
(69) InputIteratorTransformer
Input [2]: [s_name#X, s_nationkey#X]
-(70) Scan parquet
+(70) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 44c57da7482f..c48c13779d64 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
@@ -22,13 +22,13 @@ AdaptiveSparkPlan (52)
: +- VeloxResizeBatches (5)
: +- ^ ProjectExecTransformer (3)
: +- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
+ : +- ^ ScanTransformer parquet (1)
+- ^ InputIteratorTransformer (17)
+- ShuffleQueryStage (15), Statistics(X)
+- ColumnarExchange (14)
+- VeloxResizeBatches (13)
+- ^ ProjectExecTransformer (11)
- +- ^ Scan parquet (10)
+ +- ^ ScanTransformer parquet (10)
+- == Initial Plan ==
Sort (51)
+- Exchange (50)
@@ -46,7 +46,7 @@ AdaptiveSparkPlan (52)
+- Scan parquet (42)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [3]: [c_custkey#X, c_phone#X, c_acctbal#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -83,7 +83,7 @@ Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X]
(9) InputIteratorTransformer
Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [1]: [o_custkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -283,7 +283,7 @@ AdaptiveSparkPlan (72)
+- ^ FlushableHashAggregateExecTransformer (56)
+- ^ ProjectExecTransformer (55)
+- ^ FilterExecTransformer (54)
- +- ^ Scan parquet (53)
+ +- ^ ScanTransformer parquet (53)
+- == Initial Plan ==
HashAggregate (71)
+- Exchange (70)
@@ -293,7 +293,7 @@ AdaptiveSparkPlan (72)
+- Scan parquet (66)
-(53) Scan parquet
+(53) ScanTransformer parquet
Output [2]: [c_phone#X, c_acctbal#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -400,7 +400,7 @@ AdaptiveSparkPlan (72)
+- ^ FlushableHashAggregateExecTransformer (56)
+- ^ ProjectExecTransformer (55)
+- ^ FilterExecTransformer (54)
- +- ^ Scan parquet (53)
+ +- ^ ScanTransformer parquet (53)
+- == Initial Plan ==
HashAggregate (71)
+- Exchange (70)
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 a698975568fb..d09fe3e19a9b 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
@@ -1,67 +1,66 @@
== Physical Plan ==
-AdaptiveSparkPlan (67)
+AdaptiveSparkPlan (66)
+- == Final Plan ==
- VeloxColumnarToRow (43)
- +- TakeOrderedAndProjectExecTransformer (42)
- +- ^ ProjectExecTransformer (40)
- +- ^ RegularHashAggregateExecTransformer (39)
- +- ^ RegularHashAggregateExecTransformer (38)
- +- ^ ProjectExecTransformer (37)
- +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (36)
- :- ^ InputIteratorTransformer (26)
- : +- ShuffleQueryStage (24), Statistics(X)
- : +- ColumnarExchange (23)
- : +- VeloxResizeBatches (22)
- : +- ^ ProjectExecTransformer (20)
- : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19)
- : :- ^ InputIteratorTransformer (9)
- : : +- ShuffleQueryStage (7), Statistics(X)
- : : +- ColumnarExchange (6)
- : : +- VeloxResizeBatches (5)
- : : +- ^ ProjectExecTransformer (3)
- : : +- ^ FilterExecTransformer (2)
- : : +- ^ Scan parquet (1)
- : +- ^ InputIteratorTransformer (18)
- : +- ShuffleQueryStage (16), Statistics(X)
- : +- ColumnarExchange (15)
- : +- VeloxResizeBatches (14)
- : +- ^ ProjectExecTransformer (12)
- : +- ^ FilterExecTransformer (11)
- : +- ^ Scan parquet (10)
- +- ^ InputIteratorTransformer (35)
- +- ShuffleQueryStage (33), Statistics(X)
- +- ColumnarExchange (32)
- +- VeloxResizeBatches (31)
- +- ^ ProjectExecTransformer (29)
- +- ^ FilterExecTransformer (28)
- +- ^ Scan parquet (27)
+ VeloxColumnarToRow (42)
+ +- TakeOrderedAndProjectExecTransformer (41)
+ +- ^ ProjectExecTransformer (39)
+ +- ^ RegularHashAggregateExecTransformer (38)
+ +- ^ ProjectExecTransformer (37)
+ +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (36)
+ :- ^ InputIteratorTransformer (26)
+ : +- ShuffleQueryStage (24), Statistics(X)
+ : +- ColumnarExchange (23)
+ : +- VeloxResizeBatches (22)
+ : +- ^ ProjectExecTransformer (20)
+ : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19)
+ : :- ^ InputIteratorTransformer (9)
+ : : +- ShuffleQueryStage (7), Statistics(X)
+ : : +- ColumnarExchange (6)
+ : : +- VeloxResizeBatches (5)
+ : : +- ^ ProjectExecTransformer (3)
+ : : +- ^ FilterExecTransformer (2)
+ : : +- ^ ScanTransformer parquet (1)
+ : +- ^ InputIteratorTransformer (18)
+ : +- ShuffleQueryStage (16), Statistics(X)
+ : +- ColumnarExchange (15)
+ : +- VeloxResizeBatches (14)
+ : +- ^ ProjectExecTransformer (12)
+ : +- ^ FilterExecTransformer (11)
+ : +- ^ ScanTransformer parquet (10)
+ +- ^ InputIteratorTransformer (35)
+ +- ShuffleQueryStage (33), Statistics(X)
+ +- ColumnarExchange (32)
+ +- VeloxResizeBatches (31)
+ +- ^ ProjectExecTransformer (29)
+ +- ^ FilterExecTransformer (28)
+ +- ^ ScanTransformer parquet (27)
+- == Initial Plan ==
- TakeOrderedAndProject (66)
- +- HashAggregate (65)
- +- HashAggregate (64)
- +- Project (63)
- +- SortMergeJoin Inner (62)
- :- Sort (56)
- : +- Exchange (55)
- : +- Project (54)
- : +- SortMergeJoin Inner (53)
- : :- Sort (48)
- : : +- Exchange (47)
- : : +- Project (46)
- : : +- Filter (45)
- : : +- Scan parquet (44)
- : +- Sort (52)
- : +- Exchange (51)
- : +- Filter (50)
- : +- Scan parquet (49)
- +- Sort (61)
- +- Exchange (60)
- +- Project (59)
- +- Filter (58)
- +- Scan parquet (57)
-
-
-(1) Scan parquet
+ TakeOrderedAndProject (65)
+ +- HashAggregate (64)
+ +- HashAggregate (63)
+ +- Project (62)
+ +- SortMergeJoin Inner (61)
+ :- Sort (55)
+ : +- Exchange (54)
+ : +- Project (53)
+ : +- SortMergeJoin Inner (52)
+ : :- Sort (47)
+ : : +- Exchange (46)
+ : : +- Project (45)
+ : : +- Filter (44)
+ : : +- Scan parquet (43)
+ : +- Sort (51)
+ : +- Exchange (50)
+ : +- Filter (49)
+ : +- Scan parquet (48)
+ +- Sort (60)
+ +- Exchange (59)
+ +- Project (58)
+ +- Filter (57)
+ +- Scan parquet (56)
+
+
+(1) ScanTransformer parquet
Output [2]: [c_custkey#X, c_mktsegment#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -98,7 +97,7 @@ Input [1]: [c_custkey#X]
(9) InputIteratorTransformer
Input [1]: [c_custkey#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -166,7 +165,7 @@ Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X]
(26) InputIteratorTransformer
Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X]
-(27) Scan parquet
+(27) ScanTransformer parquet
Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -215,141 +214,134 @@ Input [6]: [o_orderkey#X, o_orderdate#X, o_shippriority#X, l_orderkey#X, l_exten
(38) RegularHashAggregateExecTransformer
Input [6]: [o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X, _pre_X#X]
Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X]
-Functions [1]: [partial_sum(_pre_X#X)]
-Aggregate Attributes [2]: [sum#X, isEmpty#X]
-Results [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X]
-
-(39) RegularHashAggregateExecTransformer
-Input [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X]
-Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X]
-Functions [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))]
+Functions [1]: [sum(_pre_X#X)]
Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X]
Results [4]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X]
-(40) ProjectExecTransformer
+(39) ProjectExecTransformer
Output [4]: [l_orderkey#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X AS revenue#X, o_orderdate#X, o_shippriority#X]
Input [4]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X]
-(41) WholeStageCodegenTransformer (X)
+(40) WholeStageCodegenTransformer (X)
Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X]
Arguments: false
-(42) TakeOrderedAndProjectExecTransformer
+(41) TakeOrderedAndProjectExecTransformer
Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X]
Arguments: X, [revenue#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X], 0
-(43) VeloxColumnarToRow
+(42) VeloxColumnarToRow
Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X]
-(44) Scan parquet
+(43) Scan parquet
Output [2]: [c_custkey#X, c_mktsegment#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(c_mktsegment), EqualTo(c_mktsegment,BUILDING), IsNotNull(c_custkey)]
ReadSchema: struct
-(45) Filter
+(44) Filter
Input [2]: [c_custkey#X, c_mktsegment#X]
Condition : ((isnotnull(c_mktsegment#X) AND (c_mktsegment#X = BUILDING)) AND isnotnull(c_custkey#X))
-(46) Project
+(45) Project
Output [1]: [c_custkey#X]
Input [2]: [c_custkey#X, c_mktsegment#X]
-(47) Exchange
+(46) Exchange
Input [1]: [c_custkey#X]
Arguments: hashpartitioning(c_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(48) Sort
+(47) Sort
Input [1]: [c_custkey#X]
Arguments: [c_custkey#X ASC NULLS FIRST], false, 0
-(49) Scan parquet
+(48) Scan parquet
Output [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(o_orderdate), LessThan(o_orderdate,1995-03-15), IsNotNull(o_custkey), IsNotNull(o_orderkey)]
ReadSchema: struct
-(50) Filter
+(49) Filter
Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X]
Condition : (((isnotnull(o_orderdate#X) AND (o_orderdate#X < 1995-03-15)) AND isnotnull(o_custkey#X)) AND isnotnull(o_orderkey#X))
-(51) Exchange
+(50) Exchange
Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X]
Arguments: hashpartitioning(o_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(52) Sort
+(51) Sort
Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X]
Arguments: [o_custkey#X ASC NULLS FIRST], false, 0
-(53) SortMergeJoin
+(52) SortMergeJoin
Left keys [1]: [c_custkey#X]
Right keys [1]: [o_custkey#X]
Join condition: None
-(54) Project
+(53) Project
Output [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X]
Input [5]: [c_custkey#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X]
-(55) Exchange
+(54) Exchange
Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X]
Arguments: hashpartitioning(o_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(56) Sort
+(55) Sort
Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X]
Arguments: [o_orderkey#X ASC NULLS FIRST], false, 0
-(57) Scan parquet
+(56) Scan parquet
Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(l_shipdate), GreaterThan(l_shipdate,1995-03-15), IsNotNull(l_orderkey)]
ReadSchema: struct
-(58) Filter
+(57) Filter
Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Condition : ((isnotnull(l_shipdate#X) AND (l_shipdate#X > 1995-03-15)) AND isnotnull(l_orderkey#X))
-(59) Project
+(58) Project
Output [3]: [l_orderkey#X, l_extendedprice#X, l_discount#X]
Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
-(60) Exchange
+(59) Exchange
Input [3]: [l_orderkey#X, l_extendedprice#X, l_discount#X]
Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(61) Sort
+(60) Sort
Input [3]: [l_orderkey#X, l_extendedprice#X, l_discount#X]
Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0
-(62) SortMergeJoin
+(61) SortMergeJoin
Left keys [1]: [o_orderkey#X]
Right keys [1]: [l_orderkey#X]
Join condition: None
-(63) Project
+(62) Project
Output [5]: [o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X]
Input [6]: [o_orderkey#X, o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X]
-(64) HashAggregate
+(63) HashAggregate
Input [5]: [o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X]
Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X]
Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))]
Aggregate Attributes [2]: [sum#X, isEmpty#X]
Results [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X]
-(65) HashAggregate
+(64) HashAggregate
Input [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X]
Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X]
Functions [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))]
Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X]
Results [4]: [l_orderkey#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X AS revenue#X, o_orderdate#X, o_shippriority#X]
-(66) TakeOrderedAndProject
+(65) TakeOrderedAndProject
Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X]
Arguments: X, [revenue#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X]
-(67) AdaptiveSparkPlan
+(66) AdaptiveSparkPlan
Output [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X]
Arguments: isFinalPlan=true
\ No newline at end of file
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 8b00d867cd70..23fb123956dc 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
@@ -22,14 +22,14 @@ AdaptiveSparkPlan (56)
: +- VeloxResizeBatches (5)
: +- ^ ProjectExecTransformer (3)
: +- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
+ : +- ^ ScanTransformer parquet (1)
+- ^ InputIteratorTransformer (18)
+- ShuffleQueryStage (16), Statistics(X)
+- ColumnarExchange (15)
+- VeloxResizeBatches (14)
+- ^ ProjectExecTransformer (12)
+- ^ FilterExecTransformer (11)
- +- ^ Scan parquet (10)
+ +- ^ ScanTransformer parquet (10)
+- == Initial Plan ==
Sort (55)
+- Exchange (54)
@@ -50,7 +50,7 @@ AdaptiveSparkPlan (56)
+- Scan parquet (44)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -87,7 +87,7 @@ Input [2]: [o_orderkey#X, o_orderpriority#X]
(9) InputIteratorTransformer
Input [2]: [o_orderkey#X, o_orderpriority#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [3]: [l_orderkey#X, l_commitdate#X, l_receiptdate#X]
Batched: true
Location: InMemoryFileIndex [*]
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 dd869d25d416..ea8f7dbd3371 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
@@ -46,42 +46,42 @@ AdaptiveSparkPlan (156)
: : : : : +- VeloxResizeBatches (5)
: : : : : +- ^ ProjectExecTransformer (3)
: : : : : +- ^ FilterExecTransformer (2)
- : : : : : +- ^ Scan parquet (1)
+ : : : : : +- ^ ScanTransformer parquet (1)
: : : : +- ^ InputIteratorTransformer (18)
: : : : +- ShuffleQueryStage (16), Statistics(X)
: : : : +- ColumnarExchange (15)
: : : : +- VeloxResizeBatches (14)
: : : : +- ^ ProjectExecTransformer (12)
: : : : +- ^ FilterExecTransformer (11)
- : : : : +- ^ Scan parquet (10)
+ : : : : +- ^ ScanTransformer parquet (10)
: : : +- ^ InputIteratorTransformer (35)
: : : +- ShuffleQueryStage (33), Statistics(X)
: : : +- ColumnarExchange (32)
: : : +- VeloxResizeBatches (31)
: : : +- ^ ProjectExecTransformer (29)
: : : +- ^ FilterExecTransformer (28)
- : : : +- ^ Scan parquet (27)
+ : : : +- ^ ScanTransformer parquet (27)
: : +- ^ InputIteratorTransformer (52)
: : +- ShuffleQueryStage (50), Statistics(X)
: : +- ColumnarExchange (49)
: : +- VeloxResizeBatches (48)
: : +- ^ ProjectExecTransformer (46)
: : +- ^ FilterExecTransformer (45)
- : : +- ^ Scan parquet (44)
+ : : +- ^ ScanTransformer parquet (44)
: +- ^ InputIteratorTransformer (69)
: +- ShuffleQueryStage (67), Statistics(X)
: +- ColumnarExchange (66)
: +- VeloxResizeBatches (65)
: +- ^ ProjectExecTransformer (63)
: +- ^ FilterExecTransformer (62)
- : +- ^ Scan parquet (61)
+ : +- ^ ScanTransformer parquet (61)
+- ^ InputIteratorTransformer (86)
+- ShuffleQueryStage (84), Statistics(X)
+- ColumnarExchange (83)
+- VeloxResizeBatches (82)
+- ^ ProjectExecTransformer (80)
+- ^ FilterExecTransformer (79)
- +- ^ Scan parquet (78)
+ +- ^ ScanTransformer parquet (78)
+- == Initial Plan ==
Sort (155)
+- Exchange (154)
@@ -134,7 +134,7 @@ AdaptiveSparkPlan (156)
+- Scan parquet (144)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [c_custkey#X, c_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -171,7 +171,7 @@ Input [2]: [c_custkey#X, c_nationkey#X]
(9) InputIteratorTransformer
Input [2]: [c_custkey#X, c_nationkey#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -239,7 +239,7 @@ Input [2]: [c_nationkey#X, o_orderkey#X]
(26) InputIteratorTransformer
Input [2]: [c_nationkey#X, o_orderkey#X]
-(27) Scan parquet
+(27) ScanTransformer parquet
Output [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -307,7 +307,7 @@ Input [4]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
(43) InputIteratorTransformer
Input [4]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
-(44) Scan parquet
+(44) ScanTransformer parquet
Output [2]: [s_suppkey#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -375,7 +375,7 @@ Input [3]: [l_extendedprice#X, l_discount#X, s_nationkey#X]
(60) InputIteratorTransformer
Input [3]: [l_extendedprice#X, l_discount#X, s_nationkey#X]
-(61) Scan parquet
+(61) ScanTransformer parquet
Output [3]: [n_nationkey#X, n_name#X, n_regionkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -443,7 +443,7 @@ Input [4]: [l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X]
(77) InputIteratorTransformer
Input [4]: [l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X]
-(78) Scan parquet
+(78) ScanTransformer parquet
Output [2]: [r_regionkey#X, r_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 5df63675caee..d5569132a41f 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
@@ -10,7 +10,7 @@ AdaptiveSparkPlan (20)
+- ^ FlushableHashAggregateExecTransformer (4)
+- ^ ProjectExecTransformer (3)
+- ^ FilterExecTransformer (2)
- +- ^ Scan parquet (1)
+ +- ^ ScanTransformer parquet (1)
+- == Initial Plan ==
HashAggregate (19)
+- Exchange (18)
@@ -20,7 +20,7 @@ AdaptiveSparkPlan (20)
+- Scan parquet (14)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [4]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
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 ac6dda065bb0..ccf77e181d75 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
@@ -46,35 +46,35 @@ AdaptiveSparkPlan (149)
: : : : : +- VeloxResizeBatches (5)
: : : : : +- ^ ProjectExecTransformer (3)
: : : : : +- ^ FilterExecTransformer (2)
- : : : : : +- ^ Scan parquet (1)
+ : : : : : +- ^ ScanTransformer parquet (1)
: : : : +- ^ InputIteratorTransformer (18)
: : : : +- ShuffleQueryStage (16), Statistics(X)
: : : : +- ColumnarExchange (15)
: : : : +- VeloxResizeBatches (14)
: : : : +- ^ ProjectExecTransformer (12)
: : : : +- ^ FilterExecTransformer (11)
- : : : : +- ^ Scan parquet (10)
+ : : : : +- ^ ScanTransformer parquet (10)
: : : +- ^ InputIteratorTransformer (35)
: : : +- ShuffleQueryStage (33), Statistics(X)
: : : +- ColumnarExchange (32)
: : : +- VeloxResizeBatches (31)
: : : +- ^ ProjectExecTransformer (29)
: : : +- ^ FilterExecTransformer (28)
- : : : +- ^ Scan parquet (27)
+ : : : +- ^ ScanTransformer parquet (27)
: : +- ^ InputIteratorTransformer (52)
: : +- ShuffleQueryStage (50), Statistics(X)
: : +- ColumnarExchange (49)
: : +- VeloxResizeBatches (48)
: : +- ^ ProjectExecTransformer (46)
: : +- ^ FilterExecTransformer (45)
- : : +- ^ Scan parquet (44)
+ : : +- ^ ScanTransformer parquet (44)
: +- ^ InputIteratorTransformer (69)
: +- ShuffleQueryStage (67), Statistics(X)
: +- ColumnarExchange (66)
: +- VeloxResizeBatches (65)
: +- ^ ProjectExecTransformer (63)
: +- ^ FilterExecTransformer (62)
- : +- ^ Scan parquet (61)
+ : +- ^ ScanTransformer parquet (61)
+- ^ InputIteratorTransformer (81)
+- ShuffleQueryStage (79), Statistics(X)
+- ReusedExchange (78)
@@ -128,7 +128,7 @@ AdaptiveSparkPlan (149)
+- Scan parquet (138)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [s_suppkey#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -165,7 +165,7 @@ Input [2]: [s_suppkey#X, s_nationkey#X]
(9) InputIteratorTransformer
Input [2]: [s_suppkey#X, s_nationkey#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [5]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -233,7 +233,7 @@ Input [5]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_ship
(26) InputIteratorTransformer
Input [5]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
-(27) Scan parquet
+(27) ScanTransformer parquet
Output [2]: [o_orderkey#X, o_custkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -301,7 +301,7 @@ Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_cust
(43) InputIteratorTransformer
Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X]
-(44) Scan parquet
+(44) ScanTransformer parquet
Output [2]: [c_custkey#X, c_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -369,7 +369,7 @@ Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nati
(60) InputIteratorTransformer
Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X]
-(61) Scan parquet
+(61) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 6f5ed0d82b80..76e38b48de31 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
@@ -59,56 +59,56 @@ AdaptiveSparkPlan (207)
: : : : : : : +- VeloxResizeBatches (5)
: : : : : : : +- ^ ProjectExecTransformer (3)
: : : : : : : +- ^ FilterExecTransformer (2)
- : : : : : : : +- ^ Scan parquet (1)
+ : : : : : : : +- ^ ScanTransformer parquet (1)
: : : : : : +- ^ InputIteratorTransformer (18)
: : : : : : +- ShuffleQueryStage (16), Statistics(X)
: : : : : : +- ColumnarExchange (15)
: : : : : : +- VeloxResizeBatches (14)
: : : : : : +- ^ ProjectExecTransformer (12)
: : : : : : +- ^ FilterExecTransformer (11)
- : : : : : : +- ^ Scan parquet (10)
+ : : : : : : +- ^ ScanTransformer parquet (10)
: : : : : +- ^ InputIteratorTransformer (35)
: : : : : +- ShuffleQueryStage (33), Statistics(X)
: : : : : +- ColumnarExchange (32)
: : : : : +- VeloxResizeBatches (31)
: : : : : +- ^ ProjectExecTransformer (29)
: : : : : +- ^ FilterExecTransformer (28)
- : : : : : +- ^ Scan parquet (27)
+ : : : : : +- ^ ScanTransformer parquet (27)
: : : : +- ^ InputIteratorTransformer (52)
: : : : +- ShuffleQueryStage (50), Statistics(X)
: : : : +- ColumnarExchange (49)
: : : : +- VeloxResizeBatches (48)
: : : : +- ^ ProjectExecTransformer (46)
: : : : +- ^ FilterExecTransformer (45)
- : : : : +- ^ Scan parquet (44)
+ : : : : +- ^ ScanTransformer parquet (44)
: : : +- ^ InputIteratorTransformer (69)
: : : +- ShuffleQueryStage (67), Statistics(X)
: : : +- ColumnarExchange (66)
: : : +- VeloxResizeBatches (65)
: : : +- ^ ProjectExecTransformer (63)
: : : +- ^ FilterExecTransformer (62)
- : : : +- ^ Scan parquet (61)
+ : : : +- ^ ScanTransformer parquet (61)
: : +- ^ InputIteratorTransformer (86)
: : +- ShuffleQueryStage (84), Statistics(X)
: : +- ColumnarExchange (83)
: : +- VeloxResizeBatches (82)
: : +- ^ ProjectExecTransformer (80)
: : +- ^ FilterExecTransformer (79)
- : : +- ^ Scan parquet (78)
+ : : +- ^ ScanTransformer parquet (78)
: +- ^ InputIteratorTransformer (103)
: +- ShuffleQueryStage (101), Statistics(X)
: +- ColumnarExchange (100)
: +- VeloxResizeBatches (99)
: +- ^ ProjectExecTransformer (97)
: +- ^ FilterExecTransformer (96)
- : +- ^ Scan parquet (95)
+ : +- ^ ScanTransformer parquet (95)
+- ^ InputIteratorTransformer (120)
+- ShuffleQueryStage (118), Statistics(X)
+- ColumnarExchange (117)
+- VeloxResizeBatches (116)
+- ^ ProjectExecTransformer (114)
+- ^ FilterExecTransformer (113)
- +- ^ Scan parquet (112)
+ +- ^ ScanTransformer parquet (112)
+- == Initial Plan ==
Sort (206)
+- Exchange (205)
@@ -177,7 +177,7 @@ AdaptiveSparkPlan (207)
+- Scan parquet (195)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [p_partkey#X, p_type#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -214,7 +214,7 @@ Input [1]: [p_partkey#X]
(9) InputIteratorTransformer
Input [1]: [p_partkey#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [5]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -282,7 +282,7 @@ Input [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
(26) InputIteratorTransformer
Input [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
-(27) Scan parquet
+(27) ScanTransformer parquet
Output [2]: [s_suppkey#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -350,7 +350,7 @@ Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X]
(43) InputIteratorTransformer
Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X]
-(44) Scan parquet
+(44) ScanTransformer parquet
Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -418,7 +418,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_order
(60) InputIteratorTransformer
Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X]
-(61) Scan parquet
+(61) ScanTransformer parquet
Output [2]: [c_custkey#X, c_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -486,7 +486,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nat
(77) InputIteratorTransformer
Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X]
-(78) Scan parquet
+(78) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_regionkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -554,7 +554,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_reg
(94) InputIteratorTransformer
Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X]
-(95) Scan parquet
+(95) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -622,7 +622,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_nam
(111) InputIteratorTransformer
Input [5]: [l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X]
-(112) Scan parquet
+(112) ScanTransformer parquet
Output [2]: [r_regionkey#X, r_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 22cd2b9967fb..3c152f565673 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
@@ -46,42 +46,42 @@ AdaptiveSparkPlan (155)
: : : : : +- VeloxResizeBatches (5)
: : : : : +- ^ ProjectExecTransformer (3)
: : : : : +- ^ FilterExecTransformer (2)
- : : : : : +- ^ Scan parquet (1)
+ : : : : : +- ^ ScanTransformer parquet (1)
: : : : +- ^ InputIteratorTransformer (18)
: : : : +- ShuffleQueryStage (16), Statistics(X)
: : : : +- ColumnarExchange (15)
: : : : +- VeloxResizeBatches (14)
: : : : +- ^ ProjectExecTransformer (12)
: : : : +- ^ FilterExecTransformer (11)
- : : : : +- ^ Scan parquet (10)
+ : : : : +- ^ ScanTransformer parquet (10)
: : : +- ^ InputIteratorTransformer (35)
: : : +- ShuffleQueryStage (33), Statistics(X)
: : : +- ColumnarExchange (32)
: : : +- VeloxResizeBatches (31)
: : : +- ^ ProjectExecTransformer (29)
: : : +- ^ FilterExecTransformer (28)
- : : : +- ^ Scan parquet (27)
+ : : : +- ^ ScanTransformer parquet (27)
: : +- ^ InputIteratorTransformer (52)
: : +- ShuffleQueryStage (50), Statistics(X)
: : +- ColumnarExchange (49)
: : +- VeloxResizeBatches (48)
: : +- ^ ProjectExecTransformer (46)
: : +- ^ FilterExecTransformer (45)
- : : +- ^ Scan parquet (44)
+ : : +- ^ ScanTransformer parquet (44)
: +- ^ InputIteratorTransformer (69)
: +- ShuffleQueryStage (67), Statistics(X)
: +- ColumnarExchange (66)
: +- VeloxResizeBatches (65)
: +- ^ ProjectExecTransformer (63)
: +- ^ FilterExecTransformer (62)
- : +- ^ Scan parquet (61)
+ : +- ^ ScanTransformer parquet (61)
+- ^ InputIteratorTransformer (86)
+- ShuffleQueryStage (84), Statistics(X)
+- ColumnarExchange (83)
+- VeloxResizeBatches (82)
+- ^ ProjectExecTransformer (80)
+- ^ FilterExecTransformer (79)
- +- ^ Scan parquet (78)
+ +- ^ ScanTransformer parquet (78)
+- == Initial Plan ==
Sort (154)
+- Exchange (153)
@@ -133,7 +133,7 @@ AdaptiveSparkPlan (155)
+- Scan parquet (144)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [p_partkey#X, p_name#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -170,7 +170,7 @@ Input [1]: [p_partkey#X]
(9) InputIteratorTransformer
Input [1]: [p_partkey#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -238,7 +238,7 @@ Input [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpric
(26) InputIteratorTransformer
Input [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X]
-(27) Scan parquet
+(27) ScanTransformer parquet
Output [2]: [s_suppkey#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -306,7 +306,7 @@ Input [7]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpric
(43) InputIteratorTransformer
Input [7]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X]
-(44) Scan parquet
+(44) ScanTransformer parquet
Output [3]: [ps_partkey#X, ps_suppkey#X, ps_supplycost#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -374,7 +374,7 @@ Input [6]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_natio
(60) InputIteratorTransformer
Input [6]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X]
-(61) Scan parquet
+(61) ScanTransformer parquet
Output [2]: [o_orderkey#X, o_orderdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -442,7 +442,7 @@ Input [6]: [l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_sup
(77) InputIteratorTransformer
Input [6]: [l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X]
-(78) Scan parquet
+(78) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 6dbee1ad7ffb..4d61f904dbe9 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
@@ -16,7 +16,7 @@ AdaptiveSparkPlan (30)
+- ^ FlushableHashAggregateExecTransformer (4)
+- ^ ProjectExecTransformer (3)
+- ^ FilterExecTransformer (2)
- +- ^ Scan parquet (1)
+ +- ^ ScanTransformer parquet (1)
+- == Initial Plan ==
Sort (29)
+- Exchange (28)
@@ -28,7 +28,7 @@ AdaptiveSparkPlan (30)
+- Scan parquet (22)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [7]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_tax#X, l_returnflag#X, l_linestatus#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
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 a052989ba525..1a64123cc8c0 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
@@ -31,28 +31,28 @@ AdaptiveSparkPlan (100)
: : : +- VeloxResizeBatches (5)
: : : +- ^ ProjectExecTransformer (3)
: : : +- ^ FilterExecTransformer (2)
- : : : +- ^ Scan parquet (1)
+ : : : +- ^ ScanTransformer parquet (1)
: : +- ^ InputIteratorTransformer (18)
: : +- ShuffleQueryStage (16), Statistics(X)
: : +- ColumnarExchange (15)
: : +- VeloxResizeBatches (14)
: : +- ^ ProjectExecTransformer (12)
: : +- ^ FilterExecTransformer (11)
- : : +- ^ Scan parquet (10)
+ : : +- ^ ScanTransformer parquet (10)
: +- ^ InputIteratorTransformer (35)
: +- ShuffleQueryStage (33), Statistics(X)
: +- ColumnarExchange (32)
: +- VeloxResizeBatches (31)
: +- ^ ProjectExecTransformer (29)
: +- ^ FilterExecTransformer (28)
- : +- ^ Scan parquet (27)
+ : +- ^ ScanTransformer parquet (27)
+- ^ InputIteratorTransformer (52)
+- ShuffleQueryStage (50), Statistics(X)
+- ColumnarExchange (49)
+- VeloxResizeBatches (48)
+- ^ ProjectExecTransformer (46)
+- ^ FilterExecTransformer (45)
- +- ^ Scan parquet (44)
+ +- ^ ScanTransformer parquet (44)
+- == Initial Plan ==
TakeOrderedAndProject (99)
+- HashAggregate (98)
@@ -88,7 +88,7 @@ AdaptiveSparkPlan (100)
+- Scan parquet (90)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -125,7 +125,7 @@ Input [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct
(9) InputIteratorTransformer
Input [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -194,7 +194,7 @@ Input [8]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct
(26) InputIteratorTransformer
Input [8]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, o_orderkey#X]
-(27) Scan parquet
+(27) ScanTransformer parquet
Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_returnflag#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -263,7 +263,7 @@ Input [9]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct
(43) InputIteratorTransformer
Input [9]: [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]
-(44) Scan parquet
+(44) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 68e87c39a078..e2d5e58eba47 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
@@ -29,21 +29,21 @@ AdaptiveSparkPlan (82)
: : +- VeloxResizeBatches (5)
: : +- ^ ProjectExecTransformer (3)
: : +- ^ FilterExecTransformer (2)
- : : +- ^ Scan parquet (1)
+ : : +- ^ ScanTransformer parquet (1)
: +- ^ InputIteratorTransformer (18)
: +- ShuffleQueryStage (16), Statistics(X)
: +- ColumnarExchange (15)
: +- VeloxResizeBatches (14)
: +- ^ ProjectExecTransformer (12)
: +- ^ FilterExecTransformer (11)
- : +- ^ Scan parquet (10)
+ : +- ^ ScanTransformer parquet (10)
+- ^ InputIteratorTransformer (35)
+- ShuffleQueryStage (33), Statistics(X)
+- ColumnarExchange (32)
+- VeloxResizeBatches (31)
+- ^ ProjectExecTransformer (29)
+- ^ FilterExecTransformer (28)
- +- ^ Scan parquet (27)
+ +- ^ ScanTransformer parquet (27)
+- == Initial Plan ==
Sort (81)
+- Exchange (80)
@@ -72,7 +72,7 @@ AdaptiveSparkPlan (82)
+- Scan parquet (69)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -109,7 +109,7 @@ Input [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X]
(9) InputIteratorTransformer
Input [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [2]: [s_suppkey#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -178,7 +178,7 @@ Input [4]: [ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X]
(26) InputIteratorTransformer
Input [4]: [ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X]
-(27) Scan parquet
+(27) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -428,58 +428,57 @@ Arguments: isFinalPlan=true
===== Subqueries =====
Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery subquery#X, [id=#X]
-AdaptiveSparkPlan (136)
+AdaptiveSparkPlan (135)
+- == Final Plan ==
- VeloxColumnarToRow (114)
- +- ^ ProjectExecTransformer (112)
- +- ^ RegularHashAggregateExecTransformer (111)
- +- ^ RegularHashAggregateExecTransformer (110)
- +- ^ ProjectExecTransformer (109)
- +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (108)
- :- ^ InputIteratorTransformer (103)
- : +- ShuffleQueryStage (101), Statistics(X)
- : +- ColumnarExchange (100)
- : +- VeloxResizeBatches (99)
- : +- ^ ProjectExecTransformer (97)
- : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (96)
- : :- ^ InputIteratorTransformer (91)
- : : +- ShuffleQueryStage (89), Statistics(X)
- : : +- ColumnarExchange (88)
- : : +- VeloxResizeBatches (87)
- : : +- ^ ProjectExecTransformer (85)
- : : +- ^ FilterExecTransformer (84)
- : : +- ^ Scan parquet (83)
- : +- ^ InputIteratorTransformer (95)
- : +- ShuffleQueryStage (93), Statistics(X)
- : +- ReusedExchange (92)
- +- ^ InputIteratorTransformer (107)
- +- ShuffleQueryStage (105), Statistics(X)
- +- ReusedExchange (104)
+ VeloxColumnarToRow (113)
+ +- ^ ProjectExecTransformer (111)
+ +- ^ RegularHashAggregateExecTransformer (110)
+ +- ^ ProjectExecTransformer (109)
+ +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (108)
+ :- ^ InputIteratorTransformer (103)
+ : +- ShuffleQueryStage (101), Statistics(X)
+ : +- ColumnarExchange (100)
+ : +- VeloxResizeBatches (99)
+ : +- ^ ProjectExecTransformer (97)
+ : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (96)
+ : :- ^ InputIteratorTransformer (91)
+ : : +- ShuffleQueryStage (89), Statistics(X)
+ : : +- ColumnarExchange (88)
+ : : +- VeloxResizeBatches (87)
+ : : +- ^ ProjectExecTransformer (85)
+ : : +- ^ FilterExecTransformer (84)
+ : : +- ^ ScanTransformer parquet (83)
+ : +- ^ InputIteratorTransformer (95)
+ : +- ShuffleQueryStage (93), Statistics(X)
+ : +- ReusedExchange (92)
+ +- ^ InputIteratorTransformer (107)
+ +- ShuffleQueryStage (105), Statistics(X)
+ +- ReusedExchange (104)
+- == Initial Plan ==
- HashAggregate (135)
- +- HashAggregate (134)
- +- Project (133)
- +- SortMergeJoin Inner (132)
- :- Sort (126)
- : +- Exchange (125)
- : +- Project (124)
- : +- SortMergeJoin Inner (123)
- : :- Sort (118)
- : : +- Exchange (117)
- : : +- Filter (116)
- : : +- Scan parquet (115)
- : +- Sort (122)
- : +- Exchange (121)
- : +- Filter (120)
- : +- Scan parquet (119)
- +- Sort (131)
- +- Exchange (130)
- +- Project (129)
- +- Filter (128)
- +- Scan parquet (127)
-
-
-(83) Scan parquet
+ HashAggregate (134)
+ +- HashAggregate (133)
+ +- Project (132)
+ +- SortMergeJoin Inner (131)
+ :- Sort (125)
+ : +- Exchange (124)
+ : +- Project (123)
+ : +- SortMergeJoin Inner (122)
+ : :- Sort (117)
+ : : +- Exchange (116)
+ : : +- Filter (115)
+ : : +- Scan parquet (114)
+ : +- Sort (121)
+ : +- Exchange (120)
+ : +- Filter (119)
+ : +- Scan parquet (118)
+ +- Sort (130)
+ +- Exchange (129)
+ +- Project (128)
+ +- Filter (127)
+ +- Scan parquet (126)
+
+
+(83) ScanTransformer parquet
Output [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -587,131 +586,124 @@ Input [4]: [ps_availqty#X, ps_supplycost#X, s_nationkey#X, n_nationkey#X]
(110) RegularHashAggregateExecTransformer
Input [3]: [ps_availqty#X, ps_supplycost#X, _pre_X#X]
Keys: []
-Functions [1]: [partial_sum(_pre_X#X)]
-Aggregate Attributes [2]: [sum#X, isEmpty#X]
-Results [2]: [sum#X, isEmpty#X]
-
-(111) RegularHashAggregateExecTransformer
-Input [2]: [sum#X, isEmpty#X]
-Keys: []
-Functions [1]: [sum((ps_supplycost#X * cast(ps_availqty#X as decimal(10,0))))]
+Functions [1]: [sum(_pre_X#X)]
Aggregate Attributes [1]: [sum((ps_supplycost#X * cast(ps_availqty#X as decimal(10,0))))#X]
Results [1]: [sum((ps_supplycost#X * cast(ps_availqty#X as decimal(10,0))))#X]
-(112) ProjectExecTransformer
+(111) ProjectExecTransformer
Output [1]: [(sum((ps_supplycost#X * cast(ps_availqty#X as decimal(10,0))))#X * 0.0001000000) AS (sum((ps_supplycost * ps_availqty)) * 0.0001000000)#X]
Input [1]: [sum((ps_supplycost#X * cast(ps_availqty#X as decimal(10,0))))#X]
-(113) WholeStageCodegenTransformer (X)
+(112) WholeStageCodegenTransformer (X)
Input [1]: [(sum((ps_supplycost * ps_availqty)) * 0.0001000000)#X]
Arguments: false
-(114) VeloxColumnarToRow
+(113) VeloxColumnarToRow
Input [1]: [(sum((ps_supplycost * ps_availqty)) * 0.0001000000)#X]
-(115) Scan parquet
+(114) Scan parquet
Output [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(ps_suppkey)]
ReadSchema: struct
-(116) Filter
+(115) Filter
Input [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X]
Condition : isnotnull(ps_suppkey#X)
-(117) Exchange
+(116) Exchange
Input [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X]
Arguments: hashpartitioning(ps_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(118) Sort
+(117) Sort
Input [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X]
Arguments: [ps_suppkey#X ASC NULLS FIRST], false, 0
-(119) Scan parquet
+(118) Scan parquet
Output [2]: [s_suppkey#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(s_suppkey), IsNotNull(s_nationkey)]
ReadSchema: struct
-(120) Filter
+(119) Filter
Input [2]: [s_suppkey#X, s_nationkey#X]
Condition : (isnotnull(s_suppkey#X) AND isnotnull(s_nationkey#X))
-(121) Exchange
+(120) Exchange
Input [2]: [s_suppkey#X, s_nationkey#X]
Arguments: hashpartitioning(s_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(122) Sort
+(121) Sort
Input [2]: [s_suppkey#X, s_nationkey#X]
Arguments: [s_suppkey#X ASC NULLS FIRST], false, 0
-(123) SortMergeJoin
+(122) SortMergeJoin
Left keys [1]: [ps_suppkey#X]
Right keys [1]: [s_suppkey#X]
Join type: Inner
Join condition: None
-(124) Project
+(123) Project
Output [3]: [ps_availqty#X, ps_supplycost#X, s_nationkey#X]
Input [5]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X, s_suppkey#X, s_nationkey#X]
-(125) Exchange
+(124) Exchange
Input [3]: [ps_availqty#X, ps_supplycost#X, s_nationkey#X]
Arguments: hashpartitioning(s_nationkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(126) Sort
+(125) Sort
Input [3]: [ps_availqty#X, ps_supplycost#X, s_nationkey#X]
Arguments: [s_nationkey#X ASC NULLS FIRST], false, 0
-(127) Scan parquet
+(126) Scan parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(n_name), EqualTo(n_name,GERMANY), IsNotNull(n_nationkey)]
ReadSchema: struct
-(128) Filter
+(127) Filter
Input [2]: [n_nationkey#X, n_name#X]
Condition : ((isnotnull(n_name#X) AND (n_name#X = GERMANY)) AND isnotnull(n_nationkey#X))
-(129) Project
+(128) Project
Output [1]: [n_nationkey#X]
Input [2]: [n_nationkey#X, n_name#X]
-(130) Exchange
+(129) Exchange
Input [1]: [n_nationkey#X]
Arguments: hashpartitioning(n_nationkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(131) Sort
+(130) Sort
Input [1]: [n_nationkey#X]
Arguments: [n_nationkey#X ASC NULLS FIRST], false, 0
-(132) SortMergeJoin
+(131) SortMergeJoin
Left keys [1]: [s_nationkey#X]
Right keys [1]: [n_nationkey#X]
Join type: Inner
Join condition: None
-(133) Project
+(132) Project
Output [2]: [ps_availqty#X, ps_supplycost#X]
Input [4]: [ps_availqty#X, ps_supplycost#X, s_nationkey#X, n_nationkey#X]
-(134) HashAggregate
+(133) HashAggregate
Input [2]: [ps_availqty#X, ps_supplycost#X]
Keys: []
Functions [1]: [partial_sum((ps_supplycost#X * cast(ps_availqty#X as decimal(10,0))))]
Aggregate Attributes [2]: [sum#X, isEmpty#X]
Results [2]: [sum#X, isEmpty#X]
-(135) HashAggregate
+(134) HashAggregate
Input [2]: [sum#X, isEmpty#X]
Keys: []
Functions [1]: [sum((ps_supplycost#X * cast(ps_availqty#X as decimal(10,0))))]
Aggregate Attributes [1]: [sum((ps_supplycost#X * cast(ps_availqty#X as decimal(10,0))))#X]
Results [1]: [(sum((ps_supplycost#X * cast(ps_availqty#X as decimal(10,0))))#X * 0.0001000000) AS (sum((ps_supplycost * ps_availqty)) * 0.0001000000)#X]
-(136) AdaptiveSparkPlan
+(135) AdaptiveSparkPlan
Output [1]: [(sum((ps_supplycost * ps_availqty)) * 0.0001000000)#X]
Arguments: isFinalPlan=true
\ No newline at end of file
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 9175845083ca..3bfd39bdaf49 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
@@ -22,14 +22,14 @@ AdaptiveSparkPlan (55)
: +- VeloxResizeBatches (5)
: +- ^ ProjectExecTransformer (3)
: +- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
+ : +- ^ ScanTransformer parquet (1)
+- ^ InputIteratorTransformer (18)
+- ShuffleQueryStage (16), Statistics(X)
+- ColumnarExchange (15)
+- VeloxResizeBatches (14)
+- ^ ProjectExecTransformer (12)
+- ^ FilterExecTransformer (11)
- +- ^ Scan parquet (10)
+ +- ^ ScanTransformer parquet (10)
+- == Initial Plan ==
Sort (54)
+- Exchange (53)
@@ -49,7 +49,7 @@ AdaptiveSparkPlan (55)
+- Scan parquet (43)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [o_orderkey#X, o_orderpriority#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -86,7 +86,7 @@ Input [2]: [o_orderkey#X, o_orderpriority#X]
(9) InputIteratorTransformer
Input [2]: [o_orderkey#X, o_orderpriority#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [5]: [l_orderkey#X, l_shipdate#X, l_commitdate#X, l_receiptdate#X, l_shipmode#X]
Batched: true
Location: InMemoryFileIndex [*]
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 b527c2bc3bc3..59abfb682c42 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
@@ -1,58 +1,57 @@
== Physical Plan ==
-AdaptiveSparkPlan (58)
+AdaptiveSparkPlan (57)
+- == Final Plan ==
- VeloxColumnarToRow (40)
- +- ^ SortExecTransformer (38)
- +- ^ InputIteratorTransformer (37)
- +- ShuffleQueryStage (35), Statistics(X)
- +- ColumnarExchange (34)
- +- VeloxResizeBatches (33)
- +- ^ RegularHashAggregateExecTransformer (31)
- +- ^ InputIteratorTransformer (30)
- +- ShuffleQueryStage (28), Statistics(X)
- +- ColumnarExchange (27)
- +- VeloxResizeBatches (26)
- +- ^ ProjectExecTransformer (24)
- +- ^ FlushableHashAggregateExecTransformer (23)
- +- ^ ProjectExecTransformer (22)
- +- ^ RegularHashAggregateExecTransformer (21)
- +- ^ RegularHashAggregateExecTransformer (20)
- +- ^ ProjectExecTransformer (19)
- +- ^ ShuffledHashJoinExecTransformer LeftOuter BuildLeft (18)
- :- ^ InputIteratorTransformer (8)
- : +- ShuffleQueryStage (6), Statistics(X)
- : +- ColumnarExchange (5)
- : +- VeloxResizeBatches (4)
- : +- ^ ProjectExecTransformer (2)
- : +- ^ Scan parquet (1)
- +- ^ InputIteratorTransformer (17)
- +- ShuffleQueryStage (15), Statistics(X)
- +- ColumnarExchange (14)
- +- VeloxResizeBatches (13)
- +- ^ ProjectExecTransformer (11)
- +- ^ FilterExecTransformer (10)
- +- ^ Scan parquet (9)
+ VeloxColumnarToRow (39)
+ +- ^ SortExecTransformer (37)
+ +- ^ InputIteratorTransformer (36)
+ +- ShuffleQueryStage (34), Statistics(X)
+ +- ColumnarExchange (33)
+ +- VeloxResizeBatches (32)
+ +- ^ RegularHashAggregateExecTransformer (30)
+ +- ^ InputIteratorTransformer (29)
+ +- ShuffleQueryStage (27), Statistics(X)
+ +- ColumnarExchange (26)
+ +- VeloxResizeBatches (25)
+ +- ^ ProjectExecTransformer (23)
+ +- ^ FlushableHashAggregateExecTransformer (22)
+ +- ^ ProjectExecTransformer (21)
+ +- ^ RegularHashAggregateExecTransformer (20)
+ +- ^ ProjectExecTransformer (19)
+ +- ^ ShuffledHashJoinExecTransformer LeftOuter BuildLeft (18)
+ :- ^ InputIteratorTransformer (8)
+ : +- ShuffleQueryStage (6), Statistics(X)
+ : +- ColumnarExchange (5)
+ : +- VeloxResizeBatches (4)
+ : +- ^ ProjectExecTransformer (2)
+ : +- ^ ScanTransformer parquet (1)
+ +- ^ InputIteratorTransformer (17)
+ +- ShuffleQueryStage (15), Statistics(X)
+ +- ColumnarExchange (14)
+ +- VeloxResizeBatches (13)
+ +- ^ ProjectExecTransformer (11)
+ +- ^ FilterExecTransformer (10)
+ +- ^ ScanTransformer parquet (9)
+- == Initial Plan ==
- Sort (57)
- +- Exchange (56)
- +- HashAggregate (55)
- +- Exchange (54)
- +- HashAggregate (53)
- +- HashAggregate (52)
- +- HashAggregate (51)
- +- Project (50)
- +- SortMergeJoin LeftOuter (49)
- :- Sort (43)
- : +- Exchange (42)
- : +- Scan parquet (41)
- +- Sort (48)
- +- Exchange (47)
- +- Project (46)
- +- Filter (45)
- +- Scan parquet (44)
-
-
-(1) Scan parquet
+ Sort (56)
+ +- Exchange (55)
+ +- HashAggregate (54)
+ +- Exchange (53)
+ +- HashAggregate (52)
+ +- HashAggregate (51)
+ +- HashAggregate (50)
+ +- Project (49)
+ +- SortMergeJoin LeftOuter (48)
+ :- Sort (42)
+ : +- Exchange (41)
+ : +- Scan parquet (40)
+ +- Sort (47)
+ +- Exchange (46)
+ +- Project (45)
+ +- Filter (44)
+ +- Scan parquet (43)
+
+
+(1) ScanTransformer parquet
Output [1]: [c_custkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -84,7 +83,7 @@ Input [1]: [c_custkey#X]
(8) InputIteratorTransformer
Input [1]: [c_custkey#X]
-(9) Scan parquet
+(9) ScanTransformer parquet
Output [3]: [o_orderkey#X, o_custkey#X, o_comment#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -134,181 +133,174 @@ Input [3]: [c_custkey#X, o_orderkey#X, o_custkey#X]
(20) RegularHashAggregateExecTransformer
Input [2]: [c_custkey#X, o_orderkey#X]
Keys [1]: [c_custkey#X]
-Functions [1]: [partial_count(o_orderkey#X)]
-Aggregate Attributes [1]: [count#X]
-Results [2]: [c_custkey#X, count#X]
-
-(21) RegularHashAggregateExecTransformer
-Input [2]: [c_custkey#X, count#X]
-Keys [1]: [c_custkey#X]
Functions [1]: [count(o_orderkey#X)]
Aggregate Attributes [1]: [count(o_orderkey#X)#X]
Results [2]: [c_custkey#X, count(o_orderkey#X)#X]
-(22) ProjectExecTransformer
+(21) ProjectExecTransformer
Output [1]: [count(o_orderkey#X)#X AS c_count#X]
Input [2]: [c_custkey#X, count(o_orderkey#X)#X]
-(23) FlushableHashAggregateExecTransformer
+(22) FlushableHashAggregateExecTransformer
Input [1]: [c_count#X]
Keys [1]: [c_count#X]
Functions [1]: [partial_count(1)]
Aggregate Attributes [1]: [count#X]
Results [2]: [c_count#X, count#X]
-(24) ProjectExecTransformer
+(23) ProjectExecTransformer
Output [3]: [hash(c_count#X, 42) AS hash_partition_key#X, c_count#X, count#X]
Input [2]: [c_count#X, count#X]
-(25) WholeStageCodegenTransformer (X)
+(24) WholeStageCodegenTransformer (X)
Input [3]: [hash_partition_key#X, c_count#X, count#X]
Arguments: false
-(26) VeloxResizeBatches
+(25) VeloxResizeBatches
Input [3]: [hash_partition_key#X, c_count#X, count#X]
Arguments: X, X
-(27) ColumnarExchange
+(26) ColumnarExchange
Input [3]: [hash_partition_key#X, c_count#X, count#X]
Arguments: hashpartitioning(c_count#X, 1), ENSURE_REQUIREMENTS, [c_count#X, count#X], [plan_id=X], [shuffle_writer_type=hash]
-(28) ShuffleQueryStage
+(27) ShuffleQueryStage
Output [2]: [c_count#X, count#X]
Arguments: X
-(29) InputAdapter
+(28) InputAdapter
Input [2]: [c_count#X, count#X]
-(30) InputIteratorTransformer
+(29) InputIteratorTransformer
Input [2]: [c_count#X, count#X]
-(31) RegularHashAggregateExecTransformer
+(30) RegularHashAggregateExecTransformer
Input [2]: [c_count#X, count#X]
Keys [1]: [c_count#X]
Functions [1]: [count(1)]
Aggregate Attributes [1]: [count(1)#X]
Results [2]: [c_count#X, count(1)#X AS custdist#X]
-(32) WholeStageCodegenTransformer (X)
+(31) WholeStageCodegenTransformer (X)
Input [2]: [c_count#X, custdist#X]
Arguments: false
-(33) VeloxResizeBatches
+(32) VeloxResizeBatches
Input [2]: [c_count#X, custdist#X]
Arguments: X, X
-(34) ColumnarExchange
+(33) ColumnarExchange
Input [2]: [c_count#X, custdist#X]
Arguments: rangepartitioning(custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST, 1), ENSURE_REQUIREMENTS, [plan_id=X], [shuffle_writer_type=hash]
-(35) ShuffleQueryStage
+(34) ShuffleQueryStage
Output [2]: [c_count#X, custdist#X]
Arguments: X
-(36) InputAdapter
+(35) InputAdapter
Input [2]: [c_count#X, custdist#X]
-(37) InputIteratorTransformer
+(36) InputIteratorTransformer
Input [2]: [c_count#X, custdist#X]
-(38) SortExecTransformer
+(37) SortExecTransformer
Input [2]: [c_count#X, custdist#X]
Arguments: [custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST], true, 0
-(39) WholeStageCodegenTransformer (X)
+(38) WholeStageCodegenTransformer (X)
Input [2]: [c_count#X, custdist#X]
Arguments: false
-(40) VeloxColumnarToRow
+(39) VeloxColumnarToRow
Input [2]: [c_count#X, custdist#X]
-(41) Scan parquet
+(40) Scan parquet
Output [1]: [c_custkey#X]
Batched: true
Location: InMemoryFileIndex [*]
ReadSchema: struct
-(42) Exchange
+(41) Exchange
Input [1]: [c_custkey#X]
Arguments: hashpartitioning(c_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(43) Sort
+(42) Sort
Input [1]: [c_custkey#X]
Arguments: [c_custkey#X ASC NULLS FIRST], false, 0
-(44) Scan parquet
+(43) Scan parquet
Output [3]: [o_orderkey#X, o_custkey#X, o_comment#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(o_comment), IsNotNull(o_custkey)]
ReadSchema: struct
-(45) Filter
+(44) Filter
Input [3]: [o_orderkey#X, o_custkey#X, o_comment#X]
Condition : ((isnotnull(o_comment#X) AND NOT o_comment#X LIKE %special%requests%) AND isnotnull(o_custkey#X))
-(46) Project
+(45) Project
Output [2]: [o_orderkey#X, o_custkey#X]
Input [3]: [o_orderkey#X, o_custkey#X, o_comment#X]
-(47) Exchange
+(46) Exchange
Input [2]: [o_orderkey#X, o_custkey#X]
Arguments: hashpartitioning(o_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(48) Sort
+(47) Sort
Input [2]: [o_orderkey#X, o_custkey#X]
Arguments: [o_custkey#X ASC NULLS FIRST], false, 0
-(49) SortMergeJoin
+(48) SortMergeJoin
Left keys [1]: [c_custkey#X]
Right keys [1]: [o_custkey#X]
Join type: LeftOuter
Join condition: None
-(50) Project
+(49) Project
Output [2]: [c_custkey#X, o_orderkey#X]
Input [3]: [c_custkey#X, o_orderkey#X, o_custkey#X]
-(51) HashAggregate
+(50) HashAggregate
Input [2]: [c_custkey#X, o_orderkey#X]
Keys [1]: [c_custkey#X]
Functions [1]: [partial_count(o_orderkey#X)]
Aggregate Attributes [1]: [count#X]
Results [2]: [c_custkey#X, count#X]
-(52) HashAggregate
+(51) HashAggregate
Input [2]: [c_custkey#X, count#X]
Keys [1]: [c_custkey#X]
Functions [1]: [count(o_orderkey#X)]
Aggregate Attributes [1]: [count(o_orderkey#X)#X]
Results [1]: [count(o_orderkey#X)#X AS c_count#X]
-(53) HashAggregate
+(52) HashAggregate
Input [1]: [c_count#X]
Keys [1]: [c_count#X]
Functions [1]: [partial_count(1)]
Aggregate Attributes [1]: [count#X]
Results [2]: [c_count#X, count#X]
-(54) Exchange
+(53) Exchange
Input [2]: [c_count#X, count#X]
Arguments: hashpartitioning(c_count#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(55) HashAggregate
+(54) HashAggregate
Input [2]: [c_count#X, count#X]
Keys [1]: [c_count#X]
Functions [1]: [count(1)]
Aggregate Attributes [1]: [count(1)#X]
Results [2]: [c_count#X, count(1)#X AS custdist#X]
-(56) Exchange
+(55) Exchange
Input [2]: [c_count#X, custdist#X]
Arguments: rangepartitioning(custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(57) Sort
+(56) Sort
Input [2]: [c_count#X, custdist#X]
Arguments: [custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST], true, 0
-(58) AdaptiveSparkPlan
+(57) AdaptiveSparkPlan
Output [2]: [c_count#X, custdist#X]
Arguments: isFinalPlan=true
\ No newline at end of file
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 cc6d364dea22..531cc5aaab55 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
@@ -1,43 +1,42 @@
== Physical Plan ==
-AdaptiveSparkPlan (39)
+AdaptiveSparkPlan (38)
+- == Final Plan ==
- VeloxColumnarToRow (25)
- +- ^ ProjectExecTransformer (23)
- +- ^ RegularHashAggregateExecTransformer (22)
- +- ^ RegularHashAggregateExecTransformer (21)
- +- ^ ProjectExecTransformer (20)
- +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19)
- :- ^ InputIteratorTransformer (9)
- : +- ShuffleQueryStage (7), Statistics(X)
- : +- ColumnarExchange (6)
- : +- VeloxResizeBatches (5)
- : +- ^ ProjectExecTransformer (3)
- : +- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
- +- ^ InputIteratorTransformer (18)
- +- ShuffleQueryStage (16), Statistics(X)
- +- ColumnarExchange (15)
- +- VeloxResizeBatches (14)
- +- ^ ProjectExecTransformer (12)
- +- ^ FilterExecTransformer (11)
- +- ^ Scan parquet (10)
+ VeloxColumnarToRow (24)
+ +- ^ ProjectExecTransformer (22)
+ +- ^ RegularHashAggregateExecTransformer (21)
+ +- ^ ProjectExecTransformer (20)
+ +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19)
+ :- ^ InputIteratorTransformer (9)
+ : +- ShuffleQueryStage (7), Statistics(X)
+ : +- ColumnarExchange (6)
+ : +- VeloxResizeBatches (5)
+ : +- ^ ProjectExecTransformer (3)
+ : +- ^ FilterExecTransformer (2)
+ : +- ^ ScanTransformer parquet (1)
+ +- ^ InputIteratorTransformer (18)
+ +- ShuffleQueryStage (16), Statistics(X)
+ +- ColumnarExchange (15)
+ +- VeloxResizeBatches (14)
+ +- ^ ProjectExecTransformer (12)
+ +- ^ FilterExecTransformer (11)
+ +- ^ ScanTransformer parquet (10)
+- == Initial Plan ==
- HashAggregate (38)
- +- HashAggregate (37)
- +- Project (36)
- +- SortMergeJoin Inner (35)
- :- Sort (30)
- : +- Exchange (29)
- : +- Project (28)
- : +- Filter (27)
- : +- Scan parquet (26)
- +- Sort (34)
- +- Exchange (33)
- +- Filter (32)
- +- Scan parquet (31)
-
-
-(1) Scan parquet
+ HashAggregate (37)
+ +- HashAggregate (36)
+ +- Project (35)
+ +- SortMergeJoin Inner (34)
+ :- Sort (29)
+ : +- Exchange (28)
+ : +- Project (27)
+ : +- Filter (26)
+ : +- Scan parquet (25)
+ +- Sort (33)
+ +- Exchange (32)
+ +- Filter (31)
+ +- Scan parquet (30)
+
+
+(1) ScanTransformer parquet
Output [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -74,7 +73,7 @@ Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X]
(9) InputIteratorTransformer
Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [2]: [p_partkey#X, p_type#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -124,94 +123,87 @@ Input [5]: [l_partkey#X, l_extendedprice#X, l_discount#X, p_partkey#X, p_type#X]
(21) RegularHashAggregateExecTransformer
Input [5]: [l_extendedprice#X, l_discount#X, p_type#X, _pre_X#X, _pre_X#X]
Keys: []
-Functions [2]: [partial_sum(_pre_X#X), partial_sum(_pre_X#X)]
-Aggregate Attributes [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X]
-Results [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X]
-
-(22) RegularHashAggregateExecTransformer
-Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X]
-Keys: []
-Functions [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN (l_extendedprice#X * (1 - l_discount#X)) ELSE 0.0000 END), sum((l_extendedprice#X * (1 - l_discount#X)))]
+Functions [2]: [sum(_pre_X#X), sum(_pre_X#X)]
Aggregate Attributes [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN (l_extendedprice#X * (1 - l_discount#X)) ELSE 0.0000 END)#X, sum((l_extendedprice#X * (1 - l_discount#X)))#X]
Results [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN (l_extendedprice#X * (1 - l_discount#X)) ELSE 0.0000 END)#X, sum((l_extendedprice#X * (1 - l_discount#X)))#X]
-(23) ProjectExecTransformer
+(22) ProjectExecTransformer
Output [1]: [((100.00 * sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN (l_extendedprice#X * (1 - l_discount#X)) ELSE 0.0000 END)#X) / sum((l_extendedprice#X * (1 - l_discount#X)))#X) AS promo_revenue#X]
Input [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN (l_extendedprice#X * (1 - l_discount#X)) ELSE 0.0000 END)#X, sum((l_extendedprice#X * (1 - l_discount#X)))#X]
-(24) WholeStageCodegenTransformer (X)
+(23) WholeStageCodegenTransformer (X)
Input [1]: [promo_revenue#X]
Arguments: false
-(25) VeloxColumnarToRow
+(24) VeloxColumnarToRow
Input [1]: [promo_revenue#X]
-(26) Scan parquet
+(25) Scan parquet
Output [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1995-09-01), LessThan(l_shipdate,1995-10-01), IsNotNull(l_partkey)]
ReadSchema: struct
-(27) Filter
+(26) Filter
Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Condition : (((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1995-09-01)) AND (l_shipdate#X < 1995-10-01)) AND isnotnull(l_partkey#X))
-(28) Project
+(27) Project
Output [3]: [l_partkey#X, l_extendedprice#X, l_discount#X]
Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
-(29) Exchange
+(28) Exchange
Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X]
Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(30) Sort
+(29) Sort
Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X]
Arguments: [l_partkey#X ASC NULLS FIRST], false, 0
-(31) Scan parquet
+(30) Scan parquet
Output [2]: [p_partkey#X, p_type#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(p_partkey)]
ReadSchema: struct
-(32) Filter
+(31) Filter
Input [2]: [p_partkey#X, p_type#X]
Condition : isnotnull(p_partkey#X)
-(33) Exchange
+(32) Exchange
Input [2]: [p_partkey#X, p_type#X]
Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(34) Sort
+(33) Sort
Input [2]: [p_partkey#X, p_type#X]
Arguments: [p_partkey#X ASC NULLS FIRST], false, 0
-(35) SortMergeJoin
+(34) SortMergeJoin
Left keys [1]: [l_partkey#X]
Right keys [1]: [p_partkey#X]
Join type: Inner
Join condition: None
-(36) Project
+(35) Project
Output [3]: [l_extendedprice#X, l_discount#X, p_type#X]
Input [5]: [l_partkey#X, l_extendedprice#X, l_discount#X, p_partkey#X, p_type#X]
-(37) HashAggregate
+(36) HashAggregate
Input [3]: [l_extendedprice#X, l_discount#X, p_type#X]
Keys: []
Functions [2]: [partial_sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN (l_extendedprice#X * (1 - l_discount#X)) ELSE 0.0000 END), partial_sum((l_extendedprice#X * (1 - l_discount#X)))]
Aggregate Attributes [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X]
Results [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X]
-(38) HashAggregate
+(37) HashAggregate
Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X]
Keys: []
Functions [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN (l_extendedprice#X * (1 - l_discount#X)) ELSE 0.0000 END), sum((l_extendedprice#X * (1 - l_discount#X)))]
Aggregate Attributes [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN (l_extendedprice#X * (1 - l_discount#X)) ELSE 0.0000 END)#X, sum((l_extendedprice#X * (1 - l_discount#X)))#X]
Results [1]: [((100.00 * sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN (l_extendedprice#X * (1 - l_discount#X)) ELSE 0.0000 END)#X) / sum((l_extendedprice#X * (1 - l_discount#X)))#X) AS promo_revenue#X]
-(39) AdaptiveSparkPlan
+(38) AdaptiveSparkPlan
Output [1]: [promo_revenue#X]
Arguments: isFinalPlan=true
\ No newline at end of file
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 8f1a1b4415db..f370146f9206 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
@@ -14,7 +14,7 @@ AdaptiveSparkPlan (47)
: +- VeloxResizeBatches (5)
: +- ^ ProjectExecTransformer (3)
: +- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
+ : +- ^ ScanTransformer parquet (1)
+- ^ FilterExecTransformer (22)
+- ^ RegularHashAggregateExecTransformer (21)
+- ^ InputIteratorTransformer (20)
@@ -25,7 +25,7 @@ AdaptiveSparkPlan (47)
+- ^ FlushableHashAggregateExecTransformer (13)
+- ^ ProjectExecTransformer (12)
+- ^ FilterExecTransformer (11)
- +- ^ Scan parquet (10)
+ +- ^ ScanTransformer parquet (10)
+- == Initial Plan ==
Sort (46)
+- Exchange (45)
@@ -45,7 +45,7 @@ AdaptiveSparkPlan (47)
+- Scan parquet (35)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -82,7 +82,7 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X]
(9) InputIteratorTransformer
Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -259,34 +259,33 @@ Arguments: isFinalPlan=true
===== Subqueries =====
Subquery:1 Hosting operator id = 22 Hosting Expression = Subquery subquery#X, [id=#X]
-AdaptiveSparkPlan (73)
+AdaptiveSparkPlan (72)
+- == Final Plan ==
- VeloxColumnarToRow (64)
- +- ^ RegularHashAggregateExecTransformer (62)
- +- ^ RegularHashAggregateExecTransformer (61)
- +- ^ ProjectExecTransformer (60)
- +- ^ RegularHashAggregateExecTransformer (59)
- +- ^ InputIteratorTransformer (58)
- +- ShuffleQueryStage (56), Statistics(X)
- +- ColumnarExchange (55)
- +- VeloxResizeBatches (54)
- +- ^ ProjectExecTransformer (52)
- +- ^ FlushableHashAggregateExecTransformer (51)
- +- ^ ProjectExecTransformer (50)
- +- ^ FilterExecTransformer (49)
- +- ^ Scan parquet (48)
+ VeloxColumnarToRow (63)
+ +- ^ RegularHashAggregateExecTransformer (61)
+ +- ^ ProjectExecTransformer (60)
+ +- ^ RegularHashAggregateExecTransformer (59)
+ +- ^ InputIteratorTransformer (58)
+ +- ShuffleQueryStage (56), Statistics(X)
+ +- ColumnarExchange (55)
+ +- VeloxResizeBatches (54)
+ +- ^ ProjectExecTransformer (52)
+ +- ^ FlushableHashAggregateExecTransformer (51)
+ +- ^ ProjectExecTransformer (50)
+ +- ^ FilterExecTransformer (49)
+ +- ^ ScanTransformer parquet (48)
+- == Initial Plan ==
- HashAggregate (72)
- +- HashAggregate (71)
- +- HashAggregate (70)
- +- Exchange (69)
- +- HashAggregate (68)
- +- Project (67)
- +- Filter (66)
- +- Scan parquet (65)
+ HashAggregate (71)
+ +- HashAggregate (70)
+ +- HashAggregate (69)
+ +- Exchange (68)
+ +- HashAggregate (67)
+ +- Project (66)
+ +- Filter (65)
+ +- Scan parquet (64)
-(48) Scan parquet
+(48) ScanTransformer parquet
Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -348,71 +347,64 @@ Input [2]: [l_suppkey#X, sum((l_extendedprice#X * (1 - l_discount#X)))#X]
(61) RegularHashAggregateExecTransformer
Input [1]: [total_revenue#X]
Keys: []
-Functions [1]: [partial_max(total_revenue#X)]
-Aggregate Attributes [1]: [max#X]
-Results [1]: [max#X]
-
-(62) RegularHashAggregateExecTransformer
-Input [1]: [max#X]
-Keys: []
Functions [1]: [max(total_revenue#X)]
Aggregate Attributes [1]: [max(total_revenue#X)#X]
Results [1]: [max(total_revenue#X)#X AS max(total_revenue)#X]
-(63) WholeStageCodegenTransformer (X)
+(62) WholeStageCodegenTransformer (X)
Input [1]: [max(total_revenue)#X]
Arguments: false
-(64) VeloxColumnarToRow
+(63) VeloxColumnarToRow
Input [1]: [max(total_revenue)#X]
-(65) Scan parquet
+(64) Scan parquet
Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1996-01-01), LessThan(l_shipdate,1996-04-01)]
ReadSchema: struct
-(66) Filter
+(65) Filter
Input [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Condition : ((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1996-01-01)) AND (l_shipdate#X < 1996-04-01))
-(67) Project
+(66) Project
Output [3]: [l_suppkey#X, l_extendedprice#X, l_discount#X]
Input [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
-(68) HashAggregate
+(67) HashAggregate
Input [3]: [l_suppkey#X, l_extendedprice#X, l_discount#X]
Keys [1]: [l_suppkey#X]
Functions [1]: [partial_sum((l_extendedprice#X * (1 - l_discount#X)))]
Aggregate Attributes [2]: [sum#X, isEmpty#X]
Results [3]: [l_suppkey#X, sum#X, isEmpty#X]
-(69) Exchange
+(68) Exchange
Input [3]: [l_suppkey#X, sum#X, isEmpty#X]
Arguments: hashpartitioning(l_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(70) HashAggregate
+(69) HashAggregate
Input [3]: [l_suppkey#X, sum#X, isEmpty#X]
Keys [1]: [l_suppkey#X]
Functions [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))]
Aggregate Attributes [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))#X]
Results [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))#X AS total_revenue#X]
-(71) HashAggregate
+(70) HashAggregate
Input [1]: [total_revenue#X]
Keys: []
Functions [1]: [partial_max(total_revenue#X)]
Aggregate Attributes [1]: [max#X]
Results [1]: [max#X]
-(72) HashAggregate
+(71) HashAggregate
Input [1]: [max#X]
Keys: []
Functions [1]: [max(total_revenue#X)]
Aggregate Attributes [1]: [max(total_revenue#X)#X]
Results [1]: [max(total_revenue#X)#X AS max(total_revenue)#X]
-(73) AdaptiveSparkPlan
+(72) AdaptiveSparkPlan
Output [1]: [max(total_revenue)#X]
Arguments: isFinalPlan=true
\ No newline at end of file
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 ee7826c8aa7e..3441216f140f 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
@@ -29,14 +29,14 @@ AdaptiveSparkPlan (71)
: +- VeloxResizeBatches (5)
: +- ^ ProjectExecTransformer (3)
: +- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
+ : +- ^ ScanTransformer parquet (1)
+- ^ InputIteratorTransformer (18)
+- ShuffleQueryStage (16), Statistics(X)
+- ColumnarExchange (15)
+- VeloxResizeBatches (14)
+- ^ ProjectExecTransformer (12)
+- ^ FilterExecTransformer (11)
- +- ^ Scan parquet (10)
+ +- ^ ScanTransformer parquet (10)
+- == Initial Plan ==
Sort (70)
+- Exchange (69)
@@ -63,7 +63,7 @@ AdaptiveSparkPlan (71)
+- Scan parquet (57)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [ps_partkey#X, ps_suppkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -100,7 +100,7 @@ Input [2]: [ps_partkey#X, ps_suppkey#X]
(9) InputIteratorTransformer
Input [2]: [ps_partkey#X, ps_suppkey#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [4]: [p_partkey#X, p_brand#X, p_type#X, p_size#X]
Batched: true
Location: InMemoryFileIndex [*]
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 eaad0429ab40..c1b43b7fc421 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
@@ -1,65 +1,64 @@
== Physical Plan ==
-AdaptiveSparkPlan (63)
+AdaptiveSparkPlan (62)
+- == Final Plan ==
- VeloxColumnarToRow (40)
- +- ^ ProjectExecTransformer (38)
- +- ^ RegularHashAggregateExecTransformer (37)
- +- ^ RegularHashAggregateExecTransformer (36)
- +- ^ ProjectExecTransformer (35)
- +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (34)
- :- ^ ProjectExecTransformer (20)
- : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19)
- : :- ^ InputIteratorTransformer (9)
- : : +- ShuffleQueryStage (7), Statistics(X)
- : : +- ColumnarExchange (6)
- : : +- VeloxResizeBatches (5)
- : : +- ^ ProjectExecTransformer (3)
- : : +- ^ FilterExecTransformer (2)
- : : +- ^ Scan parquet (1)
- : +- ^ InputIteratorTransformer (18)
- : +- ShuffleQueryStage (16), Statistics(X)
- : +- ColumnarExchange (15)
- : +- VeloxResizeBatches (14)
- : +- ^ ProjectExecTransformer (12)
- : +- ^ FilterExecTransformer (11)
- : +- ^ Scan parquet (10)
- +- ^ FilterExecTransformer (33)
- +- ^ ProjectExecTransformer (32)
- +- ^ RegularHashAggregateExecTransformer (31)
- +- ^ InputIteratorTransformer (30)
- +- ShuffleQueryStage (28), Statistics(X)
- +- ColumnarExchange (27)
- +- VeloxResizeBatches (26)
- +- ^ ProjectExecTransformer (24)
- +- ^ FlushableHashAggregateExecTransformer (23)
- +- ^ FilterExecTransformer (22)
- +- ^ Scan parquet (21)
+ VeloxColumnarToRow (39)
+ +- ^ ProjectExecTransformer (37)
+ +- ^ RegularHashAggregateExecTransformer (36)
+ +- ^ ProjectExecTransformer (35)
+ +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (34)
+ :- ^ ProjectExecTransformer (20)
+ : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19)
+ : :- ^ InputIteratorTransformer (9)
+ : : +- ShuffleQueryStage (7), Statistics(X)
+ : : +- ColumnarExchange (6)
+ : : +- VeloxResizeBatches (5)
+ : : +- ^ ProjectExecTransformer (3)
+ : : +- ^ FilterExecTransformer (2)
+ : : +- ^ ScanTransformer parquet (1)
+ : +- ^ InputIteratorTransformer (18)
+ : +- ShuffleQueryStage (16), Statistics(X)
+ : +- ColumnarExchange (15)
+ : +- VeloxResizeBatches (14)
+ : +- ^ ProjectExecTransformer (12)
+ : +- ^ FilterExecTransformer (11)
+ : +- ^ ScanTransformer parquet (10)
+ +- ^ FilterExecTransformer (33)
+ +- ^ ProjectExecTransformer (32)
+ +- ^ RegularHashAggregateExecTransformer (31)
+ +- ^ InputIteratorTransformer (30)
+ +- ShuffleQueryStage (28), Statistics(X)
+ +- ColumnarExchange (27)
+ +- VeloxResizeBatches (26)
+ +- ^ ProjectExecTransformer (24)
+ +- ^ FlushableHashAggregateExecTransformer (23)
+ +- ^ FilterExecTransformer (22)
+ +- ^ ScanTransformer parquet (21)
+- == Initial Plan ==
- HashAggregate (62)
- +- HashAggregate (61)
- +- Project (60)
- +- SortMergeJoin Inner (59)
- :- Project (51)
- : +- SortMergeJoin Inner (50)
- : :- Sort (44)
- : : +- Exchange (43)
- : : +- Filter (42)
- : : +- Scan parquet (41)
- : +- Sort (49)
- : +- Exchange (48)
- : +- Project (47)
- : +- Filter (46)
- : +- Scan parquet (45)
- +- Sort (58)
- +- Filter (57)
- +- HashAggregate (56)
- +- Exchange (55)
- +- HashAggregate (54)
- +- Filter (53)
- +- Scan parquet (52)
-
-
-(1) Scan parquet
+ HashAggregate (61)
+ +- HashAggregate (60)
+ +- Project (59)
+ +- SortMergeJoin Inner (58)
+ :- Project (50)
+ : +- SortMergeJoin Inner (49)
+ : :- Sort (43)
+ : : +- Exchange (42)
+ : : +- Filter (41)
+ : : +- Scan parquet (40)
+ : +- Sort (48)
+ : +- Exchange (47)
+ : +- Project (46)
+ : +- Filter (45)
+ : +- Scan parquet (44)
+ +- Sort (57)
+ +- Filter (56)
+ +- HashAggregate (55)
+ +- Exchange (54)
+ +- HashAggregate (53)
+ +- Filter (52)
+ +- Scan parquet (51)
+
+
+(1) ScanTransformer parquet
Output [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -96,7 +95,7 @@ Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X]
(9) InputIteratorTransformer
Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [3]: [p_partkey#X, p_brand#X, p_container#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -143,7 +142,7 @@ Join condition: None
Output [3]: [l_quantity#X, l_extendedprice#X, p_partkey#X]
Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, p_partkey#X]
-(21) Scan parquet
+(21) ScanTransformer parquet
Output [2]: [l_partkey#X, l_quantity#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -215,141 +214,134 @@ Input [5]: [l_quantity#X, l_extendedprice#X, p_partkey#X, (0.2 * avg(l_quantity)
(36) RegularHashAggregateExecTransformer
Input [1]: [l_extendedprice#X]
Keys: []
-Functions [1]: [partial_sum(l_extendedprice#X)]
-Aggregate Attributes [2]: [sum#X, isEmpty#X]
-Results [2]: [sum#X, isEmpty#X]
-
-(37) RegularHashAggregateExecTransformer
-Input [2]: [sum#X, isEmpty#X]
-Keys: []
Functions [1]: [sum(l_extendedprice#X)]
Aggregate Attributes [1]: [sum(l_extendedprice#X)#X]
Results [1]: [sum(l_extendedprice#X)#X]
-(38) ProjectExecTransformer
+(37) ProjectExecTransformer
Output [1]: [(sum(l_extendedprice#X)#X / 7.0) AS avg_yearly#X]
Input [1]: [sum(l_extendedprice#X)#X]
-(39) WholeStageCodegenTransformer (X)
+(38) WholeStageCodegenTransformer (X)
Input [1]: [avg_yearly#X]
Arguments: false
-(40) VeloxColumnarToRow
+(39) VeloxColumnarToRow
Input [1]: [avg_yearly#X]
-(41) Scan parquet
+(40) Scan parquet
Output [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(l_partkey), IsNotNull(l_quantity)]
ReadSchema: struct
-(42) Filter
+(41) Filter
Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X]
Condition : (isnotnull(l_partkey#X) AND isnotnull(l_quantity#X))
-(43) Exchange
+(42) Exchange
Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X]
Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(44) Sort
+(43) Sort
Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X]
Arguments: [l_partkey#X ASC NULLS FIRST], false, 0
-(45) Scan parquet
+(44) Scan parquet
Output [3]: [p_partkey#X, p_brand#X, p_container#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(p_brand), IsNotNull(p_container), EqualTo(p_brand,Brand#X), EqualTo(p_container,MED BOX), IsNotNull(p_partkey)]
ReadSchema: struct
-(46) Filter
+(45) Filter
Input [3]: [p_partkey#X, p_brand#X, p_container#X]
Condition : ((((isnotnull(p_brand#X) AND isnotnull(p_container#X)) AND (p_brand#X = Brand#X)) AND (p_container#X = MED BOX)) AND isnotnull(p_partkey#X))
-(47) Project
+(46) Project
Output [1]: [p_partkey#X]
Input [3]: [p_partkey#X, p_brand#X, p_container#X]
-(48) Exchange
+(47) Exchange
Input [1]: [p_partkey#X]
Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(49) Sort
+(48) Sort
Input [1]: [p_partkey#X]
Arguments: [p_partkey#X ASC NULLS FIRST], false, 0
-(50) SortMergeJoin
+(49) SortMergeJoin
Left keys [1]: [l_partkey#X]
Right keys [1]: [p_partkey#X]
Join type: Inner
Join condition: None
-(51) Project
+(50) Project
Output [3]: [l_quantity#X, l_extendedprice#X, p_partkey#X]
Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, p_partkey#X]
-(52) Scan parquet
+(51) Scan parquet
Output [2]: [l_partkey#X, l_quantity#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(l_partkey)]
ReadSchema: struct
-(53) Filter
+(52) Filter
Input [2]: [l_partkey#X, l_quantity#X]
Condition : isnotnull(l_partkey#X)
-(54) HashAggregate
+(53) HashAggregate
Input [2]: [l_partkey#X, l_quantity#X]
Keys [1]: [l_partkey#X]
Functions [1]: [partial_avg(l_quantity#X)]
Aggregate Attributes [2]: [sum#X, count#X]
Results [3]: [l_partkey#X, sum#X, count#X]
-(55) Exchange
+(54) Exchange
Input [3]: [l_partkey#X, sum#X, count#X]
Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(56) HashAggregate
+(55) HashAggregate
Input [3]: [l_partkey#X, sum#X, count#X]
Keys [1]: [l_partkey#X]
Functions [1]: [avg(l_quantity#X)]
Aggregate Attributes [1]: [avg(l_quantity#X)#X]
Results [2]: [(0.2 * avg(l_quantity#X)#X) AS (0.2 * avg(l_quantity))#X, l_partkey#X]
-(57) Filter
+(56) Filter
Input [2]: [(0.2 * avg(l_quantity))#X, l_partkey#X]
Condition : isnotnull((0.2 * avg(l_quantity))#X)
-(58) Sort
+(57) Sort
Input [2]: [(0.2 * avg(l_quantity))#X, l_partkey#X]
Arguments: [l_partkey#X ASC NULLS FIRST], false, 0
-(59) SortMergeJoin
+(58) SortMergeJoin
Left keys [1]: [p_partkey#X]
Right keys [1]: [l_partkey#X]
Join type: Inner
Join condition: (cast(l_quantity#X as decimal(18,7)) < (0.2 * avg(l_quantity))#X)
-(60) Project
+(59) Project
Output [1]: [l_extendedprice#X]
Input [5]: [l_quantity#X, l_extendedprice#X, p_partkey#X, (0.2 * avg(l_quantity))#X, l_partkey#X]
-(61) HashAggregate
+(60) HashAggregate
Input [1]: [l_extendedprice#X]
Keys: []
Functions [1]: [partial_sum(l_extendedprice#X)]
Aggregate Attributes [2]: [sum#X, isEmpty#X]
Results [2]: [sum#X, isEmpty#X]
-(62) HashAggregate
+(61) HashAggregate
Input [2]: [sum#X, isEmpty#X]
Keys: []
Functions [1]: [sum(l_extendedprice#X)]
Aggregate Attributes [1]: [sum(l_extendedprice#X)#X]
Results [1]: [(sum(l_extendedprice#X)#X / 7.0) AS avg_yearly#X]
-(63) AdaptiveSparkPlan
+(62) AdaptiveSparkPlan
Output [1]: [avg_yearly#X]
Arguments: isFinalPlan=true
\ No newline at end of file
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 85b11fa8ca18..c1a6b181bb34 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
@@ -1,105 +1,104 @@
== Physical Plan ==
-AdaptiveSparkPlan (110)
+AdaptiveSparkPlan (109)
+- == Final Plan ==
- VeloxColumnarToRow (70)
- +- TakeOrderedAndProjectExecTransformer (69)
- +- ^ RegularHashAggregateExecTransformer (67)
- +- ^ RegularHashAggregateExecTransformer (66)
- +- ^ ProjectExecTransformer (65)
- +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (64)
- :- ^ InputIteratorTransformer (46)
- : +- ShuffleQueryStage (44), Statistics(X)
- : +- ColumnarExchange (43)
- : +- VeloxResizeBatches (42)
- : +- ^ ProjectExecTransformer (40)
- : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (39)
- : :- ^ InputIteratorTransformer (9)
- : : +- ShuffleQueryStage (7), Statistics(X)
- : : +- ColumnarExchange (6)
- : : +- VeloxResizeBatches (5)
- : : +- ^ ProjectExecTransformer (3)
- : : +- ^ FilterExecTransformer (2)
- : : +- ^ Scan parquet (1)
- : +- ^ InputIteratorTransformer (38)
- : +- ShuffleQueryStage (36), Statistics(X)
- : +- ColumnarExchange (35)
- : +- VeloxResizeBatches (34)
- : +- ^ ProjectExecTransformer (32)
- : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (31)
- : :- ^ InputIteratorTransformer (18)
- : : +- ShuffleQueryStage (16), Statistics(X)
- : : +- ColumnarExchange (15)
- : : +- VeloxResizeBatches (14)
- : : +- ^ ProjectExecTransformer (12)
- : : +- ^ FilterExecTransformer (11)
- : : +- ^ Scan parquet (10)
- : +- ^ ProjectExecTransformer (30)
- : +- ^ FilterExecTransformer (29)
- : +- ^ RegularHashAggregateExecTransformer (28)
- : +- ^ InputIteratorTransformer (27)
- : +- ShuffleQueryStage (25), Statistics(X)
- : +- ColumnarExchange (24)
- : +- VeloxResizeBatches (23)
- : +- ^ ProjectExecTransformer (21)
- : +- ^ FlushableHashAggregateExecTransformer (20)
- : +- ^ Scan parquet (19)
- +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (63)
- :- ^ InputIteratorTransformer (55)
- : +- ShuffleQueryStage (53), Statistics(X)
- : +- ColumnarExchange (52)
- : +- VeloxResizeBatches (51)
- : +- ^ ProjectExecTransformer (49)
- : +- ^ FilterExecTransformer (48)
- : +- ^ Scan parquet (47)
- +- ^ ProjectExecTransformer (62)
- +- ^ FilterExecTransformer (61)
- +- ^ RegularHashAggregateExecTransformer (60)
- +- ^ InputIteratorTransformer (59)
- +- ShuffleQueryStage (57), Statistics(X)
- +- ReusedExchange (56)
+ VeloxColumnarToRow (69)
+ +- TakeOrderedAndProjectExecTransformer (68)
+ +- ^ RegularHashAggregateExecTransformer (66)
+ +- ^ ProjectExecTransformer (65)
+ +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (64)
+ :- ^ InputIteratorTransformer (46)
+ : +- ShuffleQueryStage (44), Statistics(X)
+ : +- ColumnarExchange (43)
+ : +- VeloxResizeBatches (42)
+ : +- ^ ProjectExecTransformer (40)
+ : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (39)
+ : :- ^ InputIteratorTransformer (9)
+ : : +- ShuffleQueryStage (7), Statistics(X)
+ : : +- ColumnarExchange (6)
+ : : +- VeloxResizeBatches (5)
+ : : +- ^ ProjectExecTransformer (3)
+ : : +- ^ FilterExecTransformer (2)
+ : : +- ^ ScanTransformer parquet (1)
+ : +- ^ InputIteratorTransformer (38)
+ : +- ShuffleQueryStage (36), Statistics(X)
+ : +- ColumnarExchange (35)
+ : +- VeloxResizeBatches (34)
+ : +- ^ ProjectExecTransformer (32)
+ : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (31)
+ : :- ^ InputIteratorTransformer (18)
+ : : +- ShuffleQueryStage (16), Statistics(X)
+ : : +- ColumnarExchange (15)
+ : : +- VeloxResizeBatches (14)
+ : : +- ^ ProjectExecTransformer (12)
+ : : +- ^ FilterExecTransformer (11)
+ : : +- ^ ScanTransformer parquet (10)
+ : +- ^ ProjectExecTransformer (30)
+ : +- ^ FilterExecTransformer (29)
+ : +- ^ RegularHashAggregateExecTransformer (28)
+ : +- ^ InputIteratorTransformer (27)
+ : +- ShuffleQueryStage (25), Statistics(X)
+ : +- ColumnarExchange (24)
+ : +- VeloxResizeBatches (23)
+ : +- ^ ProjectExecTransformer (21)
+ : +- ^ FlushableHashAggregateExecTransformer (20)
+ : +- ^ ScanTransformer parquet (19)
+ +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (63)
+ :- ^ InputIteratorTransformer (55)
+ : +- ShuffleQueryStage (53), Statistics(X)
+ : +- ColumnarExchange (52)
+ : +- VeloxResizeBatches (51)
+ : +- ^ ProjectExecTransformer (49)
+ : +- ^ FilterExecTransformer (48)
+ : +- ^ ScanTransformer parquet (47)
+ +- ^ ProjectExecTransformer (62)
+ +- ^ FilterExecTransformer (61)
+ +- ^ RegularHashAggregateExecTransformer (60)
+ +- ^ InputIteratorTransformer (59)
+ +- ShuffleQueryStage (57), Statistics(X)
+ +- ReusedExchange (56)
+- == Initial Plan ==
- TakeOrderedAndProject (109)
- +- HashAggregate (108)
- +- HashAggregate (107)
- +- Project (106)
- +- SortMergeJoin Inner (105)
- :- Sort (92)
- : +- Exchange (91)
- : +- Project (90)
- : +- SortMergeJoin Inner (89)
- : :- Sort (74)
- : : +- Exchange (73)
- : : +- Filter (72)
- : : +- Scan parquet (71)
- : +- Sort (88)
- : +- Exchange (87)
- : +- SortMergeJoin LeftSemi (86)
- : :- Sort (78)
- : : +- Exchange (77)
- : : +- Filter (76)
- : : +- Scan parquet (75)
- : +- Sort (85)
- : +- Project (84)
- : +- Filter (83)
- : +- HashAggregate (82)
- : +- Exchange (81)
- : +- HashAggregate (80)
- : +- Scan parquet (79)
- +- SortMergeJoin LeftSemi (104)
- :- Sort (96)
- : +- Exchange (95)
- : +- Filter (94)
- : +- Scan parquet (93)
- +- Sort (103)
- +- Project (102)
- +- Filter (101)
- +- HashAggregate (100)
- +- Exchange (99)
- +- HashAggregate (98)
- +- Scan parquet (97)
-
-
-(1) Scan parquet
+ TakeOrderedAndProject (108)
+ +- HashAggregate (107)
+ +- HashAggregate (106)
+ +- Project (105)
+ +- SortMergeJoin Inner (104)
+ :- Sort (91)
+ : +- Exchange (90)
+ : +- Project (89)
+ : +- SortMergeJoin Inner (88)
+ : :- Sort (73)
+ : : +- Exchange (72)
+ : : +- Filter (71)
+ : : +- Scan parquet (70)
+ : +- Sort (87)
+ : +- Exchange (86)
+ : +- SortMergeJoin LeftSemi (85)
+ : :- Sort (77)
+ : : +- Exchange (76)
+ : : +- Filter (75)
+ : : +- Scan parquet (74)
+ : +- Sort (84)
+ : +- Project (83)
+ : +- Filter (82)
+ : +- HashAggregate (81)
+ : +- Exchange (80)
+ : +- HashAggregate (79)
+ : +- Scan parquet (78)
+ +- SortMergeJoin LeftSemi (103)
+ :- Sort (95)
+ : +- Exchange (94)
+ : +- Filter (93)
+ : +- Scan parquet (92)
+ +- Sort (102)
+ +- Project (101)
+ +- Filter (100)
+ +- HashAggregate (99)
+ +- Exchange (98)
+ +- HashAggregate (97)
+ +- Scan parquet (96)
+
+
+(1) ScanTransformer parquet
Output [2]: [c_custkey#X, c_name#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -136,7 +135,7 @@ Input [2]: [c_custkey#X, c_name#X]
(9) InputIteratorTransformer
Input [2]: [c_custkey#X, c_name#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -173,7 +172,7 @@ Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X]
(18) InputIteratorTransformer
Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X]
-(19) Scan parquet
+(19) ScanTransformer parquet
Output [2]: [l_orderkey#X, l_quantity#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -291,7 +290,7 @@ Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X]
(46) InputIteratorTransformer
Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X]
-(47) Scan parquet
+(47) ScanTransformer parquet
Output [2]: [l_orderkey#X, l_quantity#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -375,223 +374,216 @@ Input [7]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X,
(66) RegularHashAggregateExecTransformer
Input [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_quantity#X]
Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X]
-Functions [1]: [partial_sum(l_quantity#X)]
-Aggregate Attributes [2]: [sum#X, isEmpty#X]
-Results [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X]
-
-(67) RegularHashAggregateExecTransformer
-Input [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X]
-Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X]
Functions [1]: [sum(l_quantity#X)]
Aggregate Attributes [1]: [sum(l_quantity#X)#X]
Results [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity#X)#X AS sum(l_quantity)#X]
-(68) WholeStageCodegenTransformer (X)
+(67) WholeStageCodegenTransformer (X)
Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X]
Arguments: false
-(69) TakeOrderedAndProjectExecTransformer
+(68) TakeOrderedAndProjectExecTransformer
Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X]
Arguments: X, [o_totalprice#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X], 0
-(70) VeloxColumnarToRow
+(69) VeloxColumnarToRow
Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X]
-(71) Scan parquet
+(70) Scan parquet
Output [2]: [c_custkey#X, c_name#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(c_custkey)]
ReadSchema: struct
-(72) Filter
+(71) Filter
Input [2]: [c_custkey#X, c_name#X]
Condition : isnotnull(c_custkey#X)
-(73) Exchange
+(72) Exchange
Input [2]: [c_custkey#X, c_name#X]
Arguments: hashpartitioning(c_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(74) Sort
+(73) Sort
Input [2]: [c_custkey#X, c_name#X]
Arguments: [c_custkey#X ASC NULLS FIRST], false, 0
-(75) Scan parquet
+(74) Scan parquet
Output [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(o_custkey), IsNotNull(o_orderkey)]
ReadSchema: struct
-(76) Filter
+(75) Filter
Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X]
Condition : (isnotnull(o_custkey#X) AND isnotnull(o_orderkey#X))
-(77) Exchange
+(76) Exchange
Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X]
Arguments: hashpartitioning(o_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(78) Sort
+(77) Sort
Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X]
Arguments: [o_orderkey#X ASC NULLS FIRST], false, 0
-(79) Scan parquet
+(78) Scan parquet
Output [2]: [l_orderkey#X, l_quantity#X]
Batched: true
Location: InMemoryFileIndex [*]
ReadSchema: struct
-(80) HashAggregate
+(79) HashAggregate
Input [2]: [l_orderkey#X, l_quantity#X]
Keys [1]: [l_orderkey#X]
Functions [1]: [partial_sum(l_quantity#X)]
Aggregate Attributes [2]: [sum#X, isEmpty#X]
Results [3]: [l_orderkey#X, sum#X, isEmpty#X]
-(81) Exchange
+(80) Exchange
Input [3]: [l_orderkey#X, sum#X, isEmpty#X]
Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(82) HashAggregate
+(81) HashAggregate
Input [3]: [l_orderkey#X, sum#X, isEmpty#X]
Keys [1]: [l_orderkey#X]
Functions [1]: [sum(l_quantity#X)]
Aggregate Attributes [1]: [sum(l_quantity#X)#X]
Results [2]: [l_orderkey#X, sum(l_quantity#X)#X AS sum(l_quantity#X)#X]
-(83) Filter
+(82) Filter
Input [2]: [l_orderkey#X, sum(l_quantity#X)#X]
Condition : (isnotnull(sum(l_quantity#X)#X) AND (sum(l_quantity#X)#X > 300.00))
-(84) Project
+(83) Project
Output [1]: [l_orderkey#X]
Input [2]: [l_orderkey#X, sum(l_quantity#X)#X]
-(85) Sort
+(84) Sort
Input [1]: [l_orderkey#X]
Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0
-(86) SortMergeJoin
+(85) SortMergeJoin
Left keys [1]: [o_orderkey#X]
Right keys [1]: [l_orderkey#X]
Join type: LeftSemi
Join condition: None
-(87) Exchange
+(86) Exchange
Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X]
Arguments: hashpartitioning(o_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(88) Sort
+(87) Sort
Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X]
Arguments: [o_custkey#X ASC NULLS FIRST], false, 0
-(89) SortMergeJoin
+(88) SortMergeJoin
Left keys [1]: [c_custkey#X]
Right keys [1]: [o_custkey#X]
Join type: Inner
Join condition: None
-(90) Project
+(89) Project
Output [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X]
Input [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X]
-(91) Exchange
+(90) Exchange
Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X]
Arguments: hashpartitioning(o_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(92) Sort
+(91) Sort
Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X]
Arguments: [o_orderkey#X ASC NULLS FIRST], false, 0
-(93) Scan parquet
+(92) Scan parquet
Output [2]: [l_orderkey#X, l_quantity#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(l_orderkey)]
ReadSchema: struct
-(94) Filter
+(93) Filter
Input [2]: [l_orderkey#X, l_quantity#X]
Condition : isnotnull(l_orderkey#X)
-(95) Exchange
+(94) Exchange
Input [2]: [l_orderkey#X, l_quantity#X]
Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(96) Sort
+(95) Sort
Input [2]: [l_orderkey#X, l_quantity#X]
Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0
-(97) Scan parquet
+(96) Scan parquet
Output [2]: [l_orderkey#X, l_quantity#X]
Batched: true
Location: InMemoryFileIndex [*]
ReadSchema: struct
-(98) HashAggregate
+(97) HashAggregate
Input [2]: [l_orderkey#X, l_quantity#X]
Keys [1]: [l_orderkey#X]
Functions [1]: [partial_sum(l_quantity#X)]
Aggregate Attributes [2]: [sum#X, isEmpty#X]
Results [3]: [l_orderkey#X, sum#X, isEmpty#X]
-(99) Exchange
+(98) Exchange
Input [3]: [l_orderkey#X, sum#X, isEmpty#X]
Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(100) HashAggregate
+(99) HashAggregate
Input [3]: [l_orderkey#X, sum#X, isEmpty#X]
Keys [1]: [l_orderkey#X]
Functions [1]: [sum(l_quantity#X)]
Aggregate Attributes [1]: [sum(l_quantity#X)#X]
Results [2]: [l_orderkey#X, sum(l_quantity#X)#X AS sum(l_quantity#X)#X]
-(101) Filter
+(100) Filter
Input [2]: [l_orderkey#X, sum(l_quantity#X)#X]
Condition : (isnotnull(sum(l_quantity#X)#X) AND (sum(l_quantity#X)#X > 300.00))
-(102) Project
+(101) Project
Output [1]: [l_orderkey#X]
Input [2]: [l_orderkey#X, sum(l_quantity#X)#X]
-(103) Sort
+(102) Sort
Input [1]: [l_orderkey#X]
Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0
-(104) SortMergeJoin
+(103) SortMergeJoin
Left keys [1]: [l_orderkey#X]
Right keys [1]: [l_orderkey#X]
Join type: LeftSemi
Join condition: None
-(105) SortMergeJoin
+(104) SortMergeJoin
Left keys [1]: [o_orderkey#X]
Right keys [1]: [l_orderkey#X]
Join type: Inner
Join condition: None
-(106) Project
+(105) Project
Output [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_quantity#X]
Input [7]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_orderkey#X, l_quantity#X]
-(107) HashAggregate
+(106) HashAggregate
Input [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_quantity#X]
Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X]
Functions [1]: [partial_sum(l_quantity#X)]
Aggregate Attributes [2]: [sum#X, isEmpty#X]
Results [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X]
-(108) HashAggregate
+(107) HashAggregate
Input [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X]
Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X]
Functions [1]: [sum(l_quantity#X)]
Aggregate Attributes [1]: [sum(l_quantity#X)#X]
Results [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity#X)#X AS sum(l_quantity)#X]
-(109) TakeOrderedAndProject
+(108) TakeOrderedAndProject
Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X]
Arguments: X, [o_totalprice#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X]
-(110) AdaptiveSparkPlan
+(109) AdaptiveSparkPlan
Output [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X]
Arguments: isFinalPlan=true
\ No newline at end of file
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 c72ca859ce37..4db9ca0c8393 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
@@ -1,42 +1,41 @@
== Physical Plan ==
-AdaptiveSparkPlan (38)
+AdaptiveSparkPlan (37)
+- == Final Plan ==
- VeloxColumnarToRow (24)
- +- ^ RegularHashAggregateExecTransformer (22)
- +- ^ RegularHashAggregateExecTransformer (21)
- +- ^ ProjectExecTransformer (20)
- +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19)
- :- ^ InputIteratorTransformer (9)
- : +- ShuffleQueryStage (7), Statistics(X)
- : +- ColumnarExchange (6)
- : +- VeloxResizeBatches (5)
- : +- ^ ProjectExecTransformer (3)
- : +- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
- +- ^ InputIteratorTransformer (18)
- +- ShuffleQueryStage (16), Statistics(X)
- +- ColumnarExchange (15)
- +- VeloxResizeBatches (14)
- +- ^ ProjectExecTransformer (12)
- +- ^ FilterExecTransformer (11)
- +- ^ Scan parquet (10)
+ VeloxColumnarToRow (23)
+ +- ^ RegularHashAggregateExecTransformer (21)
+ +- ^ ProjectExecTransformer (20)
+ +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19)
+ :- ^ InputIteratorTransformer (9)
+ : +- ShuffleQueryStage (7), Statistics(X)
+ : +- ColumnarExchange (6)
+ : +- VeloxResizeBatches (5)
+ : +- ^ ProjectExecTransformer (3)
+ : +- ^ FilterExecTransformer (2)
+ : +- ^ ScanTransformer parquet (1)
+ +- ^ InputIteratorTransformer (18)
+ +- ShuffleQueryStage (16), Statistics(X)
+ +- ColumnarExchange (15)
+ +- VeloxResizeBatches (14)
+ +- ^ ProjectExecTransformer (12)
+ +- ^ FilterExecTransformer (11)
+ +- ^ ScanTransformer parquet (10)
+- == Initial Plan ==
- HashAggregate (37)
- +- HashAggregate (36)
- +- Project (35)
- +- SortMergeJoin Inner (34)
- :- Sort (29)
- : +- Exchange (28)
- : +- Project (27)
- : +- Filter (26)
- : +- Scan parquet (25)
- +- Sort (33)
- +- Exchange (32)
- +- Filter (31)
- +- Scan parquet (30)
-
-
-(1) Scan parquet
+ HashAggregate (36)
+ +- HashAggregate (35)
+ +- Project (34)
+ +- SortMergeJoin Inner (33)
+ :- Sort (28)
+ : +- Exchange (27)
+ : +- Project (26)
+ : +- Filter (25)
+ : +- Scan parquet (24)
+ +- Sort (32)
+ +- Exchange (31)
+ +- Filter (30)
+ +- Scan parquet (29)
+
+
+(1) ScanTransformer parquet
Output [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -73,7 +72,7 @@ Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X]
(9) InputIteratorTransformer
Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -123,90 +122,83 @@ Input [8]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, p_partke
(21) RegularHashAggregateExecTransformer
Input [3]: [l_extendedprice#X, l_discount#X, _pre_X#X]
Keys: []
-Functions [1]: [partial_sum(_pre_X#X)]
-Aggregate Attributes [2]: [sum#X, isEmpty#X]
-Results [2]: [sum#X, isEmpty#X]
-
-(22) RegularHashAggregateExecTransformer
-Input [2]: [sum#X, isEmpty#X]
-Keys: []
-Functions [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))]
+Functions [1]: [sum(_pre_X#X)]
Aggregate Attributes [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))#X]
Results [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))#X AS revenue#X]
-(23) WholeStageCodegenTransformer (X)
+(22) WholeStageCodegenTransformer (X)
Input [1]: [revenue#X]
Arguments: false
-(24) VeloxColumnarToRow
+(23) VeloxColumnarToRow
Input [1]: [revenue#X]
-(25) Scan parquet
+(24) Scan parquet
Output [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(l_shipinstruct), In(l_shipmode, [AIR,AIR REG]), EqualTo(l_shipinstruct,DELIVER IN PERSON), IsNotNull(l_partkey), Or(Or(And(GreaterThanOrEqual(l_quantity,1.00),LessThanOrEqual(l_quantity,11.00)),And(GreaterThanOrEqual(l_quantity,10.00),LessThanOrEqual(l_quantity,20.00))),And(GreaterThanOrEqual(l_quantity,20.00),LessThanOrEqual(l_quantity,30.00)))]
ReadSchema: struct
-(26) Filter
+(25) Filter
Input [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X]
Condition : ((((isnotnull(l_shipinstruct#X) AND l_shipmode#X IN (AIR,AIR REG)) AND (l_shipinstruct#X = DELIVER IN PERSON)) AND isnotnull(l_partkey#X)) AND ((((l_quantity#X >= 1.00) AND (l_quantity#X <= 11.00)) OR ((l_quantity#X >= 10.00) AND (l_quantity#X <= 20.00))) OR ((l_quantity#X >= 20.00) AND (l_quantity#X <= 30.00))))
-(27) Project
+(26) Project
Output [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X]
Input [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X]
-(28) Exchange
+(27) Exchange
Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X]
Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(29) Sort
+(28) Sort
Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X]
Arguments: [l_partkey#X ASC NULLS FIRST], false, 0
-(30) Scan parquet
+(29) Scan parquet
Output [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(p_size), GreaterThanOrEqual(p_size,1), IsNotNull(p_partkey), Or(Or(And(And(EqualTo(p_brand,Brand#X),In(p_container, [SM BOX,SM CASE,SM PACK,SM PKG])),LessThanOrEqual(p_size,5)),And(And(EqualTo(p_brand,Brand#X),In(p_container, [MED BAG,MED BOX,MED PACK,MED PKG])),LessThanOrEqual(p_size,10))),And(And(EqualTo(p_brand,Brand#X),In(p_container, [LG BOX,LG CASE,LG PACK,LG PKG])),LessThanOrEqual(p_size,15)))]
ReadSchema: struct
-(31) Filter
+(30) Filter
Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X]
Condition : (((isnotnull(p_size#X) AND (p_size#X >= 1)) AND isnotnull(p_partkey#X)) AND (((((p_brand#X = Brand#X) AND p_container#X IN (SM CASE,SM BOX,SM PACK,SM PKG)) AND (p_size#X <= 5)) OR (((p_brand#X = Brand#X) AND p_container#X IN (MED BAG,MED BOX,MED PKG,MED PACK)) AND (p_size#X <= 10))) OR (((p_brand#X = Brand#X) AND p_container#X IN (LG CASE,LG BOX,LG PACK,LG PKG)) AND (p_size#X <= 15))))
-(32) Exchange
+(31) Exchange
Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X]
Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(33) Sort
+(32) Sort
Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X]
Arguments: [p_partkey#X ASC NULLS FIRST], false, 0
-(34) SortMergeJoin
+(33) SortMergeJoin
Left keys [1]: [l_partkey#X]
Right keys [1]: [p_partkey#X]
Join type: Inner
Join condition: (((((((p_brand#X = Brand#X) AND p_container#X IN (SM CASE,SM BOX,SM PACK,SM PKG)) AND (l_quantity#X >= 1.00)) AND (l_quantity#X <= 11.00)) AND (p_size#X <= 5)) OR (((((p_brand#X = Brand#X) AND p_container#X IN (MED BAG,MED BOX,MED PKG,MED PACK)) AND (l_quantity#X >= 10.00)) AND (l_quantity#X <= 20.00)) AND (p_size#X <= 10))) OR (((((p_brand#X = Brand#X) AND p_container#X IN (LG CASE,LG BOX,LG PACK,LG PKG)) AND (l_quantity#X >= 20.00)) AND (l_quantity#X <= 30.00)) AND (p_size#X <= 15)))
-(35) Project
+(34) Project
Output [2]: [l_extendedprice#X, l_discount#X]
Input [8]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, p_partkey#X, p_brand#X, p_size#X, p_container#X]
-(36) HashAggregate
+(35) HashAggregate
Input [2]: [l_extendedprice#X, l_discount#X]
Keys: []
Functions [1]: [partial_sum((l_extendedprice#X * (1 - l_discount#X)))]
Aggregate Attributes [2]: [sum#X, isEmpty#X]
Results [2]: [sum#X, isEmpty#X]
-(37) HashAggregate
+(36) HashAggregate
Input [2]: [sum#X, isEmpty#X]
Keys: []
Functions [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))]
Aggregate Attributes [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))#X]
Results [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))#X AS revenue#X]
-(38) AdaptiveSparkPlan
+(37) AdaptiveSparkPlan
Output [1]: [revenue#X]
Arguments: isFinalPlan=true
\ No newline at end of file
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 ed09215178a4..4d8eee1f1fb7 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
@@ -1,32 +1,32 @@
== Physical Plan ==
-AdaptiveSparkPlan (143)
+AdaptiveSparkPlan (142)
+- == Final Plan ==
- VeloxColumnarToRow (93)
- +- AQEShuffleRead (92)
- +- ShuffleQueryStage (91), Statistics(X)
- +- ColumnarExchange (90)
- +- VeloxResizeBatches (89)
- +- ^ ProjectExecTransformer (87)
- +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (86)
- :- ^ InputIteratorTransformer (76)
- : +- ShuffleQueryStage (74), Statistics(X)
- : +- ColumnarExchange (73)
- : +- VeloxResizeBatches (72)
- : +- ^ ProjectExecTransformer (70)
- : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (69)
+ VeloxColumnarToRow (92)
+ +- AQEShuffleRead (91)
+ +- ShuffleQueryStage (90), Statistics(X)
+ +- ColumnarExchange (89)
+ +- VeloxResizeBatches (88)
+ +- ^ ProjectExecTransformer (86)
+ +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (85)
+ :- ^ InputIteratorTransformer (75)
+ : +- ShuffleQueryStage (73), Statistics(X)
+ : +- ColumnarExchange (72)
+ : +- VeloxResizeBatches (71)
+ : +- ^ ProjectExecTransformer (69)
+ : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (68)
: :- ^ InputIteratorTransformer (9)
: : +- ShuffleQueryStage (7), Statistics(X)
: : +- ColumnarExchange (6)
: : +- VeloxResizeBatches (5)
: : +- ^ ProjectExecTransformer (3)
: : +- ^ FilterExecTransformer (2)
- : : +- ^ Scan parquet (1)
- : +- ^ InputIteratorTransformer (68)
- : +- ShuffleQueryStage (66), Statistics(X)
- : +- ColumnarExchange (65)
- : +- VeloxResizeBatches (64)
- : +- ^ ProjectExecTransformer (62)
- : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (61)
+ : : +- ^ ScanTransformer parquet (1)
+ : +- ^ InputIteratorTransformer (67)
+ : +- ShuffleQueryStage (65), Statistics(X)
+ : +- ColumnarExchange (64)
+ : +- VeloxResizeBatches (63)
+ : +- ^ ProjectExecTransformer (61)
+ : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (60)
: :- ^ InputIteratorTransformer (35)
: : +- ShuffleQueryStage (33), Statistics(X)
: : +- ColumnarExchange (32)
@@ -39,94 +39,93 @@ AdaptiveSparkPlan (143)
: : : +- VeloxResizeBatches (14)
: : : +- ^ ProjectExecTransformer (12)
: : : +- ^ FilterExecTransformer (11)
- : : : +- ^ Scan parquet (10)
+ : : : +- ^ ScanTransformer parquet (10)
: : +- ^ InputIteratorTransformer (27)
: : +- ShuffleQueryStage (25), Statistics(X)
: : +- ColumnarExchange (24)
: : +- VeloxResizeBatches (23)
: : +- ^ ProjectExecTransformer (21)
: : +- ^ FilterExecTransformer (20)
- : : +- ^ Scan parquet (19)
- : +- ^ InputIteratorTransformer (60)
- : +- ShuffleQueryStage (58), Statistics(X)
- : +- ColumnarExchange (57)
- : +- VeloxResizeBatches (56)
- : +- ^ ProjectExecTransformer (54)
- : +- ^ FilterExecTransformer (53)
- : +- ^ ProjectExecTransformer (52)
- : +- ^ RegularHashAggregateExecTransformer (51)
- : +- ^ RegularHashAggregateExecTransformer (50)
- : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (49)
- : :- ^ InputIteratorTransformer (44)
- : : +- ShuffleQueryStage (42), Statistics(X)
- : : +- ColumnarExchange (41)
- : : +- VeloxResizeBatches (40)
- : : +- ^ ProjectExecTransformer (38)
- : : +- ^ FilterExecTransformer (37)
- : : +- ^ Scan parquet (36)
- : +- ^ InputIteratorTransformer (48)
- : +- ShuffleQueryStage (46), Statistics(X)
- : +- ReusedExchange (45)
- +- ^ InputIteratorTransformer (85)
- +- ShuffleQueryStage (83), Statistics(X)
- +- ColumnarExchange (82)
- +- VeloxResizeBatches (81)
- +- ^ ProjectExecTransformer (79)
- +- ^ FilterExecTransformer (78)
- +- ^ Scan parquet (77)
+ : : +- ^ ScanTransformer parquet (19)
+ : +- ^ InputIteratorTransformer (59)
+ : +- ShuffleQueryStage (57), Statistics(X)
+ : +- ColumnarExchange (56)
+ : +- VeloxResizeBatches (55)
+ : +- ^ ProjectExecTransformer (53)
+ : +- ^ FilterExecTransformer (52)
+ : +- ^ ProjectExecTransformer (51)
+ : +- ^ RegularHashAggregateExecTransformer (50)
+ : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (49)
+ : :- ^ InputIteratorTransformer (44)
+ : : +- ShuffleQueryStage (42), Statistics(X)
+ : : +- ColumnarExchange (41)
+ : : +- VeloxResizeBatches (40)
+ : : +- ^ ProjectExecTransformer (38)
+ : : +- ^ FilterExecTransformer (37)
+ : : +- ^ ScanTransformer parquet (36)
+ : +- ^ InputIteratorTransformer (48)
+ : +- ShuffleQueryStage (46), Statistics(X)
+ : +- ReusedExchange (45)
+ +- ^ InputIteratorTransformer (84)
+ +- ShuffleQueryStage (82), Statistics(X)
+ +- ColumnarExchange (81)
+ +- VeloxResizeBatches (80)
+ +- ^ ProjectExecTransformer (78)
+ +- ^ FilterExecTransformer (77)
+ +- ^ ScanTransformer parquet (76)
+- == Initial Plan ==
- Sort (142)
- +- Exchange (141)
- +- Project (140)
- +- SortMergeJoin Inner (139)
- :- Sort (133)
- : +- Exchange (132)
- : +- Project (131)
- : +- SortMergeJoin LeftSemi (130)
- : :- Sort (97)
- : : +- Exchange (96)
- : : +- Filter (95)
- : : +- Scan parquet (94)
- : +- Sort (129)
- : +- Exchange (128)
- : +- Project (127)
- : +- SortMergeJoin Inner (126)
- : :- Sort (109)
- : : +- Exchange (108)
- : : +- SortMergeJoin LeftSemi (107)
- : : :- Sort (101)
- : : : +- Exchange (100)
- : : : +- Filter (99)
- : : : +- Scan parquet (98)
- : : +- Sort (106)
- : : +- Exchange (105)
- : : +- Project (104)
- : : +- Filter (103)
- : : +- Scan parquet (102)
- : +- Sort (125)
- : +- Exchange (124)
- : +- Filter (123)
- : +- HashAggregate (122)
- : +- HashAggregate (121)
- : +- SortMergeJoin LeftSemi (120)
- : :- Sort (114)
- : : +- Exchange (113)
- : : +- Project (112)
- : : +- Filter (111)
- : : +- Scan parquet (110)
- : +- Sort (119)
- : +- Exchange (118)
- : +- Project (117)
- : +- Filter (116)
- : +- Scan parquet (115)
- +- Sort (138)
- +- Exchange (137)
- +- Project (136)
- +- Filter (135)
- +- Scan parquet (134)
-
-
-(1) Scan parquet
+ Sort (141)
+ +- Exchange (140)
+ +- Project (139)
+ +- SortMergeJoin Inner (138)
+ :- Sort (132)
+ : +- Exchange (131)
+ : +- Project (130)
+ : +- SortMergeJoin LeftSemi (129)
+ : :- Sort (96)
+ : : +- Exchange (95)
+ : : +- Filter (94)
+ : : +- Scan parquet (93)
+ : +- Sort (128)
+ : +- Exchange (127)
+ : +- Project (126)
+ : +- SortMergeJoin Inner (125)
+ : :- Sort (108)
+ : : +- Exchange (107)
+ : : +- SortMergeJoin LeftSemi (106)
+ : : :- Sort (100)
+ : : : +- Exchange (99)
+ : : : +- Filter (98)
+ : : : +- Scan parquet (97)
+ : : +- Sort (105)
+ : : +- Exchange (104)
+ : : +- Project (103)
+ : : +- Filter (102)
+ : : +- Scan parquet (101)
+ : +- Sort (124)
+ : +- Exchange (123)
+ : +- Filter (122)
+ : +- HashAggregate (121)
+ : +- HashAggregate (120)
+ : +- SortMergeJoin LeftSemi (119)
+ : :- Sort (113)
+ : : +- Exchange (112)
+ : : +- Project (111)
+ : : +- Filter (110)
+ : : +- Scan parquet (109)
+ : +- Sort (118)
+ : +- Exchange (117)
+ : +- Project (116)
+ : +- Filter (115)
+ : +- Scan parquet (114)
+ +- Sort (137)
+ +- Exchange (136)
+ +- Project (135)
+ +- Filter (134)
+ +- Scan parquet (133)
+
+
+(1) ScanTransformer parquet
Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -163,7 +162,7 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X]
(9) InputIteratorTransformer
Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -200,7 +199,7 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
(18) InputIteratorTransformer
Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
-(19) Scan parquet
+(19) ScanTransformer parquet
Output [2]: [p_partkey#X, p_name#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -269,7 +268,7 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
(35) InputIteratorTransformer
Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
-(36) Scan parquet
+(36) ScanTransformer parquet
Output [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -328,415 +327,408 @@ Join condition: None
(50) RegularHashAggregateExecTransformer
Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X]
Keys [2]: [l_partkey#X, l_suppkey#X]
-Functions [1]: [partial_sum(l_quantity#X)]
-Aggregate Attributes [2]: [sum#X, isEmpty#X]
-Results [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X]
-
-(51) RegularHashAggregateExecTransformer
-Input [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X]
-Keys [2]: [l_partkey#X, l_suppkey#X]
Functions [1]: [sum(l_quantity#X)]
Aggregate Attributes [1]: [sum(l_quantity#X)#X]
Results [3]: [l_partkey#X, l_suppkey#X, sum(l_quantity#X)#X]
-(52) ProjectExecTransformer
+(51) ProjectExecTransformer
Output [3]: [(0.5 * sum(l_quantity#X)#X) AS (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
Input [3]: [l_partkey#X, l_suppkey#X, sum(l_quantity#X)#X]
-(53) FilterExecTransformer
+(52) FilterExecTransformer
Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
Arguments: isnotnull((0.5 * sum(l_quantity))#X)
-(54) ProjectExecTransformer
+(53) ProjectExecTransformer
Output [4]: [hash(l_partkey#X, l_suppkey#X, 42) AS hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
-(55) WholeStageCodegenTransformer (X)
+(54) WholeStageCodegenTransformer (X)
Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
Arguments: false
-(56) VeloxResizeBatches
+(55) VeloxResizeBatches
Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
Arguments: X, X
-(57) ColumnarExchange
+(56) ColumnarExchange
Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
Arguments: hashpartitioning(l_partkey#X, l_suppkey#X, 1), ENSURE_REQUIREMENTS, [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X], [plan_id=X], [shuffle_writer_type=hash]
-(58) ShuffleQueryStage
+(57) ShuffleQueryStage
Output [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
Arguments: X
-(59) InputAdapter
+(58) InputAdapter
Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
-(60) InputIteratorTransformer
+(59) InputIteratorTransformer
Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
-(61) ShuffledHashJoinExecTransformer
+(60) ShuffledHashJoinExecTransformer
Left keys [2]: [ps_partkey#X, ps_suppkey#X]
Right keys [2]: [l_partkey#X, l_suppkey#X]
Join type: Inner
Join condition: (cast(ps_availqty#X as decimal(24,3)) > (0.5 * sum(l_quantity))#X)
-(62) ProjectExecTransformer
+(61) ProjectExecTransformer
Output [2]: [hash(ps_suppkey#X, 42) AS hash_partition_key#X, ps_suppkey#X]
Input [6]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
-(63) WholeStageCodegenTransformer (X)
+(62) WholeStageCodegenTransformer (X)
Input [2]: [hash_partition_key#X, ps_suppkey#X]
Arguments: false
-(64) VeloxResizeBatches
+(63) VeloxResizeBatches
Input [2]: [hash_partition_key#X, ps_suppkey#X]
Arguments: X, X
-(65) ColumnarExchange
+(64) ColumnarExchange
Input [2]: [hash_partition_key#X, ps_suppkey#X]
Arguments: hashpartitioning(ps_suppkey#X, 1), ENSURE_REQUIREMENTS, [ps_suppkey#X], [plan_id=X], [shuffle_writer_type=hash]
-(66) ShuffleQueryStage
+(65) ShuffleQueryStage
Output [1]: [ps_suppkey#X]
Arguments: X
-(67) InputAdapter
+(66) InputAdapter
Input [1]: [ps_suppkey#X]
-(68) InputIteratorTransformer
+(67) InputIteratorTransformer
Input [1]: [ps_suppkey#X]
-(69) ShuffledHashJoinExecTransformer
+(68) ShuffledHashJoinExecTransformer
Left keys [1]: [s_suppkey#X]
Right keys [1]: [ps_suppkey#X]
Join type: LeftSemi
Join condition: None
-(70) ProjectExecTransformer
+(69) ProjectExecTransformer
Output [4]: [hash(s_nationkey#X, 42) AS hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X]
Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X]
-(71) WholeStageCodegenTransformer (X)
+(70) WholeStageCodegenTransformer (X)
Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X]
Arguments: false
-(72) VeloxResizeBatches
+(71) VeloxResizeBatches
Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X]
Arguments: X, X
-(73) ColumnarExchange
+(72) ColumnarExchange
Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X]
Arguments: hashpartitioning(s_nationkey#X, 1), ENSURE_REQUIREMENTS, [s_name#X, s_address#X, s_nationkey#X], [plan_id=X], [shuffle_writer_type=hash]
-(74) ShuffleQueryStage
+(73) ShuffleQueryStage
Output [3]: [s_name#X, s_address#X, s_nationkey#X]
Arguments: X
-(75) InputAdapter
+(74) InputAdapter
Input [3]: [s_name#X, s_address#X, s_nationkey#X]
-(76) InputIteratorTransformer
+(75) InputIteratorTransformer
Input [3]: [s_name#X, s_address#X, s_nationkey#X]
-(77) Scan parquet
+(76) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(n_name), EqualTo(n_name,CANADA), IsNotNull(n_nationkey)]
ReadSchema: struct
-(78) FilterExecTransformer
+(77) FilterExecTransformer
Input [2]: [n_nationkey#X, n_name#X]
Arguments: ((isnotnull(n_name#X) AND (n_name#X = CANADA)) AND isnotnull(n_nationkey#X))
-(79) ProjectExecTransformer
+(78) ProjectExecTransformer
Output [2]: [hash(n_nationkey#X, 42) AS hash_partition_key#X, n_nationkey#X]
Input [2]: [n_nationkey#X, n_name#X]
-(80) WholeStageCodegenTransformer (X)
+(79) WholeStageCodegenTransformer (X)
Input [2]: [hash_partition_key#X, n_nationkey#X]
Arguments: false
-(81) VeloxResizeBatches
+(80) VeloxResizeBatches
Input [2]: [hash_partition_key#X, n_nationkey#X]
Arguments: X, X
-(82) ColumnarExchange
+(81) ColumnarExchange
Input [2]: [hash_partition_key#X, n_nationkey#X]
Arguments: hashpartitioning(n_nationkey#X, 1), ENSURE_REQUIREMENTS, [n_nationkey#X], [plan_id=X], [shuffle_writer_type=hash]
-(83) ShuffleQueryStage
+(82) ShuffleQueryStage
Output [1]: [n_nationkey#X]
Arguments: X
-(84) InputAdapter
+(83) InputAdapter
Input [1]: [n_nationkey#X]
-(85) InputIteratorTransformer
+(84) InputIteratorTransformer
Input [1]: [n_nationkey#X]
-(86) ShuffledHashJoinExecTransformer
+(85) ShuffledHashJoinExecTransformer
Left keys [1]: [s_nationkey#X]
Right keys [1]: [n_nationkey#X]
Join type: Inner
Join condition: None
-(87) ProjectExecTransformer
+(86) ProjectExecTransformer
Output [2]: [s_name#X, s_address#X]
Input [4]: [s_name#X, s_address#X, s_nationkey#X, n_nationkey#X]
-(88) WholeStageCodegenTransformer (X)
+(87) WholeStageCodegenTransformer (X)
Input [2]: [s_name#X, s_address#X]
Arguments: false
-(89) VeloxResizeBatches
+(88) VeloxResizeBatches
Input [2]: [s_name#X, s_address#X]
Arguments: X, X
-(90) ColumnarExchange
+(89) ColumnarExchange
Input [2]: [s_name#X, s_address#X]
Arguments: rangepartitioning(s_name#X ASC NULLS FIRST, 1), ENSURE_REQUIREMENTS, [plan_id=X], [shuffle_writer_type=hash]
-(91) ShuffleQueryStage
+(90) ShuffleQueryStage
Output [2]: [s_name#X, s_address#X]
Arguments: X
-(92) AQEShuffleRead
+(91) AQEShuffleRead
Input [2]: [s_name#X, s_address#X]
Arguments: local
-(93) VeloxColumnarToRow
+(92) VeloxColumnarToRow
Input [2]: [s_name#X, s_address#X]
-(94) Scan parquet
+(93) Scan parquet
Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(s_nationkey)]
ReadSchema: struct
-(95) Filter
+(94) Filter
Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X]
Condition : isnotnull(s_nationkey#X)
-(96) Exchange
+(95) Exchange
Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X]
Arguments: hashpartitioning(s_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(97) Sort
+(96) Sort
Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X]
Arguments: [s_suppkey#X ASC NULLS FIRST], false, 0
-(98) Scan parquet
+(97) Scan parquet
Output [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(ps_availqty), IsNotNull(ps_partkey), IsNotNull(ps_suppkey)]
ReadSchema: struct
-(99) Filter
+(98) Filter
Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
Condition : ((isnotnull(ps_availqty#X) AND isnotnull(ps_partkey#X)) AND isnotnull(ps_suppkey#X))
-(100) Exchange
+(99) Exchange
Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
Arguments: hashpartitioning(ps_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(101) Sort
+(100) Sort
Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
Arguments: [ps_partkey#X ASC NULLS FIRST], false, 0
-(102) Scan parquet
+(101) Scan parquet
Output [2]: [p_partkey#X, p_name#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(p_name), StringStartsWith(p_name,forest)]
ReadSchema: struct
-(103) Filter
+(102) Filter
Input [2]: [p_partkey#X, p_name#X]
Condition : (isnotnull(p_name#X) AND StartsWith(p_name#X, forest))
-(104) Project
+(103) Project
Output [1]: [p_partkey#X]
Input [2]: [p_partkey#X, p_name#X]
-(105) Exchange
+(104) Exchange
Input [1]: [p_partkey#X]
Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(106) Sort
+(105) Sort
Input [1]: [p_partkey#X]
Arguments: [p_partkey#X ASC NULLS FIRST], false, 0
-(107) SortMergeJoin
+(106) SortMergeJoin
Left keys [1]: [ps_partkey#X]
Right keys [1]: [p_partkey#X]
Join type: LeftSemi
Join condition: None
-(108) Exchange
+(107) Exchange
Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
Arguments: hashpartitioning(ps_partkey#X, ps_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(109) Sort
+(108) Sort
Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
Arguments: [ps_partkey#X ASC NULLS FIRST, ps_suppkey#X ASC NULLS FIRST], false, 0
-(110) Scan parquet
+(109) Scan parquet
Output [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1994-01-01), LessThan(l_shipdate,1995-01-01), IsNotNull(l_partkey), IsNotNull(l_suppkey)]
ReadSchema: struct
-(111) Filter
+(110) Filter
Input [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X]
Condition : ((((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1994-01-01)) AND (l_shipdate#X < 1995-01-01)) AND isnotnull(l_partkey#X)) AND isnotnull(l_suppkey#X))
-(112) Project
+(111) Project
Output [3]: [l_partkey#X, l_suppkey#X, l_quantity#X]
Input [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X]
-(113) Exchange
+(112) Exchange
Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X]
Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(114) Sort
+(113) Sort
Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X]
Arguments: [l_partkey#X ASC NULLS FIRST], false, 0
-(115) Scan parquet
+(114) Scan parquet
Output [2]: [p_partkey#X, p_name#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(p_name), StringStartsWith(p_name,forest)]
ReadSchema: struct
-(116) Filter
+(115) Filter
Input [2]: [p_partkey#X, p_name#X]
Condition : (isnotnull(p_name#X) AND StartsWith(p_name#X, forest))
-(117) Project
+(116) Project
Output [1]: [p_partkey#X]
Input [2]: [p_partkey#X, p_name#X]
-(118) Exchange
+(117) Exchange
Input [1]: [p_partkey#X]
Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(119) Sort
+(118) Sort
Input [1]: [p_partkey#X]
Arguments: [p_partkey#X ASC NULLS FIRST], false, 0
-(120) SortMergeJoin
+(119) SortMergeJoin
Left keys [1]: [l_partkey#X]
Right keys [1]: [p_partkey#X]
Join type: LeftSemi
Join condition: None
-(121) HashAggregate
+(120) HashAggregate
Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X]
Keys [2]: [l_partkey#X, l_suppkey#X]
Functions [1]: [partial_sum(l_quantity#X)]
Aggregate Attributes [2]: [sum#X, isEmpty#X]
Results [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X]
-(122) HashAggregate
+(121) HashAggregate
Input [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X]
Keys [2]: [l_partkey#X, l_suppkey#X]
Functions [1]: [sum(l_quantity#X)]
Aggregate Attributes [1]: [sum(l_quantity#X)#X]
Results [3]: [(0.5 * sum(l_quantity#X)#X) AS (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
-(123) Filter
+(122) Filter
Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
Condition : isnotnull((0.5 * sum(l_quantity))#X)
-(124) Exchange
+(123) Exchange
Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
Arguments: hashpartitioning(l_partkey#X, l_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(125) Sort
+(124) Sort
Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
Arguments: [l_partkey#X ASC NULLS FIRST, l_suppkey#X ASC NULLS FIRST], false, 0
-(126) SortMergeJoin
+(125) SortMergeJoin
Left keys [2]: [ps_partkey#X, ps_suppkey#X]
Right keys [2]: [l_partkey#X, l_suppkey#X]
Join type: Inner
Join condition: (cast(ps_availqty#X as decimal(24,3)) > (0.5 * sum(l_quantity))#X)
-(127) Project
+(126) Project
Output [1]: [ps_suppkey#X]
Input [6]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
-(128) Exchange
+(127) Exchange
Input [1]: [ps_suppkey#X]
Arguments: hashpartitioning(ps_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(129) Sort
+(128) Sort
Input [1]: [ps_suppkey#X]
Arguments: [ps_suppkey#X ASC NULLS FIRST], false, 0
-(130) SortMergeJoin
+(129) SortMergeJoin
Left keys [1]: [s_suppkey#X]
Right keys [1]: [ps_suppkey#X]
Join type: LeftSemi
Join condition: None
-(131) Project
+(130) Project
Output [3]: [s_name#X, s_address#X, s_nationkey#X]
Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X]
-(132) Exchange
+(131) Exchange
Input [3]: [s_name#X, s_address#X, s_nationkey#X]
Arguments: hashpartitioning(s_nationkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(133) Sort
+(132) Sort
Input [3]: [s_name#X, s_address#X, s_nationkey#X]
Arguments: [s_nationkey#X ASC NULLS FIRST], false, 0
-(134) Scan parquet
+(133) Scan parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(n_name), EqualTo(n_name,CANADA), IsNotNull(n_nationkey)]
ReadSchema: struct
-(135) Filter
+(134) Filter
Input [2]: [n_nationkey#X, n_name#X]
Condition : ((isnotnull(n_name#X) AND (n_name#X = CANADA)) AND isnotnull(n_nationkey#X))
-(136) Project
+(135) Project
Output [1]: [n_nationkey#X]
Input [2]: [n_nationkey#X, n_name#X]
-(137) Exchange
+(136) Exchange
Input [1]: [n_nationkey#X]
Arguments: hashpartitioning(n_nationkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(138) Sort
+(137) Sort
Input [1]: [n_nationkey#X]
Arguments: [n_nationkey#X ASC NULLS FIRST], false, 0
-(139) SortMergeJoin
+(138) SortMergeJoin
Left keys [1]: [s_nationkey#X]
Right keys [1]: [n_nationkey#X]
Join type: Inner
Join condition: None
-(140) Project
+(139) Project
Output [2]: [s_name#X, s_address#X]
Input [4]: [s_name#X, s_address#X, s_nationkey#X, n_nationkey#X]
-(141) Exchange
+(140) Exchange
Input [2]: [s_name#X, s_address#X]
Arguments: rangepartitioning(s_name#X ASC NULLS FIRST, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(142) Sort
+(141) Sort
Input [2]: [s_name#X, s_address#X]
Arguments: [s_name#X ASC NULLS FIRST], true, 0
-(143) AdaptiveSparkPlan
+(142) AdaptiveSparkPlan
Output [2]: [s_name#X, s_address#X]
Arguments: isFinalPlan=true
\ No newline at end of file
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 271d0c6d5098..96750b2b3ecc 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
@@ -29,7 +29,7 @@ AdaptiveSparkPlan (137)
: : : +- VeloxResizeBatches (5)
: : : +- ^ ProjectExecTransformer (3)
: : : +- ^ FilterExecTransformer (2)
- : : : +- ^ Scan parquet (1)
+ : : : +- ^ ScanTransformer parquet (1)
: : +- ^ InputIteratorTransformer (44)
: : +- ShuffleQueryStage (42), Statistics(X)
: : +- ColumnarExchange (41)
@@ -43,34 +43,34 @@ AdaptiveSparkPlan (137)
: : : : +- VeloxResizeBatches (14)
: : : : +- ^ ProjectExecTransformer (12)
: : : : +- ^ FilterExecTransformer (11)
- : : : : +- ^ Scan parquet (10)
+ : : : : +- ^ ScanTransformer parquet (10)
: : : +- ^ InputIteratorTransformer (26)
: : : +- ShuffleQueryStage (24), Statistics(X)
: : : +- ColumnarExchange (23)
: : : +- VeloxResizeBatches (22)
: : : +- ^ ProjectExecTransformer (20)
- : : : +- ^ Scan parquet (19)
+ : : : +- ^ ScanTransformer parquet (19)
: : +- ^ InputIteratorTransformer (36)
: : +- ShuffleQueryStage (34), Statistics(X)
: : +- ColumnarExchange (33)
: : +- VeloxResizeBatches (32)
: : +- ^ ProjectExecTransformer (30)
: : +- ^ FilterExecTransformer (29)
- : : +- ^ Scan parquet (28)
+ : : +- ^ ScanTransformer parquet (28)
: +- ^ InputIteratorTransformer (61)
: +- ShuffleQueryStage (59), Statistics(X)
: +- ColumnarExchange (58)
: +- VeloxResizeBatches (57)
: +- ^ ProjectExecTransformer (55)
: +- ^ FilterExecTransformer (54)
- : +- ^ Scan parquet (53)
+ : +- ^ ScanTransformer parquet (53)
+- ^ InputIteratorTransformer (78)
+- ShuffleQueryStage (76), Statistics(X)
+- ColumnarExchange (75)
+- VeloxResizeBatches (74)
+- ^ ProjectExecTransformer (72)
+- ^ FilterExecTransformer (71)
- +- ^ Scan parquet (70)
+ +- ^ ScanTransformer parquet (70)
+- == Initial Plan ==
TakeOrderedAndProject (136)
+- HashAggregate (135)
@@ -119,7 +119,7 @@ AdaptiveSparkPlan (137)
+- Scan parquet (126)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [3]: [s_suppkey#X, s_name#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -156,7 +156,7 @@ Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X]
(9) InputIteratorTransformer
Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -193,7 +193,7 @@ Input [2]: [l_orderkey#X, l_suppkey#X]
(18) InputIteratorTransformer
Input [2]: [l_orderkey#X, l_suppkey#X]
-(19) Scan parquet
+(19) ScanTransformer parquet
Output [2]: [l_orderkey#X, l_suppkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -231,7 +231,7 @@ Right keys [1]: [l_orderkey#X]
Join type: LeftSemi
Join condition: NOT (l_suppkey#X = l_suppkey#X)
-(28) Scan parquet
+(28) ScanTransformer parquet
Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -332,7 +332,7 @@ Input [3]: [s_name#X, s_nationkey#X, l_orderkey#X]
(52) InputIteratorTransformer
Input [3]: [s_name#X, s_nationkey#X, l_orderkey#X]
-(53) Scan parquet
+(53) ScanTransformer parquet
Output [2]: [o_orderkey#X, o_orderstatus#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -401,7 +401,7 @@ Input [2]: [s_name#X, s_nationkey#X]
(69) InputIteratorTransformer
Input [2]: [s_name#X, s_nationkey#X]
-(70) Scan parquet
+(70) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 b8123fb6dc0e..c6e147da04d8 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
@@ -22,13 +22,13 @@ AdaptiveSparkPlan (52)
: +- VeloxResizeBatches (5)
: +- ^ ProjectExecTransformer (3)
: +- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
+ : +- ^ ScanTransformer parquet (1)
+- ^ InputIteratorTransformer (17)
+- ShuffleQueryStage (15), Statistics(X)
+- ColumnarExchange (14)
+- VeloxResizeBatches (13)
+- ^ ProjectExecTransformer (11)
- +- ^ Scan parquet (10)
+ +- ^ ScanTransformer parquet (10)
+- == Initial Plan ==
Sort (51)
+- Exchange (50)
@@ -46,7 +46,7 @@ AdaptiveSparkPlan (52)
+- Scan parquet (42)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [3]: [c_custkey#X, c_phone#X, c_acctbal#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -83,7 +83,7 @@ Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X]
(9) InputIteratorTransformer
Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [1]: [o_custkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -285,7 +285,7 @@ AdaptiveSparkPlan (72)
+- ^ FlushableHashAggregateExecTransformer (56)
+- ^ ProjectExecTransformer (55)
+- ^ FilterExecTransformer (54)
- +- ^ Scan parquet (53)
+ +- ^ ScanTransformer parquet (53)
+- == Initial Plan ==
HashAggregate (71)
+- Exchange (70)
@@ -295,7 +295,7 @@ AdaptiveSparkPlan (72)
+- Scan parquet (66)
-(53) Scan parquet
+(53) ScanTransformer parquet
Output [2]: [c_phone#X, c_acctbal#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -402,7 +402,7 @@ AdaptiveSparkPlan (72)
+- ^ FlushableHashAggregateExecTransformer (56)
+- ^ ProjectExecTransformer (55)
+- ^ FilterExecTransformer (54)
- +- ^ Scan parquet (53)
+ +- ^ ScanTransformer parquet (53)
+- == Initial Plan ==
HashAggregate (71)
+- Exchange (70)
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 8eea7581a417..1f86369a62af 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
@@ -1,67 +1,66 @@
== Physical Plan ==
-AdaptiveSparkPlan (67)
+AdaptiveSparkPlan (66)
+- == Final Plan ==
- VeloxColumnarToRow (43)
- +- TakeOrderedAndProjectExecTransformer (42)
- +- ^ ProjectExecTransformer (40)
- +- ^ RegularHashAggregateExecTransformer (39)
- +- ^ RegularHashAggregateExecTransformer (38)
- +- ^ ProjectExecTransformer (37)
- +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (36)
- :- ^ InputIteratorTransformer (26)
- : +- ShuffleQueryStage (24), Statistics(X)
- : +- ColumnarExchange (23)
- : +- VeloxResizeBatches (22)
- : +- ^ ProjectExecTransformer (20)
- : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19)
- : :- ^ InputIteratorTransformer (9)
- : : +- ShuffleQueryStage (7), Statistics(X)
- : : +- ColumnarExchange (6)
- : : +- VeloxResizeBatches (5)
- : : +- ^ ProjectExecTransformer (3)
- : : +- ^ FilterExecTransformer (2)
- : : +- ^ Scan parquet (1)
- : +- ^ InputIteratorTransformer (18)
- : +- ShuffleQueryStage (16), Statistics(X)
- : +- ColumnarExchange (15)
- : +- VeloxResizeBatches (14)
- : +- ^ ProjectExecTransformer (12)
- : +- ^ FilterExecTransformer (11)
- : +- ^ Scan parquet (10)
- +- ^ InputIteratorTransformer (35)
- +- ShuffleQueryStage (33), Statistics(X)
- +- ColumnarExchange (32)
- +- VeloxResizeBatches (31)
- +- ^ ProjectExecTransformer (29)
- +- ^ FilterExecTransformer (28)
- +- ^ Scan parquet (27)
+ VeloxColumnarToRow (42)
+ +- TakeOrderedAndProjectExecTransformer (41)
+ +- ^ ProjectExecTransformer (39)
+ +- ^ RegularHashAggregateExecTransformer (38)
+ +- ^ ProjectExecTransformer (37)
+ +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (36)
+ :- ^ InputIteratorTransformer (26)
+ : +- ShuffleQueryStage (24), Statistics(X)
+ : +- ColumnarExchange (23)
+ : +- VeloxResizeBatches (22)
+ : +- ^ ProjectExecTransformer (20)
+ : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19)
+ : :- ^ InputIteratorTransformer (9)
+ : : +- ShuffleQueryStage (7), Statistics(X)
+ : : +- ColumnarExchange (6)
+ : : +- VeloxResizeBatches (5)
+ : : +- ^ ProjectExecTransformer (3)
+ : : +- ^ FilterExecTransformer (2)
+ : : +- ^ ScanTransformer parquet (1)
+ : +- ^ InputIteratorTransformer (18)
+ : +- ShuffleQueryStage (16), Statistics(X)
+ : +- ColumnarExchange (15)
+ : +- VeloxResizeBatches (14)
+ : +- ^ ProjectExecTransformer (12)
+ : +- ^ FilterExecTransformer (11)
+ : +- ^ ScanTransformer parquet (10)
+ +- ^ InputIteratorTransformer (35)
+ +- ShuffleQueryStage (33), Statistics(X)
+ +- ColumnarExchange (32)
+ +- VeloxResizeBatches (31)
+ +- ^ ProjectExecTransformer (29)
+ +- ^ FilterExecTransformer (28)
+ +- ^ ScanTransformer parquet (27)
+- == Initial Plan ==
- TakeOrderedAndProject (66)
- +- HashAggregate (65)
- +- HashAggregate (64)
- +- Project (63)
- +- SortMergeJoin Inner (62)
- :- Sort (56)
- : +- Exchange (55)
- : +- Project (54)
- : +- SortMergeJoin Inner (53)
- : :- Sort (48)
- : : +- Exchange (47)
- : : +- Project (46)
- : : +- Filter (45)
- : : +- Scan parquet (44)
- : +- Sort (52)
- : +- Exchange (51)
- : +- Filter (50)
- : +- Scan parquet (49)
- +- Sort (61)
- +- Exchange (60)
- +- Project (59)
- +- Filter (58)
- +- Scan parquet (57)
-
-
-(1) Scan parquet
+ TakeOrderedAndProject (65)
+ +- HashAggregate (64)
+ +- HashAggregate (63)
+ +- Project (62)
+ +- SortMergeJoin Inner (61)
+ :- Sort (55)
+ : +- Exchange (54)
+ : +- Project (53)
+ : +- SortMergeJoin Inner (52)
+ : :- Sort (47)
+ : : +- Exchange (46)
+ : : +- Project (45)
+ : : +- Filter (44)
+ : : +- Scan parquet (43)
+ : +- Sort (51)
+ : +- Exchange (50)
+ : +- Filter (49)
+ : +- Scan parquet (48)
+ +- Sort (60)
+ +- Exchange (59)
+ +- Project (58)
+ +- Filter (57)
+ +- Scan parquet (56)
+
+
+(1) ScanTransformer parquet
Output [2]: [c_custkey#X, c_mktsegment#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -98,7 +97,7 @@ Input [1]: [c_custkey#X]
(9) InputIteratorTransformer
Input [1]: [c_custkey#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -167,7 +166,7 @@ Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X]
(26) InputIteratorTransformer
Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X]
-(27) Scan parquet
+(27) ScanTransformer parquet
Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -217,143 +216,136 @@ Input [6]: [o_orderkey#X, o_orderdate#X, o_shippriority#X, l_orderkey#X, l_exten
(38) RegularHashAggregateExecTransformer
Input [6]: [o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X, _pre_X#X]
Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X]
-Functions [1]: [partial_sum(_pre_X#X)]
-Aggregate Attributes [2]: [sum#X, isEmpty#X]
-Results [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X]
-
-(39) RegularHashAggregateExecTransformer
-Input [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X]
-Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X]
-Functions [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))]
+Functions [1]: [sum(_pre_X#X)]
Aggregate Attributes [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))#X]
Results [4]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum((l_extendedprice#X * (1 - l_discount#X)))#X]
-(40) ProjectExecTransformer
+(39) ProjectExecTransformer
Output [4]: [l_orderkey#X, sum((l_extendedprice#X * (1 - l_discount#X)))#X AS revenue#X, o_orderdate#X, o_shippriority#X]
Input [4]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum((l_extendedprice#X * (1 - l_discount#X)))#X]
-(41) WholeStageCodegenTransformer (X)
+(40) WholeStageCodegenTransformer (X)
Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X]
Arguments: false
-(42) TakeOrderedAndProjectExecTransformer
+(41) TakeOrderedAndProjectExecTransformer
Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X]
Arguments: X, [revenue#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X], 0
-(43) VeloxColumnarToRow
+(42) VeloxColumnarToRow
Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X]
-(44) Scan parquet
+(43) Scan parquet
Output [2]: [c_custkey#X, c_mktsegment#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(c_mktsegment), EqualTo(c_mktsegment,BUILDING), IsNotNull(c_custkey)]
ReadSchema: struct
-(45) Filter
+(44) Filter
Input [2]: [c_custkey#X, c_mktsegment#X]
Condition : ((isnotnull(c_mktsegment#X) AND (c_mktsegment#X = BUILDING)) AND isnotnull(c_custkey#X))
-(46) Project
+(45) Project
Output [1]: [c_custkey#X]
Input [2]: [c_custkey#X, c_mktsegment#X]
-(47) Exchange
+(46) Exchange
Input [1]: [c_custkey#X]
Arguments: hashpartitioning(c_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(48) Sort
+(47) Sort
Input [1]: [c_custkey#X]
Arguments: [c_custkey#X ASC NULLS FIRST], false, 0
-(49) Scan parquet
+(48) Scan parquet
Output [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(o_orderdate), LessThan(o_orderdate,1995-03-15), IsNotNull(o_custkey), IsNotNull(o_orderkey)]
ReadSchema: struct
-(50) Filter
+(49) Filter
Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X]
Condition : (((isnotnull(o_orderdate#X) AND (o_orderdate#X < 1995-03-15)) AND isnotnull(o_custkey#X)) AND isnotnull(o_orderkey#X))
-(51) Exchange
+(50) Exchange
Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X]
Arguments: hashpartitioning(o_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(52) Sort
+(51) Sort
Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X]
Arguments: [o_custkey#X ASC NULLS FIRST], false, 0
-(53) SortMergeJoin
+(52) SortMergeJoin
Left keys [1]: [c_custkey#X]
Right keys [1]: [o_custkey#X]
Join type: Inner
Join condition: None
-(54) Project
+(53) Project
Output [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X]
Input [5]: [c_custkey#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X]
-(55) Exchange
+(54) Exchange
Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X]
Arguments: hashpartitioning(o_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(56) Sort
+(55) Sort
Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X]
Arguments: [o_orderkey#X ASC NULLS FIRST], false, 0
-(57) Scan parquet
+(56) Scan parquet
Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(l_shipdate), GreaterThan(l_shipdate,1995-03-15), IsNotNull(l_orderkey)]
ReadSchema: struct
-(58) Filter
+(57) Filter
Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Condition : ((isnotnull(l_shipdate#X) AND (l_shipdate#X > 1995-03-15)) AND isnotnull(l_orderkey#X))
-(59) Project
+(58) Project
Output [3]: [l_orderkey#X, l_extendedprice#X, l_discount#X]
Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
-(60) Exchange
+(59) Exchange
Input [3]: [l_orderkey#X, l_extendedprice#X, l_discount#X]
Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(61) Sort
+(60) Sort
Input [3]: [l_orderkey#X, l_extendedprice#X, l_discount#X]
Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0
-(62) SortMergeJoin
+(61) SortMergeJoin
Left keys [1]: [o_orderkey#X]
Right keys [1]: [l_orderkey#X]
Join type: Inner
Join condition: None
-(63) Project
+(62) Project
Output [5]: [o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X]
Input [6]: [o_orderkey#X, o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X]
-(64) HashAggregate
+(63) HashAggregate
Input [5]: [o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X]
Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X]
Functions [1]: [partial_sum((l_extendedprice#X * (1 - l_discount#X)))]
Aggregate Attributes [2]: [sum#X, isEmpty#X]
Results [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X]
-(65) HashAggregate
+(64) HashAggregate
Input [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X]
Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X]
Functions [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))]
Aggregate Attributes [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))#X]
Results [4]: [l_orderkey#X, sum((l_extendedprice#X * (1 - l_discount#X)))#X AS revenue#X, o_orderdate#X, o_shippriority#X]
-(66) TakeOrderedAndProject
+(65) TakeOrderedAndProject
Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X]
Arguments: X, [revenue#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X]
-(67) AdaptiveSparkPlan
+(66) AdaptiveSparkPlan
Output [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X]
Arguments: isFinalPlan=true
\ No newline at end of file
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 286881fb99b1..71a8f6974ed1 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
@@ -22,14 +22,14 @@ AdaptiveSparkPlan (56)
: +- VeloxResizeBatches (5)
: +- ^ ProjectExecTransformer (3)
: +- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
+ : +- ^ ScanTransformer parquet (1)
+- ^ InputIteratorTransformer (18)
+- ShuffleQueryStage (16), Statistics(X)
+- ColumnarExchange (15)
+- VeloxResizeBatches (14)
+- ^ ProjectExecTransformer (12)
+- ^ FilterExecTransformer (11)
- +- ^ Scan parquet (10)
+ +- ^ ScanTransformer parquet (10)
+- == Initial Plan ==
Sort (55)
+- Exchange (54)
@@ -50,7 +50,7 @@ AdaptiveSparkPlan (56)
+- Scan parquet (44)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -87,7 +87,7 @@ Input [2]: [o_orderkey#X, o_orderpriority#X]
(9) InputIteratorTransformer
Input [2]: [o_orderkey#X, o_orderpriority#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [3]: [l_orderkey#X, l_commitdate#X, l_receiptdate#X]
Batched: true
Location: InMemoryFileIndex [*]
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 0a9af460526c..da9339abd628 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
@@ -46,42 +46,42 @@ AdaptiveSparkPlan (156)
: : : : : +- VeloxResizeBatches (5)
: : : : : +- ^ ProjectExecTransformer (3)
: : : : : +- ^ FilterExecTransformer (2)
- : : : : : +- ^ Scan parquet (1)
+ : : : : : +- ^ ScanTransformer parquet (1)
: : : : +- ^ InputIteratorTransformer (18)
: : : : +- ShuffleQueryStage (16), Statistics(X)
: : : : +- ColumnarExchange (15)
: : : : +- VeloxResizeBatches (14)
: : : : +- ^ ProjectExecTransformer (12)
: : : : +- ^ FilterExecTransformer (11)
- : : : : +- ^ Scan parquet (10)
+ : : : : +- ^ ScanTransformer parquet (10)
: : : +- ^ InputIteratorTransformer (35)
: : : +- ShuffleQueryStage (33), Statistics(X)
: : : +- ColumnarExchange (32)
: : : +- VeloxResizeBatches (31)
: : : +- ^ ProjectExecTransformer (29)
: : : +- ^ FilterExecTransformer (28)
- : : : +- ^ Scan parquet (27)
+ : : : +- ^ ScanTransformer parquet (27)
: : +- ^ InputIteratorTransformer (52)
: : +- ShuffleQueryStage (50), Statistics(X)
: : +- ColumnarExchange (49)
: : +- VeloxResizeBatches (48)
: : +- ^ ProjectExecTransformer (46)
: : +- ^ FilterExecTransformer (45)
- : : +- ^ Scan parquet (44)
+ : : +- ^ ScanTransformer parquet (44)
: +- ^ InputIteratorTransformer (69)
: +- ShuffleQueryStage (67), Statistics(X)
: +- ColumnarExchange (66)
: +- VeloxResizeBatches (65)
: +- ^ ProjectExecTransformer (63)
: +- ^ FilterExecTransformer (62)
- : +- ^ Scan parquet (61)
+ : +- ^ ScanTransformer parquet (61)
+- ^ InputIteratorTransformer (86)
+- ShuffleQueryStage (84), Statistics(X)
+- ColumnarExchange (83)
+- VeloxResizeBatches (82)
+- ^ ProjectExecTransformer (80)
+- ^ FilterExecTransformer (79)
- +- ^ Scan parquet (78)
+ +- ^ ScanTransformer parquet (78)
+- == Initial Plan ==
Sort (155)
+- Exchange (154)
@@ -134,7 +134,7 @@ AdaptiveSparkPlan (156)
+- Scan parquet (144)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [c_custkey#X, c_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -171,7 +171,7 @@ Input [2]: [c_custkey#X, c_nationkey#X]
(9) InputIteratorTransformer
Input [2]: [c_custkey#X, c_nationkey#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -240,7 +240,7 @@ Input [2]: [c_nationkey#X, o_orderkey#X]
(26) InputIteratorTransformer
Input [2]: [c_nationkey#X, o_orderkey#X]
-(27) Scan parquet
+(27) ScanTransformer parquet
Output [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -309,7 +309,7 @@ Input [4]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
(43) InputIteratorTransformer
Input [4]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
-(44) Scan parquet
+(44) ScanTransformer parquet
Output [2]: [s_suppkey#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -378,7 +378,7 @@ Input [3]: [l_extendedprice#X, l_discount#X, s_nationkey#X]
(60) InputIteratorTransformer
Input [3]: [l_extendedprice#X, l_discount#X, s_nationkey#X]
-(61) Scan parquet
+(61) ScanTransformer parquet
Output [3]: [n_nationkey#X, n_name#X, n_regionkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -447,7 +447,7 @@ Input [4]: [l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X]
(77) InputIteratorTransformer
Input [4]: [l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X]
-(78) Scan parquet
+(78) ScanTransformer parquet
Output [2]: [r_regionkey#X, r_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 e9dc68456e55..319c457c1b4f 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
@@ -10,7 +10,7 @@ AdaptiveSparkPlan (20)
+- ^ FlushableHashAggregateExecTransformer (4)
+- ^ ProjectExecTransformer (3)
+- ^ FilterExecTransformer (2)
- +- ^ Scan parquet (1)
+ +- ^ ScanTransformer parquet (1)
+- == Initial Plan ==
HashAggregate (19)
+- Exchange (18)
@@ -20,7 +20,7 @@ AdaptiveSparkPlan (20)
+- Scan parquet (14)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [4]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
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 aa69dbcd11f7..a0894489f0a5 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
@@ -46,35 +46,35 @@ AdaptiveSparkPlan (149)
: : : : : +- VeloxResizeBatches (5)
: : : : : +- ^ ProjectExecTransformer (3)
: : : : : +- ^ FilterExecTransformer (2)
- : : : : : +- ^ Scan parquet (1)
+ : : : : : +- ^ ScanTransformer parquet (1)
: : : : +- ^ InputIteratorTransformer (18)
: : : : +- ShuffleQueryStage (16), Statistics(X)
: : : : +- ColumnarExchange (15)
: : : : +- VeloxResizeBatches (14)
: : : : +- ^ ProjectExecTransformer (12)
: : : : +- ^ FilterExecTransformer (11)
- : : : : +- ^ Scan parquet (10)
+ : : : : +- ^ ScanTransformer parquet (10)
: : : +- ^ InputIteratorTransformer (35)
: : : +- ShuffleQueryStage (33), Statistics(X)
: : : +- ColumnarExchange (32)
: : : +- VeloxResizeBatches (31)
: : : +- ^ ProjectExecTransformer (29)
: : : +- ^ FilterExecTransformer (28)
- : : : +- ^ Scan parquet (27)
+ : : : +- ^ ScanTransformer parquet (27)
: : +- ^ InputIteratorTransformer (52)
: : +- ShuffleQueryStage (50), Statistics(X)
: : +- ColumnarExchange (49)
: : +- VeloxResizeBatches (48)
: : +- ^ ProjectExecTransformer (46)
: : +- ^ FilterExecTransformer (45)
- : : +- ^ Scan parquet (44)
+ : : +- ^ ScanTransformer parquet (44)
: +- ^ InputIteratorTransformer (69)
: +- ShuffleQueryStage (67), Statistics(X)
: +- ColumnarExchange (66)
: +- VeloxResizeBatches (65)
: +- ^ ProjectExecTransformer (63)
: +- ^ FilterExecTransformer (62)
- : +- ^ Scan parquet (61)
+ : +- ^ ScanTransformer parquet (61)
+- ^ InputIteratorTransformer (81)
+- ShuffleQueryStage (79), Statistics(X)
+- ReusedExchange (78)
@@ -128,7 +128,7 @@ AdaptiveSparkPlan (149)
+- Scan parquet (138)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [s_suppkey#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -165,7 +165,7 @@ Input [2]: [s_suppkey#X, s_nationkey#X]
(9) InputIteratorTransformer
Input [2]: [s_suppkey#X, s_nationkey#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [5]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -234,7 +234,7 @@ Input [5]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_ship
(26) InputIteratorTransformer
Input [5]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
-(27) Scan parquet
+(27) ScanTransformer parquet
Output [2]: [o_orderkey#X, o_custkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -303,7 +303,7 @@ Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_cust
(43) InputIteratorTransformer
Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X]
-(44) Scan parquet
+(44) ScanTransformer parquet
Output [2]: [c_custkey#X, c_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -372,7 +372,7 @@ Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nati
(60) InputIteratorTransformer
Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X]
-(61) Scan parquet
+(61) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 05f9c2c90e5d..8ce3620f30d5 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
@@ -59,56 +59,56 @@ AdaptiveSparkPlan (207)
: : : : : : : +- VeloxResizeBatches (5)
: : : : : : : +- ^ ProjectExecTransformer (3)
: : : : : : : +- ^ FilterExecTransformer (2)
- : : : : : : : +- ^ Scan parquet (1)
+ : : : : : : : +- ^ ScanTransformer parquet (1)
: : : : : : +- ^ InputIteratorTransformer (18)
: : : : : : +- ShuffleQueryStage (16), Statistics(X)
: : : : : : +- ColumnarExchange (15)
: : : : : : +- VeloxResizeBatches (14)
: : : : : : +- ^ ProjectExecTransformer (12)
: : : : : : +- ^ FilterExecTransformer (11)
- : : : : : : +- ^ Scan parquet (10)
+ : : : : : : +- ^ ScanTransformer parquet (10)
: : : : : +- ^ InputIteratorTransformer (35)
: : : : : +- ShuffleQueryStage (33), Statistics(X)
: : : : : +- ColumnarExchange (32)
: : : : : +- VeloxResizeBatches (31)
: : : : : +- ^ ProjectExecTransformer (29)
: : : : : +- ^ FilterExecTransformer (28)
- : : : : : +- ^ Scan parquet (27)
+ : : : : : +- ^ ScanTransformer parquet (27)
: : : : +- ^ InputIteratorTransformer (52)
: : : : +- ShuffleQueryStage (50), Statistics(X)
: : : : +- ColumnarExchange (49)
: : : : +- VeloxResizeBatches (48)
: : : : +- ^ ProjectExecTransformer (46)
: : : : +- ^ FilterExecTransformer (45)
- : : : : +- ^ Scan parquet (44)
+ : : : : +- ^ ScanTransformer parquet (44)
: : : +- ^ InputIteratorTransformer (69)
: : : +- ShuffleQueryStage (67), Statistics(X)
: : : +- ColumnarExchange (66)
: : : +- VeloxResizeBatches (65)
: : : +- ^ ProjectExecTransformer (63)
: : : +- ^ FilterExecTransformer (62)
- : : : +- ^ Scan parquet (61)
+ : : : +- ^ ScanTransformer parquet (61)
: : +- ^ InputIteratorTransformer (86)
: : +- ShuffleQueryStage (84), Statistics(X)
: : +- ColumnarExchange (83)
: : +- VeloxResizeBatches (82)
: : +- ^ ProjectExecTransformer (80)
: : +- ^ FilterExecTransformer (79)
- : : +- ^ Scan parquet (78)
+ : : +- ^ ScanTransformer parquet (78)
: +- ^ InputIteratorTransformer (103)
: +- ShuffleQueryStage (101), Statistics(X)
: +- ColumnarExchange (100)
: +- VeloxResizeBatches (99)
: +- ^ ProjectExecTransformer (97)
: +- ^ FilterExecTransformer (96)
- : +- ^ Scan parquet (95)
+ : +- ^ ScanTransformer parquet (95)
+- ^ InputIteratorTransformer (120)
+- ShuffleQueryStage (118), Statistics(X)
+- ColumnarExchange (117)
+- VeloxResizeBatches (116)
+- ^ ProjectExecTransformer (114)
+- ^ FilterExecTransformer (113)
- +- ^ Scan parquet (112)
+ +- ^ ScanTransformer parquet (112)
+- == Initial Plan ==
Sort (206)
+- Exchange (205)
@@ -177,7 +177,7 @@ AdaptiveSparkPlan (207)
+- Scan parquet (195)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [p_partkey#X, p_type#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -214,7 +214,7 @@ Input [1]: [p_partkey#X]
(9) InputIteratorTransformer
Input [1]: [p_partkey#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [5]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -283,7 +283,7 @@ Input [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
(26) InputIteratorTransformer
Input [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
-(27) Scan parquet
+(27) ScanTransformer parquet
Output [2]: [s_suppkey#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -352,7 +352,7 @@ Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X]
(43) InputIteratorTransformer
Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X]
-(44) Scan parquet
+(44) ScanTransformer parquet
Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -421,7 +421,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_order
(60) InputIteratorTransformer
Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X]
-(61) Scan parquet
+(61) ScanTransformer parquet
Output [2]: [c_custkey#X, c_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -490,7 +490,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nat
(77) InputIteratorTransformer
Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X]
-(78) Scan parquet
+(78) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_regionkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -559,7 +559,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_reg
(94) InputIteratorTransformer
Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X]
-(95) Scan parquet
+(95) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -628,7 +628,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_nam
(111) InputIteratorTransformer
Input [5]: [l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X]
-(112) Scan parquet
+(112) ScanTransformer parquet
Output [2]: [r_regionkey#X, r_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 e298d2a75196..4f537e91e132 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
@@ -46,42 +46,42 @@ AdaptiveSparkPlan (155)
: : : : : +- VeloxResizeBatches (5)
: : : : : +- ^ ProjectExecTransformer (3)
: : : : : +- ^ FilterExecTransformer (2)
- : : : : : +- ^ Scan parquet (1)
+ : : : : : +- ^ ScanTransformer parquet (1)
: : : : +- ^ InputIteratorTransformer (18)
: : : : +- ShuffleQueryStage (16), Statistics(X)
: : : : +- ColumnarExchange (15)
: : : : +- VeloxResizeBatches (14)
: : : : +- ^ ProjectExecTransformer (12)
: : : : +- ^ FilterExecTransformer (11)
- : : : : +- ^ Scan parquet (10)
+ : : : : +- ^ ScanTransformer parquet (10)
: : : +- ^ InputIteratorTransformer (35)
: : : +- ShuffleQueryStage (33), Statistics(X)
: : : +- ColumnarExchange (32)
: : : +- VeloxResizeBatches (31)
: : : +- ^ ProjectExecTransformer (29)
: : : +- ^ FilterExecTransformer (28)
- : : : +- ^ Scan parquet (27)
+ : : : +- ^ ScanTransformer parquet (27)
: : +- ^ InputIteratorTransformer (52)
: : +- ShuffleQueryStage (50), Statistics(X)
: : +- ColumnarExchange (49)
: : +- VeloxResizeBatches (48)
: : +- ^ ProjectExecTransformer (46)
: : +- ^ FilterExecTransformer (45)
- : : +- ^ Scan parquet (44)
+ : : +- ^ ScanTransformer parquet (44)
: +- ^ InputIteratorTransformer (69)
: +- ShuffleQueryStage (67), Statistics(X)
: +- ColumnarExchange (66)
: +- VeloxResizeBatches (65)
: +- ^ ProjectExecTransformer (63)
: +- ^ FilterExecTransformer (62)
- : +- ^ Scan parquet (61)
+ : +- ^ ScanTransformer parquet (61)
+- ^ InputIteratorTransformer (86)
+- ShuffleQueryStage (84), Statistics(X)
+- ColumnarExchange (83)
+- VeloxResizeBatches (82)
+- ^ ProjectExecTransformer (80)
+- ^ FilterExecTransformer (79)
- +- ^ Scan parquet (78)
+ +- ^ ScanTransformer parquet (78)
+- == Initial Plan ==
Sort (154)
+- Exchange (153)
@@ -133,7 +133,7 @@ AdaptiveSparkPlan (155)
+- Scan parquet (144)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [p_partkey#X, p_name#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -170,7 +170,7 @@ Input [1]: [p_partkey#X]
(9) InputIteratorTransformer
Input [1]: [p_partkey#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -239,7 +239,7 @@ Input [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpric
(26) InputIteratorTransformer
Input [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X]
-(27) Scan parquet
+(27) ScanTransformer parquet
Output [2]: [s_suppkey#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -308,7 +308,7 @@ Input [7]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpric
(43) InputIteratorTransformer
Input [7]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X]
-(44) Scan parquet
+(44) ScanTransformer parquet
Output [3]: [ps_partkey#X, ps_suppkey#X, ps_supplycost#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -377,7 +377,7 @@ Input [6]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_natio
(60) InputIteratorTransformer
Input [6]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X]
-(61) Scan parquet
+(61) ScanTransformer parquet
Output [2]: [o_orderkey#X, o_orderdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -446,7 +446,7 @@ Input [6]: [l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_sup
(77) InputIteratorTransformer
Input [6]: [l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X]
-(78) Scan parquet
+(78) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 efe7f3908506..617489f1aa35 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
@@ -16,7 +16,7 @@ AdaptiveSparkPlan (30)
+- ^ FlushableHashAggregateExecTransformer (4)
+- ^ ProjectExecTransformer (3)
+- ^ FilterExecTransformer (2)
- +- ^ Scan parquet (1)
+ +- ^ ScanTransformer parquet (1)
+- == Initial Plan ==
Sort (29)
+- Exchange (28)
@@ -28,7 +28,7 @@ AdaptiveSparkPlan (30)
+- Scan parquet (22)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [7]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_tax#X, l_returnflag#X, l_linestatus#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
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 a7ced66a82e7..7c8451dfd6dd 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
@@ -31,28 +31,28 @@ AdaptiveSparkPlan (100)
: : : +- VeloxResizeBatches (5)
: : : +- ^ ProjectExecTransformer (3)
: : : +- ^ FilterExecTransformer (2)
- : : : +- ^ Scan parquet (1)
+ : : : +- ^ ScanTransformer parquet (1)
: : +- ^ InputIteratorTransformer (18)
: : +- ShuffleQueryStage (16)
: : +- ColumnarExchange (15)
: : +- VeloxResizeBatches (14)
: : +- ^ ProjectExecTransformer (12)
: : +- ^ FilterExecTransformer (11)
- : : +- ^ Scan parquet (10)
+ : : +- ^ ScanTransformer parquet (10)
: +- ^ InputIteratorTransformer (35)
: +- ShuffleQueryStage (33)
: +- ColumnarExchange (32)
: +- VeloxResizeBatches (31)
: +- ^ ProjectExecTransformer (29)
: +- ^ FilterExecTransformer (28)
- : +- ^ Scan parquet (27)
+ : +- ^ ScanTransformer parquet (27)
+- ^ InputIteratorTransformer (52)
+- ShuffleQueryStage (50)
+- ColumnarExchange (49)
+- VeloxResizeBatches (48)
+- ^ ProjectExecTransformer (46)
+- ^ FilterExecTransformer (45)
- +- ^ Scan parquet (44)
+ +- ^ ScanTransformer parquet (44)
+- == Initial Plan ==
TakeOrderedAndProject (99)
+- HashAggregate (98)
@@ -88,7 +88,7 @@ AdaptiveSparkPlan (100)
+- Scan parquet (90)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -125,7 +125,7 @@ Input [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct
(9) InputIteratorTransformer
Input [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -193,7 +193,7 @@ Input [8]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct
(26) InputIteratorTransformer
Input [8]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, o_orderkey#X]
-(27) Scan parquet
+(27) ScanTransformer parquet
Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_returnflag#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -261,7 +261,7 @@ Input [9]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct
(43) InputIteratorTransformer
Input [9]: [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]
-(44) Scan parquet
+(44) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 b385b7719de2..f943422808e8 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
@@ -29,21 +29,21 @@ AdaptiveSparkPlan (82)
: : +- VeloxResizeBatches (5)
: : +- ^ ProjectExecTransformer (3)
: : +- ^ FilterExecTransformer (2)
- : : +- ^ Scan parquet (1)
+ : : +- ^ ScanTransformer parquet (1)
: +- ^ InputIteratorTransformer (18)
: +- ShuffleQueryStage (16)
: +- ColumnarExchange (15)
: +- VeloxResizeBatches (14)
: +- ^ ProjectExecTransformer (12)
: +- ^ FilterExecTransformer (11)
- : +- ^ Scan parquet (10)
+ : +- ^ ScanTransformer parquet (10)
+- ^ InputIteratorTransformer (35)
+- ShuffleQueryStage (33)
+- ColumnarExchange (32)
+- VeloxResizeBatches (31)
+- ^ ProjectExecTransformer (29)
+- ^ FilterExecTransformer (28)
- +- ^ Scan parquet (27)
+ +- ^ ScanTransformer parquet (27)
+- == Initial Plan ==
Sort (81)
+- Exchange (80)
@@ -72,7 +72,7 @@ AdaptiveSparkPlan (82)
+- Scan parquet (69)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -109,7 +109,7 @@ Input [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X]
(9) InputIteratorTransformer
Input [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [2]: [s_suppkey#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -177,7 +177,7 @@ Input [4]: [ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X]
(26) InputIteratorTransformer
Input [4]: [ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X]
-(27) Scan parquet
+(27) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 71dee80fab86..48e7e4d4e0d5 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
@@ -22,14 +22,14 @@ AdaptiveSparkPlan (55)
: +- VeloxResizeBatches (5)
: +- ^ ProjectExecTransformer (3)
: +- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
+ : +- ^ ScanTransformer parquet (1)
+- ^ InputIteratorTransformer (18)
+- ShuffleQueryStage (16)
+- ColumnarExchange (15)
+- VeloxResizeBatches (14)
+- ^ ProjectExecTransformer (12)
+- ^ FilterExecTransformer (11)
- +- ^ Scan parquet (10)
+ +- ^ ScanTransformer parquet (10)
+- == Initial Plan ==
Sort (54)
+- Exchange (53)
@@ -49,7 +49,7 @@ AdaptiveSparkPlan (55)
+- Scan parquet (43)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [o_orderkey#X, o_orderpriority#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -86,7 +86,7 @@ Input [2]: [o_orderkey#X, o_orderpriority#X]
(9) InputIteratorTransformer
Input [2]: [o_orderkey#X, o_orderpriority#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [5]: [l_orderkey#X, l_shipdate#X, l_commitdate#X, l_receiptdate#X, l_shipmode#X]
Batched: true
Location: InMemoryFileIndex [*]
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 edbcb50206df..c8ddb5437009 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
@@ -1,58 +1,57 @@
== Physical Plan ==
-AdaptiveSparkPlan (58)
+AdaptiveSparkPlan (57)
+- == Final Plan ==
- VeloxColumnarToRow (40)
- +- ^ SortExecTransformer (38)
- +- ^ InputIteratorTransformer (37)
- +- ShuffleQueryStage (35)
- +- ColumnarExchange (34)
- +- VeloxResizeBatches (33)
- +- ^ RegularHashAggregateExecTransformer (31)
- +- ^ InputIteratorTransformer (30)
- +- ShuffleQueryStage (28)
- +- ColumnarExchange (27)
- +- VeloxResizeBatches (26)
- +- ^ ProjectExecTransformer (24)
- +- ^ FlushableHashAggregateExecTransformer (23)
- +- ^ ProjectExecTransformer (22)
- +- ^ RegularHashAggregateExecTransformer (21)
- +- ^ RegularHashAggregateExecTransformer (20)
- +- ^ ProjectExecTransformer (19)
- +- ^ ShuffledHashJoinExecTransformer LeftOuter BuildLeft (18)
- :- ^ InputIteratorTransformer (8)
- : +- ShuffleQueryStage (6)
- : +- ColumnarExchange (5)
- : +- VeloxResizeBatches (4)
- : +- ^ ProjectExecTransformer (2)
- : +- ^ Scan parquet (1)
- +- ^ InputIteratorTransformer (17)
- +- ShuffleQueryStage (15)
- +- ColumnarExchange (14)
- +- VeloxResizeBatches (13)
- +- ^ ProjectExecTransformer (11)
- +- ^ FilterExecTransformer (10)
- +- ^ Scan parquet (9)
+ VeloxColumnarToRow (39)
+ +- ^ SortExecTransformer (37)
+ +- ^ InputIteratorTransformer (36)
+ +- ShuffleQueryStage (34)
+ +- ColumnarExchange (33)
+ +- VeloxResizeBatches (32)
+ +- ^ RegularHashAggregateExecTransformer (30)
+ +- ^ InputIteratorTransformer (29)
+ +- ShuffleQueryStage (27)
+ +- ColumnarExchange (26)
+ +- VeloxResizeBatches (25)
+ +- ^ ProjectExecTransformer (23)
+ +- ^ FlushableHashAggregateExecTransformer (22)
+ +- ^ ProjectExecTransformer (21)
+ +- ^ RegularHashAggregateExecTransformer (20)
+ +- ^ ProjectExecTransformer (19)
+ +- ^ ShuffledHashJoinExecTransformer LeftOuter BuildLeft (18)
+ :- ^ InputIteratorTransformer (8)
+ : +- ShuffleQueryStage (6)
+ : +- ColumnarExchange (5)
+ : +- VeloxResizeBatches (4)
+ : +- ^ ProjectExecTransformer (2)
+ : +- ^ ScanTransformer parquet (1)
+ +- ^ InputIteratorTransformer (17)
+ +- ShuffleQueryStage (15)
+ +- ColumnarExchange (14)
+ +- VeloxResizeBatches (13)
+ +- ^ ProjectExecTransformer (11)
+ +- ^ FilterExecTransformer (10)
+ +- ^ ScanTransformer parquet (9)
+- == Initial Plan ==
- Sort (57)
- +- Exchange (56)
- +- HashAggregate (55)
- +- Exchange (54)
- +- HashAggregate (53)
- +- HashAggregate (52)
- +- HashAggregate (51)
- +- Project (50)
- +- SortMergeJoin LeftOuter (49)
- :- Sort (43)
- : +- Exchange (42)
- : +- Scan parquet (41)
- +- Sort (48)
- +- Exchange (47)
- +- Project (46)
- +- Filter (45)
- +- Scan parquet (44)
-
-
-(1) Scan parquet
+ Sort (56)
+ +- Exchange (55)
+ +- HashAggregate (54)
+ +- Exchange (53)
+ +- HashAggregate (52)
+ +- HashAggregate (51)
+ +- HashAggregate (50)
+ +- Project (49)
+ +- SortMergeJoin LeftOuter (48)
+ :- Sort (42)
+ : +- Exchange (41)
+ : +- Scan parquet (40)
+ +- Sort (47)
+ +- Exchange (46)
+ +- Project (45)
+ +- Filter (44)
+ +- Scan parquet (43)
+
+
+(1) ScanTransformer parquet
Output [1]: [c_custkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -84,7 +83,7 @@ Input [1]: [c_custkey#X]
(8) InputIteratorTransformer
Input [1]: [c_custkey#X]
-(9) Scan parquet
+(9) ScanTransformer parquet
Output [3]: [o_orderkey#X, o_custkey#X, o_comment#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -133,180 +132,173 @@ Input [3]: [c_custkey#X, o_orderkey#X, o_custkey#X]
(20) RegularHashAggregateExecTransformer
Input [2]: [c_custkey#X, o_orderkey#X]
Keys [1]: [c_custkey#X]
-Functions [1]: [partial_count(o_orderkey#X)]
-Aggregate Attributes [1]: [count#X]
-Results [2]: [c_custkey#X, count#X]
-
-(21) RegularHashAggregateExecTransformer
-Input [2]: [c_custkey#X, count#X]
-Keys [1]: [c_custkey#X]
Functions [1]: [count(o_orderkey#X)]
Aggregate Attributes [1]: [count(o_orderkey#X)#X]
Results [2]: [c_custkey#X, count(o_orderkey#X)#X]
-(22) ProjectExecTransformer
+(21) ProjectExecTransformer
Output [1]: [count(o_orderkey#X)#X AS c_count#X]
Input [2]: [c_custkey#X, count(o_orderkey#X)#X]
-(23) FlushableHashAggregateExecTransformer
+(22) FlushableHashAggregateExecTransformer
Input [1]: [c_count#X]
Keys [1]: [c_count#X]
Functions [1]: [partial_count(1)]
Aggregate Attributes [1]: [count#X]
Results [2]: [c_count#X, count#X]
-(24) ProjectExecTransformer
+(23) ProjectExecTransformer
Output [3]: [hash(c_count#X, 42) AS hash_partition_key#X, c_count#X, count#X]
Input [2]: [c_count#X, count#X]
-(25) WholeStageCodegenTransformer (X)
+(24) WholeStageCodegenTransformer (X)
Input [3]: [hash_partition_key#X, c_count#X, count#X]
Arguments: false
-(26) VeloxResizeBatches
+(25) VeloxResizeBatches
Input [3]: [hash_partition_key#X, c_count#X, count#X]
Arguments: X, X
-(27) ColumnarExchange
+(26) ColumnarExchange
Input [3]: [hash_partition_key#X, c_count#X, count#X]
Arguments: hashpartitioning(c_count#X, 1), ENSURE_REQUIREMENTS, [c_count#X, count#X], [plan_id=X], [shuffle_writer_type=hash]
-(28) ShuffleQueryStage
+(27) ShuffleQueryStage
Output [2]: [c_count#X, count#X]
Arguments: X
-(29) InputAdapter
+(28) InputAdapter
Input [2]: [c_count#X, count#X]
-(30) InputIteratorTransformer
+(29) InputIteratorTransformer
Input [2]: [c_count#X, count#X]
-(31) RegularHashAggregateExecTransformer
+(30) RegularHashAggregateExecTransformer
Input [2]: [c_count#X, count#X]
Keys [1]: [c_count#X]
Functions [1]: [count(1)]
Aggregate Attributes [1]: [count(1)#X]
Results [2]: [c_count#X, count(1)#X AS custdist#X]
-(32) WholeStageCodegenTransformer (X)
+(31) WholeStageCodegenTransformer (X)
Input [2]: [c_count#X, custdist#X]
Arguments: false
-(33) VeloxResizeBatches
+(32) VeloxResizeBatches
Input [2]: [c_count#X, custdist#X]
Arguments: X, X
-(34) ColumnarExchange
+(33) ColumnarExchange
Input [2]: [c_count#X, custdist#X]
Arguments: rangepartitioning(custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST, 1), ENSURE_REQUIREMENTS, [plan_id=X], [shuffle_writer_type=hash]
-(35) ShuffleQueryStage
+(34) ShuffleQueryStage
Output [2]: [c_count#X, custdist#X]
Arguments: X
-(36) InputAdapter
+(35) InputAdapter
Input [2]: [c_count#X, custdist#X]
-(37) InputIteratorTransformer
+(36) InputIteratorTransformer
Input [2]: [c_count#X, custdist#X]
-(38) SortExecTransformer
+(37) SortExecTransformer
Input [2]: [c_count#X, custdist#X]
Arguments: [custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST], true, 0
-(39) WholeStageCodegenTransformer (X)
+(38) WholeStageCodegenTransformer (X)
Input [2]: [c_count#X, custdist#X]
Arguments: false
-(40) VeloxColumnarToRow
+(39) VeloxColumnarToRow
Input [2]: [c_count#X, custdist#X]
-(41) Scan parquet
+(40) Scan parquet
Output [1]: [c_custkey#X]
Batched: true
Location: InMemoryFileIndex [*]
ReadSchema: struct
-(42) Exchange
+(41) Exchange
Input [1]: [c_custkey#X]
Arguments: hashpartitioning(c_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(43) Sort
+(42) Sort
Input [1]: [c_custkey#X]
Arguments: [c_custkey#X ASC NULLS FIRST], false, 0
-(44) Scan parquet
+(43) Scan parquet
Output [3]: [o_orderkey#X, o_custkey#X, o_comment#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(o_comment), IsNotNull(o_custkey)]
ReadSchema: struct
-(45) Filter
+(44) Filter
Input [3]: [o_orderkey#X, o_custkey#X, o_comment#X]
Condition : ((isnotnull(o_comment#X) AND NOT o_comment#X LIKE %special%requests%) AND isnotnull(o_custkey#X))
-(46) Project
+(45) Project
Output [2]: [o_orderkey#X, o_custkey#X]
Input [3]: [o_orderkey#X, o_custkey#X, o_comment#X]
-(47) Exchange
+(46) Exchange
Input [2]: [o_orderkey#X, o_custkey#X]
Arguments: hashpartitioning(o_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(48) Sort
+(47) Sort
Input [2]: [o_orderkey#X, o_custkey#X]
Arguments: [o_custkey#X ASC NULLS FIRST], false, 0
-(49) SortMergeJoin
+(48) SortMergeJoin
Left keys [1]: [c_custkey#X]
Right keys [1]: [o_custkey#X]
Join condition: None
-(50) Project
+(49) Project
Output [2]: [c_custkey#X, o_orderkey#X]
Input [3]: [c_custkey#X, o_orderkey#X, o_custkey#X]
-(51) HashAggregate
+(50) HashAggregate
Input [2]: [c_custkey#X, o_orderkey#X]
Keys [1]: [c_custkey#X]
Functions [1]: [partial_count(o_orderkey#X)]
Aggregate Attributes [1]: [count#X]
Results [2]: [c_custkey#X, count#X]
-(52) HashAggregate
+(51) HashAggregate
Input [2]: [c_custkey#X, count#X]
Keys [1]: [c_custkey#X]
Functions [1]: [count(o_orderkey#X)]
Aggregate Attributes [1]: [count(o_orderkey#X)#X]
Results [1]: [count(o_orderkey#X)#X AS c_count#X]
-(53) HashAggregate
+(52) HashAggregate
Input [1]: [c_count#X]
Keys [1]: [c_count#X]
Functions [1]: [partial_count(1)]
Aggregate Attributes [1]: [count#X]
Results [2]: [c_count#X, count#X]
-(54) Exchange
+(53) Exchange
Input [2]: [c_count#X, count#X]
Arguments: hashpartitioning(c_count#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(55) HashAggregate
+(54) HashAggregate
Input [2]: [c_count#X, count#X]
Keys [1]: [c_count#X]
Functions [1]: [count(1)]
Aggregate Attributes [1]: [count(1)#X]
Results [2]: [c_count#X, count(1)#X AS custdist#X]
-(56) Exchange
+(55) Exchange
Input [2]: [c_count#X, custdist#X]
Arguments: rangepartitioning(custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(57) Sort
+(56) Sort
Input [2]: [c_count#X, custdist#X]
Arguments: [custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST], true, 0
-(58) AdaptiveSparkPlan
+(57) AdaptiveSparkPlan
Output [2]: [c_count#X, custdist#X]
Arguments: isFinalPlan=true
\ No newline at end of file
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 91df642595a3..40a891738143 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
@@ -1,43 +1,42 @@
== Physical Plan ==
-AdaptiveSparkPlan (39)
+AdaptiveSparkPlan (38)
+- == Final Plan ==
- VeloxColumnarToRow (25)
- +- ^ ProjectExecTransformer (23)
- +- ^ RegularHashAggregateExecTransformer (22)
- +- ^ RegularHashAggregateExecTransformer (21)
- +- ^ ProjectExecTransformer (20)
- +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19)
- :- ^ InputIteratorTransformer (9)
- : +- ShuffleQueryStage (7)
- : +- ColumnarExchange (6)
- : +- VeloxResizeBatches (5)
- : +- ^ ProjectExecTransformer (3)
- : +- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
- +- ^ InputIteratorTransformer (18)
- +- ShuffleQueryStage (16)
- +- ColumnarExchange (15)
- +- VeloxResizeBatches (14)
- +- ^ ProjectExecTransformer (12)
- +- ^ FilterExecTransformer (11)
- +- ^ Scan parquet (10)
+ VeloxColumnarToRow (24)
+ +- ^ ProjectExecTransformer (22)
+ +- ^ RegularHashAggregateExecTransformer (21)
+ +- ^ ProjectExecTransformer (20)
+ +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19)
+ :- ^ InputIteratorTransformer (9)
+ : +- ShuffleQueryStage (7)
+ : +- ColumnarExchange (6)
+ : +- VeloxResizeBatches (5)
+ : +- ^ ProjectExecTransformer (3)
+ : +- ^ FilterExecTransformer (2)
+ : +- ^ ScanTransformer parquet (1)
+ +- ^ InputIteratorTransformer (18)
+ +- ShuffleQueryStage (16)
+ +- ColumnarExchange (15)
+ +- VeloxResizeBatches (14)
+ +- ^ ProjectExecTransformer (12)
+ +- ^ FilterExecTransformer (11)
+ +- ^ ScanTransformer parquet (10)
+- == Initial Plan ==
- HashAggregate (38)
- +- HashAggregate (37)
- +- Project (36)
- +- SortMergeJoin Inner (35)
- :- Sort (30)
- : +- Exchange (29)
- : +- Project (28)
- : +- Filter (27)
- : +- Scan parquet (26)
- +- Sort (34)
- +- Exchange (33)
- +- Filter (32)
- +- Scan parquet (31)
-
-
-(1) Scan parquet
+ HashAggregate (37)
+ +- HashAggregate (36)
+ +- Project (35)
+ +- SortMergeJoin Inner (34)
+ :- Sort (29)
+ : +- Exchange (28)
+ : +- Project (27)
+ : +- Filter (26)
+ : +- Scan parquet (25)
+ +- Sort (33)
+ +- Exchange (32)
+ +- Filter (31)
+ +- Scan parquet (30)
+
+
+(1) ScanTransformer parquet
Output [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -74,7 +73,7 @@ Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X]
(9) InputIteratorTransformer
Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [2]: [p_partkey#X, p_type#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -123,93 +122,86 @@ Input [5]: [l_partkey#X, l_extendedprice#X, l_discount#X, p_partkey#X, p_type#X]
(21) RegularHashAggregateExecTransformer
Input [5]: [l_extendedprice#X, l_discount#X, p_type#X, _pre_X#X, _pre_X#X]
Keys: []
-Functions [2]: [partial_sum(_pre_X#X), partial_sum(_pre_X#X)]
-Aggregate Attributes [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X]
-Results [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X]
-
-(22) RegularHashAggregateExecTransformer
-Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X]
-Keys: []
-Functions [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true) ELSE 0.0000 END), sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))]
+Functions [2]: [sum(_pre_X#X), sum(_pre_X#X)]
Aggregate Attributes [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true) ELSE 0.0000 END)#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X]
Results [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true) ELSE 0.0000 END)#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X]
-(23) ProjectExecTransformer
+(22) ProjectExecTransformer
Output [1]: [CheckOverflow((promote_precision(CheckOverflow((100.0000 * promote_precision(sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true) ELSE 0.0000 END)#X)), DecimalType(38,6), true)) / promote_precision(cast(sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X as decimal(38,6)))), DecimalType(38,6), true) AS promo_revenue#X]
Input [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true) ELSE 0.0000 END)#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X]
-(24) WholeStageCodegenTransformer (X)
+(23) WholeStageCodegenTransformer (X)
Input [1]: [promo_revenue#X]
Arguments: false
-(25) VeloxColumnarToRow
+(24) VeloxColumnarToRow
Input [1]: [promo_revenue#X]
-(26) Scan parquet
+(25) Scan parquet
Output [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1995-09-01), LessThan(l_shipdate,1995-10-01), IsNotNull(l_partkey)]
ReadSchema: struct
-(27) Filter
+(26) Filter
Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Condition : (((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1995-09-01)) AND (l_shipdate#X < 1995-10-01)) AND isnotnull(l_partkey#X))
-(28) Project
+(27) Project
Output [3]: [l_partkey#X, l_extendedprice#X, l_discount#X]
Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
-(29) Exchange
+(28) Exchange
Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X]
Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(30) Sort
+(29) Sort
Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X]
Arguments: [l_partkey#X ASC NULLS FIRST], false, 0
-(31) Scan parquet
+(30) Scan parquet
Output [2]: [p_partkey#X, p_type#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(p_partkey)]
ReadSchema: struct
-(32) Filter
+(31) Filter
Input [2]: [p_partkey#X, p_type#X]
Condition : isnotnull(p_partkey#X)
-(33) Exchange
+(32) Exchange
Input [2]: [p_partkey#X, p_type#X]
Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(34) Sort
+(33) Sort
Input [2]: [p_partkey#X, p_type#X]
Arguments: [p_partkey#X ASC NULLS FIRST], false, 0
-(35) SortMergeJoin
+(34) SortMergeJoin
Left keys [1]: [l_partkey#X]
Right keys [1]: [p_partkey#X]
Join condition: None
-(36) Project
+(35) Project
Output [3]: [l_extendedprice#X, l_discount#X, p_type#X]
Input [5]: [l_partkey#X, l_extendedprice#X, l_discount#X, p_partkey#X, p_type#X]
-(37) HashAggregate
+(36) HashAggregate
Input [3]: [l_extendedprice#X, l_discount#X, p_type#X]
Keys: []
Functions [2]: [partial_sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true) ELSE 0.0000 END), partial_sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))]
Aggregate Attributes [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X]
Results [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X]
-(38) HashAggregate
+(37) HashAggregate
Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X]
Keys: []
Functions [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true) ELSE 0.0000 END), sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))]
Aggregate Attributes [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true) ELSE 0.0000 END)#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X]
Results [1]: [CheckOverflow((promote_precision(CheckOverflow((100.0000 * promote_precision(sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true) ELSE 0.0000 END)#X)), DecimalType(38,6), true)) / promote_precision(cast(sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X as decimal(38,6)))), DecimalType(38,6), true) AS promo_revenue#X]
-(39) AdaptiveSparkPlan
+(38) AdaptiveSparkPlan
Output [1]: [promo_revenue#X]
Arguments: isFinalPlan=true
\ No newline at end of file
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 56169a244bf9..9317c2f895a5 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
@@ -15,7 +15,7 @@ AdaptiveSparkPlan (50)
: +- VeloxResizeBatches (5)
: +- ^ ProjectExecTransformer (3)
: +- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
+ : +- ^ ScanTransformer parquet (1)
+- ^ FilterExecTransformer (22)
+- ^ RegularHashAggregateExecTransformer (21)
+- ^ InputIteratorTransformer (20)
@@ -26,7 +26,7 @@ AdaptiveSparkPlan (50)
+- ^ FlushableHashAggregateExecTransformer (13)
+- ^ ProjectExecTransformer (12)
+- ^ FilterExecTransformer (11)
- +- ^ Scan parquet (10)
+ +- ^ ScanTransformer parquet (10)
+- == Initial Plan ==
Sort (49)
+- Exchange (48)
@@ -46,7 +46,7 @@ AdaptiveSparkPlan (50)
+- Scan parquet (38)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -83,7 +83,7 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X]
(9) InputIteratorTransformer
Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
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 8ef7d4d51441..2c41822beeeb 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
@@ -29,14 +29,14 @@ AdaptiveSparkPlan (71)
: +- VeloxResizeBatches (5)
: +- ^ ProjectExecTransformer (3)
: +- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
+ : +- ^ ScanTransformer parquet (1)
+- ^ InputIteratorTransformer (18)
+- ShuffleQueryStage (16)
+- ColumnarExchange (15)
+- VeloxResizeBatches (14)
+- ^ ProjectExecTransformer (12)
+- ^ FilterExecTransformer (11)
- +- ^ Scan parquet (10)
+ +- ^ ScanTransformer parquet (10)
+- == Initial Plan ==
Sort (70)
+- Exchange (69)
@@ -63,7 +63,7 @@ AdaptiveSparkPlan (71)
+- Scan parquet (57)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [ps_partkey#X, ps_suppkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -100,7 +100,7 @@ Input [2]: [ps_partkey#X, ps_suppkey#X]
(9) InputIteratorTransformer
Input [2]: [ps_partkey#X, ps_suppkey#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [4]: [p_partkey#X, p_brand#X, p_type#X, p_size#X]
Batched: true
Location: InMemoryFileIndex [*]
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 fe915989f9e6..30fd4b0b1124 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
@@ -1,65 +1,64 @@
== Physical Plan ==
-AdaptiveSparkPlan (63)
+AdaptiveSparkPlan (62)
+- == Final Plan ==
- VeloxColumnarToRow (40)
- +- ^ ProjectExecTransformer (38)
- +- ^ RegularHashAggregateExecTransformer (37)
- +- ^ RegularHashAggregateExecTransformer (36)
- +- ^ ProjectExecTransformer (35)
- +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (34)
- :- ^ ProjectExecTransformer (20)
- : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19)
- : :- ^ InputIteratorTransformer (9)
- : : +- ShuffleQueryStage (7)
- : : +- ColumnarExchange (6)
- : : +- VeloxResizeBatches (5)
- : : +- ^ ProjectExecTransformer (3)
- : : +- ^ FilterExecTransformer (2)
- : : +- ^ Scan parquet (1)
- : +- ^ InputIteratorTransformer (18)
- : +- ShuffleQueryStage (16)
- : +- ColumnarExchange (15)
- : +- VeloxResizeBatches (14)
- : +- ^ ProjectExecTransformer (12)
- : +- ^ FilterExecTransformer (11)
- : +- ^ Scan parquet (10)
- +- ^ FilterExecTransformer (33)
- +- ^ ProjectExecTransformer (32)
- +- ^ RegularHashAggregateExecTransformer (31)
- +- ^ InputIteratorTransformer (30)
- +- ShuffleQueryStage (28)
- +- ColumnarExchange (27)
- +- VeloxResizeBatches (26)
- +- ^ ProjectExecTransformer (24)
- +- ^ FlushableHashAggregateExecTransformer (23)
- +- ^ FilterExecTransformer (22)
- +- ^ Scan parquet (21)
+ VeloxColumnarToRow (39)
+ +- ^ ProjectExecTransformer (37)
+ +- ^ RegularHashAggregateExecTransformer (36)
+ +- ^ ProjectExecTransformer (35)
+ +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (34)
+ :- ^ ProjectExecTransformer (20)
+ : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19)
+ : :- ^ InputIteratorTransformer (9)
+ : : +- ShuffleQueryStage (7)
+ : : +- ColumnarExchange (6)
+ : : +- VeloxResizeBatches (5)
+ : : +- ^ ProjectExecTransformer (3)
+ : : +- ^ FilterExecTransformer (2)
+ : : +- ^ ScanTransformer parquet (1)
+ : +- ^ InputIteratorTransformer (18)
+ : +- ShuffleQueryStage (16)
+ : +- ColumnarExchange (15)
+ : +- VeloxResizeBatches (14)
+ : +- ^ ProjectExecTransformer (12)
+ : +- ^ FilterExecTransformer (11)
+ : +- ^ ScanTransformer parquet (10)
+ +- ^ FilterExecTransformer (33)
+ +- ^ ProjectExecTransformer (32)
+ +- ^ RegularHashAggregateExecTransformer (31)
+ +- ^ InputIteratorTransformer (30)
+ +- ShuffleQueryStage (28)
+ +- ColumnarExchange (27)
+ +- VeloxResizeBatches (26)
+ +- ^ ProjectExecTransformer (24)
+ +- ^ FlushableHashAggregateExecTransformer (23)
+ +- ^ FilterExecTransformer (22)
+ +- ^ ScanTransformer parquet (21)
+- == Initial Plan ==
- HashAggregate (62)
- +- HashAggregate (61)
- +- Project (60)
- +- SortMergeJoin Inner (59)
- :- Project (51)
- : +- SortMergeJoin Inner (50)
- : :- Sort (44)
- : : +- Exchange (43)
- : : +- Filter (42)
- : : +- Scan parquet (41)
- : +- Sort (49)
- : +- Exchange (48)
- : +- Project (47)
- : +- Filter (46)
- : +- Scan parquet (45)
- +- Sort (58)
- +- Filter (57)
- +- HashAggregate (56)
- +- Exchange (55)
- +- HashAggregate (54)
- +- Filter (53)
- +- Scan parquet (52)
-
-
-(1) Scan parquet
+ HashAggregate (61)
+ +- HashAggregate (60)
+ +- Project (59)
+ +- SortMergeJoin Inner (58)
+ :- Project (50)
+ : +- SortMergeJoin Inner (49)
+ : :- Sort (43)
+ : : +- Exchange (42)
+ : : +- Filter (41)
+ : : +- Scan parquet (40)
+ : +- Sort (48)
+ : +- Exchange (47)
+ : +- Project (46)
+ : +- Filter (45)
+ : +- Scan parquet (44)
+ +- Sort (57)
+ +- Filter (56)
+ +- HashAggregate (55)
+ +- Exchange (54)
+ +- HashAggregate (53)
+ +- Filter (52)
+ +- Scan parquet (51)
+
+
+(1) ScanTransformer parquet
Output [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -96,7 +95,7 @@ Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X]
(9) InputIteratorTransformer
Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [3]: [p_partkey#X, p_brand#X, p_container#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -142,7 +141,7 @@ Join condition: None
Output [3]: [l_quantity#X, l_extendedprice#X, p_partkey#X]
Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, p_partkey#X]
-(21) Scan parquet
+(21) ScanTransformer parquet
Output [2]: [l_partkey#X, l_quantity#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -213,139 +212,132 @@ Input [5]: [l_quantity#X, l_extendedprice#X, p_partkey#X, (0.2 * avg(l_quantity)
(36) RegularHashAggregateExecTransformer
Input [1]: [l_extendedprice#X]
Keys: []
-Functions [1]: [partial_sum(l_extendedprice#X)]
-Aggregate Attributes [2]: [sum#X, isEmpty#X]
-Results [2]: [sum#X, isEmpty#X]
-
-(37) RegularHashAggregateExecTransformer
-Input [2]: [sum#X, isEmpty#X]
-Keys: []
Functions [1]: [sum(l_extendedprice#X)]
Aggregate Attributes [1]: [sum(l_extendedprice#X)#X]
Results [1]: [sum(l_extendedprice#X)#X]
-(38) ProjectExecTransformer
+(37) ProjectExecTransformer
Output [1]: [CheckOverflow((promote_precision(sum(l_extendedprice#X)#X) / 7.00), DecimalType(27,6), true) AS avg_yearly#X]
Input [1]: [sum(l_extendedprice#X)#X]
-(39) WholeStageCodegenTransformer (X)
+(38) WholeStageCodegenTransformer (X)
Input [1]: [avg_yearly#X]
Arguments: false
-(40) VeloxColumnarToRow
+(39) VeloxColumnarToRow
Input [1]: [avg_yearly#X]
-(41) Scan parquet
+(40) Scan parquet
Output [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(l_partkey), IsNotNull(l_quantity)]
ReadSchema: struct
-(42) Filter
+(41) Filter
Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X]
Condition : (isnotnull(l_partkey#X) AND isnotnull(l_quantity#X))
-(43) Exchange
+(42) Exchange
Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X]
Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(44) Sort
+(43) Sort
Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X]
Arguments: [l_partkey#X ASC NULLS FIRST], false, 0
-(45) Scan parquet
+(44) Scan parquet
Output [3]: [p_partkey#X, p_brand#X, p_container#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(p_brand), IsNotNull(p_container), EqualTo(p_brand,Brand#X), EqualTo(p_container,MED BOX), IsNotNull(p_partkey)]
ReadSchema: struct
-(46) Filter
+(45) Filter
Input [3]: [p_partkey#X, p_brand#X, p_container#X]
Condition : ((((isnotnull(p_brand#X) AND isnotnull(p_container#X)) AND (p_brand#X = Brand#X)) AND (p_container#X = MED BOX)) AND isnotnull(p_partkey#X))
-(47) Project
+(46) Project
Output [1]: [p_partkey#X]
Input [3]: [p_partkey#X, p_brand#X, p_container#X]
-(48) Exchange
+(47) Exchange
Input [1]: [p_partkey#X]
Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(49) Sort
+(48) Sort
Input [1]: [p_partkey#X]
Arguments: [p_partkey#X ASC NULLS FIRST], false, 0
-(50) SortMergeJoin
+(49) SortMergeJoin
Left keys [1]: [l_partkey#X]
Right keys [1]: [p_partkey#X]
Join condition: None
-(51) Project
+(50) Project
Output [3]: [l_quantity#X, l_extendedprice#X, p_partkey#X]
Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, p_partkey#X]
-(52) Scan parquet
+(51) Scan parquet
Output [2]: [l_partkey#X, l_quantity#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(l_partkey)]
ReadSchema: struct
-(53) Filter
+(52) Filter
Input [2]: [l_partkey#X, l_quantity#X]
Condition : isnotnull(l_partkey#X)
-(54) HashAggregate
+(53) HashAggregate
Input [2]: [l_partkey#X, l_quantity#X]
Keys [1]: [l_partkey#X]
Functions [1]: [partial_avg(l_quantity#X)]
Aggregate Attributes [2]: [sum#X, count#X]
Results [3]: [l_partkey#X, sum#X, count#X]
-(55) Exchange
+(54) Exchange
Input [3]: [l_partkey#X, sum#X, count#X]
Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(56) HashAggregate
+(55) HashAggregate
Input [3]: [l_partkey#X, sum#X, count#X]
Keys [1]: [l_partkey#X]
Functions [1]: [avg(l_quantity#X)]
Aggregate Attributes [1]: [avg(l_quantity#X)#X]
Results [2]: [CheckOverflow((0.200000 * promote_precision(avg(l_quantity#X)#X)), DecimalType(18,7), true) AS (0.2 * avg(l_quantity))#X, l_partkey#X]
-(57) Filter
+(56) Filter
Input [2]: [(0.2 * avg(l_quantity))#X, l_partkey#X]
Condition : isnotnull((0.2 * avg(l_quantity))#X)
-(58) Sort
+(57) Sort
Input [2]: [(0.2 * avg(l_quantity))#X, l_partkey#X]
Arguments: [l_partkey#X ASC NULLS FIRST], false, 0
-(59) SortMergeJoin
+(58) SortMergeJoin
Left keys [1]: [p_partkey#X]
Right keys [1]: [l_partkey#X]
Join condition: (cast(l_quantity#X as decimal(18,7)) < (0.2 * avg(l_quantity))#X)
-(60) Project
+(59) Project
Output [1]: [l_extendedprice#X]
Input [5]: [l_quantity#X, l_extendedprice#X, p_partkey#X, (0.2 * avg(l_quantity))#X, l_partkey#X]
-(61) HashAggregate
+(60) HashAggregate
Input [1]: [l_extendedprice#X]
Keys: []
Functions [1]: [partial_sum(l_extendedprice#X)]
Aggregate Attributes [2]: [sum#X, isEmpty#X]
Results [2]: [sum#X, isEmpty#X]
-(62) HashAggregate
+(61) HashAggregate
Input [2]: [sum#X, isEmpty#X]
Keys: []
Functions [1]: [sum(l_extendedprice#X)]
Aggregate Attributes [1]: [sum(l_extendedprice#X)#X]
Results [1]: [CheckOverflow((promote_precision(sum(l_extendedprice#X)#X) / 7.00), DecimalType(27,6), true) AS avg_yearly#X]
-(63) AdaptiveSparkPlan
+(62) AdaptiveSparkPlan
Output [1]: [avg_yearly#X]
Arguments: isFinalPlan=true
\ No newline at end of file
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 a2345ca9ed35..dd4cad538e2c 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
@@ -1,105 +1,104 @@
== Physical Plan ==
-AdaptiveSparkPlan (110)
+AdaptiveSparkPlan (109)
+- == Final Plan ==
- VeloxColumnarToRow (70)
- +- TakeOrderedAndProjectExecTransformer (69)
- +- ^ RegularHashAggregateExecTransformer (67)
- +- ^ RegularHashAggregateExecTransformer (66)
- +- ^ ProjectExecTransformer (65)
- +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (64)
- :- ^ InputIteratorTransformer (46)
- : +- ShuffleQueryStage (44)
- : +- ColumnarExchange (43)
- : +- VeloxResizeBatches (42)
- : +- ^ ProjectExecTransformer (40)
- : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (39)
- : :- ^ InputIteratorTransformer (9)
- : : +- ShuffleQueryStage (7)
- : : +- ColumnarExchange (6)
- : : +- VeloxResizeBatches (5)
- : : +- ^ ProjectExecTransformer (3)
- : : +- ^ FilterExecTransformer (2)
- : : +- ^ Scan parquet (1)
- : +- ^ InputIteratorTransformer (38)
- : +- ShuffleQueryStage (36)
- : +- ColumnarExchange (35)
- : +- VeloxResizeBatches (34)
- : +- ^ ProjectExecTransformer (32)
- : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (31)
- : :- ^ InputIteratorTransformer (18)
- : : +- ShuffleQueryStage (16)
- : : +- ColumnarExchange (15)
- : : +- VeloxResizeBatches (14)
- : : +- ^ ProjectExecTransformer (12)
- : : +- ^ FilterExecTransformer (11)
- : : +- ^ Scan parquet (10)
- : +- ^ ProjectExecTransformer (30)
- : +- ^ FilterExecTransformer (29)
- : +- ^ RegularHashAggregateExecTransformer (28)
- : +- ^ InputIteratorTransformer (27)
- : +- ShuffleQueryStage (25)
- : +- ColumnarExchange (24)
- : +- VeloxResizeBatches (23)
- : +- ^ ProjectExecTransformer (21)
- : +- ^ FlushableHashAggregateExecTransformer (20)
- : +- ^ Scan parquet (19)
- +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (63)
- :- ^ InputIteratorTransformer (55)
- : +- ShuffleQueryStage (53)
- : +- ColumnarExchange (52)
- : +- VeloxResizeBatches (51)
- : +- ^ ProjectExecTransformer (49)
- : +- ^ FilterExecTransformer (48)
- : +- ^ Scan parquet (47)
- +- ^ ProjectExecTransformer (62)
- +- ^ FilterExecTransformer (61)
- +- ^ RegularHashAggregateExecTransformer (60)
- +- ^ InputIteratorTransformer (59)
- +- ShuffleQueryStage (57)
- +- ReusedExchange (56)
+ VeloxColumnarToRow (69)
+ +- TakeOrderedAndProjectExecTransformer (68)
+ +- ^ RegularHashAggregateExecTransformer (66)
+ +- ^ ProjectExecTransformer (65)
+ +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (64)
+ :- ^ InputIteratorTransformer (46)
+ : +- ShuffleQueryStage (44)
+ : +- ColumnarExchange (43)
+ : +- VeloxResizeBatches (42)
+ : +- ^ ProjectExecTransformer (40)
+ : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (39)
+ : :- ^ InputIteratorTransformer (9)
+ : : +- ShuffleQueryStage (7)
+ : : +- ColumnarExchange (6)
+ : : +- VeloxResizeBatches (5)
+ : : +- ^ ProjectExecTransformer (3)
+ : : +- ^ FilterExecTransformer (2)
+ : : +- ^ ScanTransformer parquet (1)
+ : +- ^ InputIteratorTransformer (38)
+ : +- ShuffleQueryStage (36)
+ : +- ColumnarExchange (35)
+ : +- VeloxResizeBatches (34)
+ : +- ^ ProjectExecTransformer (32)
+ : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (31)
+ : :- ^ InputIteratorTransformer (18)
+ : : +- ShuffleQueryStage (16)
+ : : +- ColumnarExchange (15)
+ : : +- VeloxResizeBatches (14)
+ : : +- ^ ProjectExecTransformer (12)
+ : : +- ^ FilterExecTransformer (11)
+ : : +- ^ ScanTransformer parquet (10)
+ : +- ^ ProjectExecTransformer (30)
+ : +- ^ FilterExecTransformer (29)
+ : +- ^ RegularHashAggregateExecTransformer (28)
+ : +- ^ InputIteratorTransformer (27)
+ : +- ShuffleQueryStage (25)
+ : +- ColumnarExchange (24)
+ : +- VeloxResizeBatches (23)
+ : +- ^ ProjectExecTransformer (21)
+ : +- ^ FlushableHashAggregateExecTransformer (20)
+ : +- ^ ScanTransformer parquet (19)
+ +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (63)
+ :- ^ InputIteratorTransformer (55)
+ : +- ShuffleQueryStage (53)
+ : +- ColumnarExchange (52)
+ : +- VeloxResizeBatches (51)
+ : +- ^ ProjectExecTransformer (49)
+ : +- ^ FilterExecTransformer (48)
+ : +- ^ ScanTransformer parquet (47)
+ +- ^ ProjectExecTransformer (62)
+ +- ^ FilterExecTransformer (61)
+ +- ^ RegularHashAggregateExecTransformer (60)
+ +- ^ InputIteratorTransformer (59)
+ +- ShuffleQueryStage (57)
+ +- ReusedExchange (56)
+- == Initial Plan ==
- TakeOrderedAndProject (109)
- +- HashAggregate (108)
- +- HashAggregate (107)
- +- Project (106)
- +- SortMergeJoin Inner (105)
- :- Sort (92)
- : +- Exchange (91)
- : +- Project (90)
- : +- SortMergeJoin Inner (89)
- : :- Sort (74)
- : : +- Exchange (73)
- : : +- Filter (72)
- : : +- Scan parquet (71)
- : +- Sort (88)
- : +- Exchange (87)
- : +- SortMergeJoin LeftSemi (86)
- : :- Sort (78)
- : : +- Exchange (77)
- : : +- Filter (76)
- : : +- Scan parquet (75)
- : +- Sort (85)
- : +- Project (84)
- : +- Filter (83)
- : +- HashAggregate (82)
- : +- Exchange (81)
- : +- HashAggregate (80)
- : +- Scan parquet (79)
- +- SortMergeJoin LeftSemi (104)
- :- Sort (96)
- : +- Exchange (95)
- : +- Filter (94)
- : +- Scan parquet (93)
- +- Sort (103)
- +- Project (102)
- +- Filter (101)
- +- HashAggregate (100)
- +- Exchange (99)
- +- HashAggregate (98)
- +- Scan parquet (97)
-
-
-(1) Scan parquet
+ TakeOrderedAndProject (108)
+ +- HashAggregate (107)
+ +- HashAggregate (106)
+ +- Project (105)
+ +- SortMergeJoin Inner (104)
+ :- Sort (91)
+ : +- Exchange (90)
+ : +- Project (89)
+ : +- SortMergeJoin Inner (88)
+ : :- Sort (73)
+ : : +- Exchange (72)
+ : : +- Filter (71)
+ : : +- Scan parquet (70)
+ : +- Sort (87)
+ : +- Exchange (86)
+ : +- SortMergeJoin LeftSemi (85)
+ : :- Sort (77)
+ : : +- Exchange (76)
+ : : +- Filter (75)
+ : : +- Scan parquet (74)
+ : +- Sort (84)
+ : +- Project (83)
+ : +- Filter (82)
+ : +- HashAggregate (81)
+ : +- Exchange (80)
+ : +- HashAggregate (79)
+ : +- Scan parquet (78)
+ +- SortMergeJoin LeftSemi (103)
+ :- Sort (95)
+ : +- Exchange (94)
+ : +- Filter (93)
+ : +- Scan parquet (92)
+ +- Sort (102)
+ +- Project (101)
+ +- Filter (100)
+ +- HashAggregate (99)
+ +- Exchange (98)
+ +- HashAggregate (97)
+ +- Scan parquet (96)
+
+
+(1) ScanTransformer parquet
Output [2]: [c_custkey#X, c_name#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -136,7 +135,7 @@ Input [2]: [c_custkey#X, c_name#X]
(9) InputIteratorTransformer
Input [2]: [c_custkey#X, c_name#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -173,7 +172,7 @@ Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X]
(18) InputIteratorTransformer
Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X]
-(19) Scan parquet
+(19) ScanTransformer parquet
Output [2]: [l_orderkey#X, l_quantity#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -289,7 +288,7 @@ Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X]
(46) InputIteratorTransformer
Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X]
-(47) Scan parquet
+(47) ScanTransformer parquet
Output [2]: [l_orderkey#X, l_quantity#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -371,219 +370,212 @@ Input [7]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X,
(66) RegularHashAggregateExecTransformer
Input [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_quantity#X]
Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X]
-Functions [1]: [partial_sum(l_quantity#X)]
-Aggregate Attributes [2]: [sum#X, isEmpty#X]
-Results [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X]
-
-(67) RegularHashAggregateExecTransformer
-Input [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X]
-Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X]
Functions [1]: [sum(l_quantity#X)]
Aggregate Attributes [1]: [sum(l_quantity#X)#X]
Results [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity#X)#X AS sum(l_quantity)#X]
-(68) WholeStageCodegenTransformer (X)
+(67) WholeStageCodegenTransformer (X)
Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X]
Arguments: false
-(69) TakeOrderedAndProjectExecTransformer
+(68) TakeOrderedAndProjectExecTransformer
Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X]
Arguments: X, [o_totalprice#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X], 0
-(70) VeloxColumnarToRow
+(69) VeloxColumnarToRow
Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X]
-(71) Scan parquet
+(70) Scan parquet
Output [2]: [c_custkey#X, c_name#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(c_custkey)]
ReadSchema: struct
-(72) Filter
+(71) Filter
Input [2]: [c_custkey#X, c_name#X]
Condition : isnotnull(c_custkey#X)
-(73) Exchange
+(72) Exchange
Input [2]: [c_custkey#X, c_name#X]
Arguments: hashpartitioning(c_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(74) Sort
+(73) Sort
Input [2]: [c_custkey#X, c_name#X]
Arguments: [c_custkey#X ASC NULLS FIRST], false, 0
-(75) Scan parquet
+(74) Scan parquet
Output [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(o_custkey), IsNotNull(o_orderkey)]
ReadSchema: struct
-(76) Filter
+(75) Filter
Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X]
Condition : (isnotnull(o_custkey#X) AND isnotnull(o_orderkey#X))
-(77) Exchange
+(76) Exchange
Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X]
Arguments: hashpartitioning(o_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(78) Sort
+(77) Sort
Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X]
Arguments: [o_orderkey#X ASC NULLS FIRST], false, 0
-(79) Scan parquet
+(78) Scan parquet
Output [2]: [l_orderkey#X, l_quantity#X]
Batched: true
Location: InMemoryFileIndex [*]
ReadSchema: struct
-(80) HashAggregate
+(79) HashAggregate
Input [2]: [l_orderkey#X, l_quantity#X]
Keys [1]: [l_orderkey#X]
Functions [1]: [partial_sum(l_quantity#X)]
Aggregate Attributes [2]: [sum#X, isEmpty#X]
Results [3]: [l_orderkey#X, sum#X, isEmpty#X]
-(81) Exchange
+(80) Exchange
Input [3]: [l_orderkey#X, sum#X, isEmpty#X]
Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(82) HashAggregate
+(81) HashAggregate
Input [3]: [l_orderkey#X, sum#X, isEmpty#X]
Keys [1]: [l_orderkey#X]
Functions [1]: [sum(l_quantity#X)]
Aggregate Attributes [1]: [sum(l_quantity#X)#X]
Results [2]: [l_orderkey#X, sum(l_quantity#X)#X AS sum(l_quantity#X)#X]
-(83) Filter
+(82) Filter
Input [2]: [l_orderkey#X, sum(l_quantity#X)#X]
Condition : (isnotnull(sum(l_quantity#X)#X) AND (sum(l_quantity#X)#X > 300.00))
-(84) Project
+(83) Project
Output [1]: [l_orderkey#X]
Input [2]: [l_orderkey#X, sum(l_quantity#X)#X]
-(85) Sort
+(84) Sort
Input [1]: [l_orderkey#X]
Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0
-(86) SortMergeJoin
+(85) SortMergeJoin
Left keys [1]: [o_orderkey#X]
Right keys [1]: [l_orderkey#X]
Join condition: None
-(87) Exchange
+(86) Exchange
Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X]
Arguments: hashpartitioning(o_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(88) Sort
+(87) Sort
Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X]
Arguments: [o_custkey#X ASC NULLS FIRST], false, 0
-(89) SortMergeJoin
+(88) SortMergeJoin
Left keys [1]: [c_custkey#X]
Right keys [1]: [o_custkey#X]
Join condition: None
-(90) Project
+(89) Project
Output [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X]
Input [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X]
-(91) Exchange
+(90) Exchange
Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X]
Arguments: hashpartitioning(o_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(92) Sort
+(91) Sort
Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X]
Arguments: [o_orderkey#X ASC NULLS FIRST], false, 0
-(93) Scan parquet
+(92) Scan parquet
Output [2]: [l_orderkey#X, l_quantity#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(l_orderkey)]
ReadSchema: struct
-(94) Filter
+(93) Filter
Input [2]: [l_orderkey#X, l_quantity#X]
Condition : isnotnull(l_orderkey#X)
-(95) Exchange
+(94) Exchange
Input [2]: [l_orderkey#X, l_quantity#X]
Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(96) Sort
+(95) Sort
Input [2]: [l_orderkey#X, l_quantity#X]
Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0
-(97) Scan parquet
+(96) Scan parquet
Output [2]: [l_orderkey#X, l_quantity#X]
Batched: true
Location: InMemoryFileIndex [*]
ReadSchema: struct
-(98) HashAggregate
+(97) HashAggregate
Input [2]: [l_orderkey#X, l_quantity#X]
Keys [1]: [l_orderkey#X]
Functions [1]: [partial_sum(l_quantity#X)]
Aggregate Attributes [2]: [sum#X, isEmpty#X]
Results [3]: [l_orderkey#X, sum#X, isEmpty#X]
-(99) Exchange
+(98) Exchange
Input [3]: [l_orderkey#X, sum#X, isEmpty#X]
Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(100) HashAggregate
+(99) HashAggregate
Input [3]: [l_orderkey#X, sum#X, isEmpty#X]
Keys [1]: [l_orderkey#X]
Functions [1]: [sum(l_quantity#X)]
Aggregate Attributes [1]: [sum(l_quantity#X)#X]
Results [2]: [l_orderkey#X, sum(l_quantity#X)#X AS sum(l_quantity#X)#X]
-(101) Filter
+(100) Filter
Input [2]: [l_orderkey#X, sum(l_quantity#X)#X]
Condition : (isnotnull(sum(l_quantity#X)#X) AND (sum(l_quantity#X)#X > 300.00))
-(102) Project
+(101) Project
Output [1]: [l_orderkey#X]
Input [2]: [l_orderkey#X, sum(l_quantity#X)#X]
-(103) Sort
+(102) Sort
Input [1]: [l_orderkey#X]
Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0
-(104) SortMergeJoin
+(103) SortMergeJoin
Left keys [1]: [l_orderkey#X]
Right keys [1]: [l_orderkey#X]
Join condition: None
-(105) SortMergeJoin
+(104) SortMergeJoin
Left keys [1]: [o_orderkey#X]
Right keys [1]: [l_orderkey#X]
Join condition: None
-(106) Project
+(105) Project
Output [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_quantity#X]
Input [7]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_orderkey#X, l_quantity#X]
-(107) HashAggregate
+(106) HashAggregate
Input [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_quantity#X]
Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X]
Functions [1]: [partial_sum(l_quantity#X)]
Aggregate Attributes [2]: [sum#X, isEmpty#X]
Results [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X]
-(108) HashAggregate
+(107) HashAggregate
Input [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X]
Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X]
Functions [1]: [sum(l_quantity#X)]
Aggregate Attributes [1]: [sum(l_quantity#X)#X]
Results [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity#X)#X AS sum(l_quantity)#X]
-(109) TakeOrderedAndProject
+(108) TakeOrderedAndProject
Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X]
Arguments: X, [o_totalprice#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X]
-(110) AdaptiveSparkPlan
+(109) AdaptiveSparkPlan
Output [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X]
Arguments: isFinalPlan=true
\ No newline at end of file
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 d06cdb0fecff..507b700e37e9 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
@@ -1,42 +1,41 @@
== Physical Plan ==
-AdaptiveSparkPlan (38)
+AdaptiveSparkPlan (37)
+- == Final Plan ==
- VeloxColumnarToRow (24)
- +- ^ RegularHashAggregateExecTransformer (22)
- +- ^ RegularHashAggregateExecTransformer (21)
- +- ^ ProjectExecTransformer (20)
- +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19)
- :- ^ InputIteratorTransformer (9)
- : +- ShuffleQueryStage (7)
- : +- ColumnarExchange (6)
- : +- VeloxResizeBatches (5)
- : +- ^ ProjectExecTransformer (3)
- : +- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
- +- ^ InputIteratorTransformer (18)
- +- ShuffleQueryStage (16)
- +- ColumnarExchange (15)
- +- VeloxResizeBatches (14)
- +- ^ ProjectExecTransformer (12)
- +- ^ FilterExecTransformer (11)
- +- ^ Scan parquet (10)
+ VeloxColumnarToRow (23)
+ +- ^ RegularHashAggregateExecTransformer (21)
+ +- ^ ProjectExecTransformer (20)
+ +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19)
+ :- ^ InputIteratorTransformer (9)
+ : +- ShuffleQueryStage (7)
+ : +- ColumnarExchange (6)
+ : +- VeloxResizeBatches (5)
+ : +- ^ ProjectExecTransformer (3)
+ : +- ^ FilterExecTransformer (2)
+ : +- ^ ScanTransformer parquet (1)
+ +- ^ InputIteratorTransformer (18)
+ +- ShuffleQueryStage (16)
+ +- ColumnarExchange (15)
+ +- VeloxResizeBatches (14)
+ +- ^ ProjectExecTransformer (12)
+ +- ^ FilterExecTransformer (11)
+ +- ^ ScanTransformer parquet (10)
+- == Initial Plan ==
- HashAggregate (37)
- +- HashAggregate (36)
- +- Project (35)
- +- SortMergeJoin Inner (34)
- :- Sort (29)
- : +- Exchange (28)
- : +- Project (27)
- : +- Filter (26)
- : +- Scan parquet (25)
- +- Sort (33)
- +- Exchange (32)
- +- Filter (31)
- +- Scan parquet (30)
-
-
-(1) Scan parquet
+ HashAggregate (36)
+ +- HashAggregate (35)
+ +- Project (34)
+ +- SortMergeJoin Inner (33)
+ :- Sort (28)
+ : +- Exchange (27)
+ : +- Project (26)
+ : +- Filter (25)
+ : +- Scan parquet (24)
+ +- Sort (32)
+ +- Exchange (31)
+ +- Filter (30)
+ +- Scan parquet (29)
+
+
+(1) ScanTransformer parquet
Output [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -73,7 +72,7 @@ Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X]
(9) InputIteratorTransformer
Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -122,89 +121,82 @@ Input [8]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, p_partke
(21) RegularHashAggregateExecTransformer
Input [3]: [l_extendedprice#X, l_discount#X, _pre_X#X]
Keys: []
-Functions [1]: [partial_sum(_pre_X#X)]
-Aggregate Attributes [2]: [sum#X, isEmpty#X]
-Results [2]: [sum#X, isEmpty#X]
-
-(22) RegularHashAggregateExecTransformer
-Input [2]: [sum#X, isEmpty#X]
-Keys: []
-Functions [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))]
+Functions [1]: [sum(_pre_X#X)]
Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X]
Results [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X AS revenue#X]
-(23) WholeStageCodegenTransformer (X)
+(22) WholeStageCodegenTransformer (X)
Input [1]: [revenue#X]
Arguments: false
-(24) VeloxColumnarToRow
+(23) VeloxColumnarToRow
Input [1]: [revenue#X]
-(25) Scan parquet
+(24) Scan parquet
Output [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(l_shipinstruct), In(l_shipmode, [AIR,AIR REG]), EqualTo(l_shipinstruct,DELIVER IN PERSON), IsNotNull(l_partkey), Or(Or(And(GreaterThanOrEqual(l_quantity,1.00),LessThanOrEqual(l_quantity,11.00)),And(GreaterThanOrEqual(l_quantity,10.00),LessThanOrEqual(l_quantity,20.00))),And(GreaterThanOrEqual(l_quantity,20.00),LessThanOrEqual(l_quantity,30.00)))]
ReadSchema: struct
-(26) Filter
+(25) Filter
Input [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X]
Condition : ((((isnotnull(l_shipinstruct#X) AND l_shipmode#X IN (AIR,AIR REG)) AND (l_shipinstruct#X = DELIVER IN PERSON)) AND isnotnull(l_partkey#X)) AND ((((l_quantity#X >= 1.00) AND (l_quantity#X <= 11.00)) OR ((l_quantity#X >= 10.00) AND (l_quantity#X <= 20.00))) OR ((l_quantity#X >= 20.00) AND (l_quantity#X <= 30.00))))
-(27) Project
+(26) Project
Output [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X]
Input [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X]
-(28) Exchange
+(27) Exchange
Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X]
Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(29) Sort
+(28) Sort
Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X]
Arguments: [l_partkey#X ASC NULLS FIRST], false, 0
-(30) Scan parquet
+(29) Scan parquet
Output [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(p_size), GreaterThanOrEqual(p_size,1), IsNotNull(p_partkey), Or(Or(And(And(EqualTo(p_brand,Brand#X),In(p_container, [SM BOX,SM CASE,SM PACK,SM PKG])),LessThanOrEqual(p_size,5)),And(And(EqualTo(p_brand,Brand#X),In(p_container, [MED BAG,MED BOX,MED PACK,MED PKG])),LessThanOrEqual(p_size,10))),And(And(EqualTo(p_brand,Brand#X),In(p_container, [LG BOX,LG CASE,LG PACK,LG PKG])),LessThanOrEqual(p_size,15)))]
ReadSchema: struct
-(31) Filter
+(30) Filter
Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X]
Condition : (((isnotnull(p_size#X) AND (p_size#X >= 1)) AND isnotnull(p_partkey#X)) AND (((((p_brand#X = Brand#X) AND p_container#X IN (SM CASE,SM BOX,SM PACK,SM PKG)) AND (p_size#X <= 5)) OR (((p_brand#X = Brand#X) AND p_container#X IN (MED BAG,MED BOX,MED PKG,MED PACK)) AND (p_size#X <= 10))) OR (((p_brand#X = Brand#X) AND p_container#X IN (LG CASE,LG BOX,LG PACK,LG PKG)) AND (p_size#X <= 15))))
-(32) Exchange
+(31) Exchange
Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X]
Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(33) Sort
+(32) Sort
Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X]
Arguments: [p_partkey#X ASC NULLS FIRST], false, 0
-(34) SortMergeJoin
+(33) SortMergeJoin
Left keys [1]: [l_partkey#X]
Right keys [1]: [p_partkey#X]
Join condition: (((((((p_brand#X = Brand#X) AND p_container#X IN (SM CASE,SM BOX,SM PACK,SM PKG)) AND (l_quantity#X >= 1.00)) AND (l_quantity#X <= 11.00)) AND (p_size#X <= 5)) OR (((((p_brand#X = Brand#X) AND p_container#X IN (MED BAG,MED BOX,MED PKG,MED PACK)) AND (l_quantity#X >= 10.00)) AND (l_quantity#X <= 20.00)) AND (p_size#X <= 10))) OR (((((p_brand#X = Brand#X) AND p_container#X IN (LG CASE,LG BOX,LG PACK,LG PKG)) AND (l_quantity#X >= 20.00)) AND (l_quantity#X <= 30.00)) AND (p_size#X <= 15)))
-(35) Project
+(34) Project
Output [2]: [l_extendedprice#X, l_discount#X]
Input [8]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, p_partkey#X, p_brand#X, p_size#X, p_container#X]
-(36) HashAggregate
+(35) HashAggregate
Input [2]: [l_extendedprice#X, l_discount#X]
Keys: []
Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))]
Aggregate Attributes [2]: [sum#X, isEmpty#X]
Results [2]: [sum#X, isEmpty#X]
-(37) HashAggregate
+(36) HashAggregate
Input [2]: [sum#X, isEmpty#X]
Keys: []
Functions [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))]
Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X]
Results [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X AS revenue#X]
-(38) AdaptiveSparkPlan
+(37) AdaptiveSparkPlan
Output [1]: [revenue#X]
Arguments: isFinalPlan=true
\ No newline at end of file
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 6fa1271a5a39..13c10a7bba29 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
@@ -1,33 +1,33 @@
== Physical Plan ==
-AdaptiveSparkPlan (146)
+AdaptiveSparkPlan (145)
+- == Final Plan ==
- VeloxColumnarToRow (96)
- +- ^ SortExecTransformer (94)
- +- ^ InputIteratorTransformer (93)
- +- ShuffleQueryStage (91)
- +- ColumnarExchange (90)
- +- VeloxResizeBatches (89)
- +- ^ ProjectExecTransformer (87)
- +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (86)
- :- ^ InputIteratorTransformer (76)
- : +- ShuffleQueryStage (74)
- : +- ColumnarExchange (73)
- : +- VeloxResizeBatches (72)
- : +- ^ ProjectExecTransformer (70)
- : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (69)
+ VeloxColumnarToRow (95)
+ +- ^ SortExecTransformer (93)
+ +- ^ InputIteratorTransformer (92)
+ +- ShuffleQueryStage (90)
+ +- ColumnarExchange (89)
+ +- VeloxResizeBatches (88)
+ +- ^ ProjectExecTransformer (86)
+ +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (85)
+ :- ^ InputIteratorTransformer (75)
+ : +- ShuffleQueryStage (73)
+ : +- ColumnarExchange (72)
+ : +- VeloxResizeBatches (71)
+ : +- ^ ProjectExecTransformer (69)
+ : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (68)
: :- ^ InputIteratorTransformer (9)
: : +- ShuffleQueryStage (7)
: : +- ColumnarExchange (6)
: : +- VeloxResizeBatches (5)
: : +- ^ ProjectExecTransformer (3)
: : +- ^ FilterExecTransformer (2)
- : : +- ^ Scan parquet (1)
- : +- ^ InputIteratorTransformer (68)
- : +- ShuffleQueryStage (66)
- : +- ColumnarExchange (65)
- : +- VeloxResizeBatches (64)
- : +- ^ ProjectExecTransformer (62)
- : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (61)
+ : : +- ^ ScanTransformer parquet (1)
+ : +- ^ InputIteratorTransformer (67)
+ : +- ShuffleQueryStage (65)
+ : +- ColumnarExchange (64)
+ : +- VeloxResizeBatches (63)
+ : +- ^ ProjectExecTransformer (61)
+ : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (60)
: :- ^ InputIteratorTransformer (35)
: : +- ShuffleQueryStage (33)
: : +- ColumnarExchange (32)
@@ -40,94 +40,93 @@ AdaptiveSparkPlan (146)
: : : +- VeloxResizeBatches (14)
: : : +- ^ ProjectExecTransformer (12)
: : : +- ^ FilterExecTransformer (11)
- : : : +- ^ Scan parquet (10)
+ : : : +- ^ ScanTransformer parquet (10)
: : +- ^ InputIteratorTransformer (27)
: : +- ShuffleQueryStage (25)
: : +- ColumnarExchange (24)
: : +- VeloxResizeBatches (23)
: : +- ^ ProjectExecTransformer (21)
: : +- ^ FilterExecTransformer (20)
- : : +- ^ Scan parquet (19)
- : +- ^ InputIteratorTransformer (60)
- : +- ShuffleQueryStage (58)
- : +- ColumnarExchange (57)
- : +- VeloxResizeBatches (56)
- : +- ^ ProjectExecTransformer (54)
- : +- ^ FilterExecTransformer (53)
- : +- ^ ProjectExecTransformer (52)
- : +- ^ RegularHashAggregateExecTransformer (51)
- : +- ^ RegularHashAggregateExecTransformer (50)
- : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (49)
- : :- ^ InputIteratorTransformer (44)
- : : +- ShuffleQueryStage (42)
- : : +- ColumnarExchange (41)
- : : +- VeloxResizeBatches (40)
- : : +- ^ ProjectExecTransformer (38)
- : : +- ^ FilterExecTransformer (37)
- : : +- ^ Scan parquet (36)
- : +- ^ InputIteratorTransformer (48)
- : +- ShuffleQueryStage (46)
- : +- ReusedExchange (45)
- +- ^ InputIteratorTransformer (85)
- +- ShuffleQueryStage (83)
- +- ColumnarExchange (82)
- +- VeloxResizeBatches (81)
- +- ^ ProjectExecTransformer (79)
- +- ^ FilterExecTransformer (78)
- +- ^ Scan parquet (77)
+ : : +- ^ ScanTransformer parquet (19)
+ : +- ^ InputIteratorTransformer (59)
+ : +- ShuffleQueryStage (57)
+ : +- ColumnarExchange (56)
+ : +- VeloxResizeBatches (55)
+ : +- ^ ProjectExecTransformer (53)
+ : +- ^ FilterExecTransformer (52)
+ : +- ^ ProjectExecTransformer (51)
+ : +- ^ RegularHashAggregateExecTransformer (50)
+ : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (49)
+ : :- ^ InputIteratorTransformer (44)
+ : : +- ShuffleQueryStage (42)
+ : : +- ColumnarExchange (41)
+ : : +- VeloxResizeBatches (40)
+ : : +- ^ ProjectExecTransformer (38)
+ : : +- ^ FilterExecTransformer (37)
+ : : +- ^ ScanTransformer parquet (36)
+ : +- ^ InputIteratorTransformer (48)
+ : +- ShuffleQueryStage (46)
+ : +- ReusedExchange (45)
+ +- ^ InputIteratorTransformer (84)
+ +- ShuffleQueryStage (82)
+ +- ColumnarExchange (81)
+ +- VeloxResizeBatches (80)
+ +- ^ ProjectExecTransformer (78)
+ +- ^ FilterExecTransformer (77)
+ +- ^ ScanTransformer parquet (76)
+- == Initial Plan ==
- Sort (145)
- +- Exchange (144)
- +- Project (143)
- +- SortMergeJoin Inner (142)
- :- Sort (136)
- : +- Exchange (135)
- : +- Project (134)
- : +- SortMergeJoin LeftSemi (133)
- : :- Sort (100)
- : : +- Exchange (99)
- : : +- Filter (98)
- : : +- Scan parquet (97)
- : +- Sort (132)
- : +- Exchange (131)
- : +- Project (130)
- : +- SortMergeJoin Inner (129)
- : :- Sort (112)
- : : +- Exchange (111)
- : : +- SortMergeJoin LeftSemi (110)
- : : :- Sort (104)
- : : : +- Exchange (103)
- : : : +- Filter (102)
- : : : +- Scan parquet (101)
- : : +- Sort (109)
- : : +- Exchange (108)
- : : +- Project (107)
- : : +- Filter (106)
- : : +- Scan parquet (105)
- : +- Sort (128)
- : +- Exchange (127)
- : +- Filter (126)
- : +- HashAggregate (125)
- : +- HashAggregate (124)
- : +- SortMergeJoin LeftSemi (123)
- : :- Sort (117)
- : : +- Exchange (116)
- : : +- Project (115)
- : : +- Filter (114)
- : : +- Scan parquet (113)
- : +- Sort (122)
- : +- Exchange (121)
- : +- Project (120)
- : +- Filter (119)
- : +- Scan parquet (118)
- +- Sort (141)
- +- Exchange (140)
- +- Project (139)
- +- Filter (138)
- +- Scan parquet (137)
-
-
-(1) Scan parquet
+ Sort (144)
+ +- Exchange (143)
+ +- Project (142)
+ +- SortMergeJoin Inner (141)
+ :- Sort (135)
+ : +- Exchange (134)
+ : +- Project (133)
+ : +- SortMergeJoin LeftSemi (132)
+ : :- Sort (99)
+ : : +- Exchange (98)
+ : : +- Filter (97)
+ : : +- Scan parquet (96)
+ : +- Sort (131)
+ : +- Exchange (130)
+ : +- Project (129)
+ : +- SortMergeJoin Inner (128)
+ : :- Sort (111)
+ : : +- Exchange (110)
+ : : +- SortMergeJoin LeftSemi (109)
+ : : :- Sort (103)
+ : : : +- Exchange (102)
+ : : : +- Filter (101)
+ : : : +- Scan parquet (100)
+ : : +- Sort (108)
+ : : +- Exchange (107)
+ : : +- Project (106)
+ : : +- Filter (105)
+ : : +- Scan parquet (104)
+ : +- Sort (127)
+ : +- Exchange (126)
+ : +- Filter (125)
+ : +- HashAggregate (124)
+ : +- HashAggregate (123)
+ : +- SortMergeJoin LeftSemi (122)
+ : :- Sort (116)
+ : : +- Exchange (115)
+ : : +- Project (114)
+ : : +- Filter (113)
+ : : +- Scan parquet (112)
+ : +- Sort (121)
+ : +- Exchange (120)
+ : +- Project (119)
+ : +- Filter (118)
+ : +- Scan parquet (117)
+ +- Sort (140)
+ +- Exchange (139)
+ +- Project (138)
+ +- Filter (137)
+ +- Scan parquet (136)
+
+
+(1) ScanTransformer parquet
Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -164,7 +163,7 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X]
(9) InputIteratorTransformer
Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -201,7 +200,7 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
(18) InputIteratorTransformer
Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
-(19) Scan parquet
+(19) ScanTransformer parquet
Output [2]: [p_partkey#X, p_name#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -269,7 +268,7 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
(35) InputIteratorTransformer
Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
-(36) Scan parquet
+(36) ScanTransformer parquet
Output [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -327,417 +326,410 @@ Join condition: None
(50) RegularHashAggregateExecTransformer
Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X]
Keys [2]: [l_partkey#X, l_suppkey#X]
-Functions [1]: [partial_sum(l_quantity#X)]
-Aggregate Attributes [2]: [sum#X, isEmpty#X]
-Results [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X]
-
-(51) RegularHashAggregateExecTransformer
-Input [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X]
-Keys [2]: [l_partkey#X, l_suppkey#X]
Functions [1]: [sum(l_quantity#X)]
Aggregate Attributes [1]: [sum(l_quantity#X)#X]
Results [3]: [l_partkey#X, l_suppkey#X, sum(l_quantity#X)#X]
-(52) ProjectExecTransformer
+(51) ProjectExecTransformer
Output [3]: [CheckOverflow((0.50 * promote_precision(sum(l_quantity#X)#X)), DecimalType(24,3), true) AS (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
Input [3]: [l_partkey#X, l_suppkey#X, sum(l_quantity#X)#X]
-(53) FilterExecTransformer
+(52) FilterExecTransformer
Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
Arguments: isnotnull((0.5 * sum(l_quantity))#X)
-(54) ProjectExecTransformer
+(53) ProjectExecTransformer
Output [4]: [hash(l_partkey#X, l_suppkey#X, 42) AS hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
-(55) WholeStageCodegenTransformer (X)
+(54) WholeStageCodegenTransformer (X)
Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
Arguments: false
-(56) VeloxResizeBatches
+(55) VeloxResizeBatches
Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
Arguments: X, X
-(57) ColumnarExchange
+(56) ColumnarExchange
Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
Arguments: hashpartitioning(l_partkey#X, l_suppkey#X, 1), ENSURE_REQUIREMENTS, [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X], [plan_id=X], [shuffle_writer_type=hash]
-(58) ShuffleQueryStage
+(57) ShuffleQueryStage
Output [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
Arguments: X
-(59) InputAdapter
+(58) InputAdapter
Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
-(60) InputIteratorTransformer
+(59) InputIteratorTransformer
Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
-(61) ShuffledHashJoinExecTransformer
+(60) ShuffledHashJoinExecTransformer
Left keys [2]: [ps_partkey#X, ps_suppkey#X]
Right keys [2]: [l_partkey#X, l_suppkey#X]
Join condition: (cast(cast(ps_availqty#X as decimal(10,0)) as decimal(24,3)) > (0.5 * sum(l_quantity))#X)
-(62) ProjectExecTransformer
+(61) ProjectExecTransformer
Output [2]: [hash(ps_suppkey#X, 42) AS hash_partition_key#X, ps_suppkey#X]
Input [6]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
-(63) WholeStageCodegenTransformer (X)
+(62) WholeStageCodegenTransformer (X)
Input [2]: [hash_partition_key#X, ps_suppkey#X]
Arguments: false
-(64) VeloxResizeBatches
+(63) VeloxResizeBatches
Input [2]: [hash_partition_key#X, ps_suppkey#X]
Arguments: X, X
-(65) ColumnarExchange
+(64) ColumnarExchange
Input [2]: [hash_partition_key#X, ps_suppkey#X]
Arguments: hashpartitioning(ps_suppkey#X, 1), ENSURE_REQUIREMENTS, [ps_suppkey#X], [plan_id=X], [shuffle_writer_type=hash]
-(66) ShuffleQueryStage
+(65) ShuffleQueryStage
Output [1]: [ps_suppkey#X]
Arguments: X
-(67) InputAdapter
+(66) InputAdapter
Input [1]: [ps_suppkey#X]
-(68) InputIteratorTransformer
+(67) InputIteratorTransformer
Input [1]: [ps_suppkey#X]
-(69) ShuffledHashJoinExecTransformer
+(68) ShuffledHashJoinExecTransformer
Left keys [1]: [s_suppkey#X]
Right keys [1]: [ps_suppkey#X]
Join condition: None
-(70) ProjectExecTransformer
+(69) ProjectExecTransformer
Output [4]: [hash(s_nationkey#X, 42) AS hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X]
Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X]
-(71) WholeStageCodegenTransformer (X)
+(70) WholeStageCodegenTransformer (X)
Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X]
Arguments: false
-(72) VeloxResizeBatches
+(71) VeloxResizeBatches
Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X]
Arguments: X, X
-(73) ColumnarExchange
+(72) ColumnarExchange
Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X]
Arguments: hashpartitioning(s_nationkey#X, 1), ENSURE_REQUIREMENTS, [s_name#X, s_address#X, s_nationkey#X], [plan_id=X], [shuffle_writer_type=hash]
-(74) ShuffleQueryStage
+(73) ShuffleQueryStage
Output [3]: [s_name#X, s_address#X, s_nationkey#X]
Arguments: X
-(75) InputAdapter
+(74) InputAdapter
Input [3]: [s_name#X, s_address#X, s_nationkey#X]
-(76) InputIteratorTransformer
+(75) InputIteratorTransformer
Input [3]: [s_name#X, s_address#X, s_nationkey#X]
-(77) Scan parquet
+(76) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(n_name), EqualTo(n_name,CANADA), IsNotNull(n_nationkey)]
ReadSchema: struct
-(78) FilterExecTransformer
+(77) FilterExecTransformer
Input [2]: [n_nationkey#X, n_name#X]
Arguments: ((isnotnull(n_name#X) AND (n_name#X = CANADA)) AND isnotnull(n_nationkey#X))
-(79) ProjectExecTransformer
+(78) ProjectExecTransformer
Output [2]: [hash(n_nationkey#X, 42) AS hash_partition_key#X, n_nationkey#X]
Input [2]: [n_nationkey#X, n_name#X]
-(80) WholeStageCodegenTransformer (X)
+(79) WholeStageCodegenTransformer (X)
Input [2]: [hash_partition_key#X, n_nationkey#X]
Arguments: false
-(81) VeloxResizeBatches
+(80) VeloxResizeBatches
Input [2]: [hash_partition_key#X, n_nationkey#X]
Arguments: X, X
-(82) ColumnarExchange
+(81) ColumnarExchange
Input [2]: [hash_partition_key#X, n_nationkey#X]
Arguments: hashpartitioning(n_nationkey#X, 1), ENSURE_REQUIREMENTS, [n_nationkey#X], [plan_id=X], [shuffle_writer_type=hash]
-(83) ShuffleQueryStage
+(82) ShuffleQueryStage
Output [1]: [n_nationkey#X]
Arguments: X
-(84) InputAdapter
+(83) InputAdapter
Input [1]: [n_nationkey#X]
-(85) InputIteratorTransformer
+(84) InputIteratorTransformer
Input [1]: [n_nationkey#X]
-(86) ShuffledHashJoinExecTransformer
+(85) ShuffledHashJoinExecTransformer
Left keys [1]: [s_nationkey#X]
Right keys [1]: [n_nationkey#X]
Join condition: None
-(87) ProjectExecTransformer
+(86) ProjectExecTransformer
Output [2]: [s_name#X, s_address#X]
Input [4]: [s_name#X, s_address#X, s_nationkey#X, n_nationkey#X]
-(88) WholeStageCodegenTransformer (X)
+(87) WholeStageCodegenTransformer (X)
Input [2]: [s_name#X, s_address#X]
Arguments: false
-(89) VeloxResizeBatches
+(88) VeloxResizeBatches
Input [2]: [s_name#X, s_address#X]
Arguments: X, X
-(90) ColumnarExchange
+(89) ColumnarExchange
Input [2]: [s_name#X, s_address#X]
Arguments: rangepartitioning(s_name#X ASC NULLS FIRST, 1), ENSURE_REQUIREMENTS, [plan_id=X], [shuffle_writer_type=hash]
-(91) ShuffleQueryStage
+(90) ShuffleQueryStage
Output [2]: [s_name#X, s_address#X]
Arguments: X
-(92) InputAdapter
+(91) InputAdapter
Input [2]: [s_name#X, s_address#X]
-(93) InputIteratorTransformer
+(92) InputIteratorTransformer
Input [2]: [s_name#X, s_address#X]
-(94) SortExecTransformer
+(93) SortExecTransformer
Input [2]: [s_name#X, s_address#X]
Arguments: [s_name#X ASC NULLS FIRST], true, 0
-(95) WholeStageCodegenTransformer (X)
+(94) WholeStageCodegenTransformer (X)
Input [2]: [s_name#X, s_address#X]
Arguments: false
-(96) VeloxColumnarToRow
+(95) VeloxColumnarToRow
Input [2]: [s_name#X, s_address#X]
-(97) Scan parquet
+(96) Scan parquet
Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(s_nationkey)]
ReadSchema: struct
-(98) Filter
+(97) Filter
Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X]
Condition : isnotnull(s_nationkey#X)
-(99) Exchange
+(98) Exchange
Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X]
Arguments: hashpartitioning(s_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(100) Sort
+(99) Sort
Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X]
Arguments: [s_suppkey#X ASC NULLS FIRST], false, 0
-(101) Scan parquet
+(100) Scan parquet
Output [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(ps_availqty), IsNotNull(ps_partkey), IsNotNull(ps_suppkey)]
ReadSchema: struct
-(102) Filter
+(101) Filter
Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
Condition : ((isnotnull(ps_availqty#X) AND isnotnull(ps_partkey#X)) AND isnotnull(ps_suppkey#X))
-(103) Exchange
+(102) Exchange
Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
Arguments: hashpartitioning(ps_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(104) Sort
+(103) Sort
Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
Arguments: [ps_partkey#X ASC NULLS FIRST], false, 0
-(105) Scan parquet
+(104) Scan parquet
Output [2]: [p_partkey#X, p_name#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(p_name), StringStartsWith(p_name,forest)]
ReadSchema: struct
-(106) Filter
+(105) Filter
Input [2]: [p_partkey#X, p_name#X]
Condition : (isnotnull(p_name#X) AND StartsWith(p_name#X, forest))
-(107) Project
+(106) Project
Output [1]: [p_partkey#X]
Input [2]: [p_partkey#X, p_name#X]
-(108) Exchange
+(107) Exchange
Input [1]: [p_partkey#X]
Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(109) Sort
+(108) Sort
Input [1]: [p_partkey#X]
Arguments: [p_partkey#X ASC NULLS FIRST], false, 0
-(110) SortMergeJoin
+(109) SortMergeJoin
Left keys [1]: [ps_partkey#X]
Right keys [1]: [p_partkey#X]
Join condition: None
-(111) Exchange
+(110) Exchange
Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
Arguments: hashpartitioning(ps_partkey#X, ps_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(112) Sort
+(111) Sort
Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
Arguments: [ps_partkey#X ASC NULLS FIRST, ps_suppkey#X ASC NULLS FIRST], false, 0
-(113) Scan parquet
+(112) Scan parquet
Output [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1994-01-01), LessThan(l_shipdate,1995-01-01), IsNotNull(l_partkey), IsNotNull(l_suppkey)]
ReadSchema: struct
-(114) Filter
+(113) Filter
Input [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X]
Condition : ((((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1994-01-01)) AND (l_shipdate#X < 1995-01-01)) AND isnotnull(l_partkey#X)) AND isnotnull(l_suppkey#X))
-(115) Project
+(114) Project
Output [3]: [l_partkey#X, l_suppkey#X, l_quantity#X]
Input [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X]
-(116) Exchange
+(115) Exchange
Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X]
Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(117) Sort
+(116) Sort
Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X]
Arguments: [l_partkey#X ASC NULLS FIRST], false, 0
-(118) Scan parquet
+(117) Scan parquet
Output [2]: [p_partkey#X, p_name#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(p_name), StringStartsWith(p_name,forest)]
ReadSchema: struct
-(119) Filter
+(118) Filter
Input [2]: [p_partkey#X, p_name#X]
Condition : (isnotnull(p_name#X) AND StartsWith(p_name#X, forest))
-(120) Project
+(119) Project
Output [1]: [p_partkey#X]
Input [2]: [p_partkey#X, p_name#X]
-(121) Exchange
+(120) Exchange
Input [1]: [p_partkey#X]
Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(122) Sort
+(121) Sort
Input [1]: [p_partkey#X]
Arguments: [p_partkey#X ASC NULLS FIRST], false, 0
-(123) SortMergeJoin
+(122) SortMergeJoin
Left keys [1]: [l_partkey#X]
Right keys [1]: [p_partkey#X]
Join condition: None
-(124) HashAggregate
+(123) HashAggregate
Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X]
Keys [2]: [l_partkey#X, l_suppkey#X]
Functions [1]: [partial_sum(l_quantity#X)]
Aggregate Attributes [2]: [sum#X, isEmpty#X]
Results [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X]
-(125) HashAggregate
+(124) HashAggregate
Input [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X]
Keys [2]: [l_partkey#X, l_suppkey#X]
Functions [1]: [sum(l_quantity#X)]
Aggregate Attributes [1]: [sum(l_quantity#X)#X]
Results [3]: [CheckOverflow((0.50 * promote_precision(sum(l_quantity#X)#X)), DecimalType(24,3), true) AS (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
-(126) Filter
+(125) Filter
Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
Condition : isnotnull((0.5 * sum(l_quantity))#X)
-(127) Exchange
+(126) Exchange
Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
Arguments: hashpartitioning(l_partkey#X, l_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(128) Sort
+(127) Sort
Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
Arguments: [l_partkey#X ASC NULLS FIRST, l_suppkey#X ASC NULLS FIRST], false, 0
-(129) SortMergeJoin
+(128) SortMergeJoin
Left keys [2]: [ps_partkey#X, ps_suppkey#X]
Right keys [2]: [l_partkey#X, l_suppkey#X]
Join condition: (cast(cast(ps_availqty#X as decimal(10,0)) as decimal(24,3)) > (0.5 * sum(l_quantity))#X)
-(130) Project
+(129) Project
Output [1]: [ps_suppkey#X]
Input [6]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
-(131) Exchange
+(130) Exchange
Input [1]: [ps_suppkey#X]
Arguments: hashpartitioning(ps_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(132) Sort
+(131) Sort
Input [1]: [ps_suppkey#X]
Arguments: [ps_suppkey#X ASC NULLS FIRST], false, 0
-(133) SortMergeJoin
+(132) SortMergeJoin
Left keys [1]: [s_suppkey#X]
Right keys [1]: [ps_suppkey#X]
Join condition: None
-(134) Project
+(133) Project
Output [3]: [s_name#X, s_address#X, s_nationkey#X]
Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X]
-(135) Exchange
+(134) Exchange
Input [3]: [s_name#X, s_address#X, s_nationkey#X]
Arguments: hashpartitioning(s_nationkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(136) Sort
+(135) Sort
Input [3]: [s_name#X, s_address#X, s_nationkey#X]
Arguments: [s_nationkey#X ASC NULLS FIRST], false, 0
-(137) Scan parquet
+(136) Scan parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(n_name), EqualTo(n_name,CANADA), IsNotNull(n_nationkey)]
ReadSchema: struct
-(138) Filter
+(137) Filter
Input [2]: [n_nationkey#X, n_name#X]
Condition : ((isnotnull(n_name#X) AND (n_name#X = CANADA)) AND isnotnull(n_nationkey#X))
-(139) Project
+(138) Project
Output [1]: [n_nationkey#X]
Input [2]: [n_nationkey#X, n_name#X]
-(140) Exchange
+(139) Exchange
Input [1]: [n_nationkey#X]
Arguments: hashpartitioning(n_nationkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(141) Sort
+(140) Sort
Input [1]: [n_nationkey#X]
Arguments: [n_nationkey#X ASC NULLS FIRST], false, 0
-(142) SortMergeJoin
+(141) SortMergeJoin
Left keys [1]: [s_nationkey#X]
Right keys [1]: [n_nationkey#X]
Join condition: None
-(143) Project
+(142) Project
Output [2]: [s_name#X, s_address#X]
Input [4]: [s_name#X, s_address#X, s_nationkey#X, n_nationkey#X]
-(144) Exchange
+(143) Exchange
Input [2]: [s_name#X, s_address#X]
Arguments: rangepartitioning(s_name#X ASC NULLS FIRST, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(145) Sort
+(144) Sort
Input [2]: [s_name#X, s_address#X]
Arguments: [s_name#X ASC NULLS FIRST], true, 0
-(146) AdaptiveSparkPlan
+(145) AdaptiveSparkPlan
Output [2]: [s_name#X, s_address#X]
Arguments: isFinalPlan=true
\ No newline at end of file
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 e70a145e9c1d..e72762dd9bfe 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
@@ -30,7 +30,7 @@ AdaptiveSparkPlan (138)
: : : +- VeloxResizeBatches (5)
: : : +- ^ ProjectExecTransformer (3)
: : : +- ^ FilterExecTransformer (2)
- : : : +- ^ Scan parquet (1)
+ : : : +- ^ ScanTransformer parquet (1)
: : +- ^ InputIteratorTransformer (44)
: : +- ShuffleQueryStage (42)
: : +- ColumnarExchange (41)
@@ -44,34 +44,34 @@ AdaptiveSparkPlan (138)
: : : : +- VeloxResizeBatches (14)
: : : : +- ^ ProjectExecTransformer (12)
: : : : +- ^ FilterExecTransformer (11)
- : : : : +- ^ Scan parquet (10)
+ : : : : +- ^ ScanTransformer parquet (10)
: : : +- ^ InputIteratorTransformer (26)
: : : +- ShuffleQueryStage (24)
: : : +- ColumnarExchange (23)
: : : +- VeloxResizeBatches (22)
: : : +- ^ ProjectExecTransformer (20)
- : : : +- ^ Scan parquet (19)
+ : : : +- ^ ScanTransformer parquet (19)
: : +- ^ InputIteratorTransformer (36)
: : +- ShuffleQueryStage (34)
: : +- ColumnarExchange (33)
: : +- VeloxResizeBatches (32)
: : +- ^ ProjectExecTransformer (30)
: : +- ^ FilterExecTransformer (29)
- : : +- ^ Scan parquet (28)
+ : : +- ^ ScanTransformer parquet (28)
: +- ^ InputIteratorTransformer (61)
: +- ShuffleQueryStage (59)
: +- ColumnarExchange (58)
: +- VeloxResizeBatches (57)
: +- ^ ProjectExecTransformer (55)
: +- ^ FilterExecTransformer (54)
- : +- ^ Scan parquet (53)
+ : +- ^ ScanTransformer parquet (53)
+- ^ InputIteratorTransformer (78)
+- ShuffleQueryStage (76)
+- ColumnarExchange (75)
+- VeloxResizeBatches (74)
+- ^ ProjectExecTransformer (72)
+- ^ FilterExecTransformer (71)
- +- ^ Scan parquet (70)
+ +- ^ ScanTransformer parquet (70)
+- == Initial Plan ==
TakeOrderedAndProject (137)
+- HashAggregate (136)
@@ -120,7 +120,7 @@ AdaptiveSparkPlan (138)
+- Scan parquet (127)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [3]: [s_suppkey#X, s_name#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -157,7 +157,7 @@ Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X]
(9) InputIteratorTransformer
Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -194,7 +194,7 @@ Input [2]: [l_orderkey#X, l_suppkey#X]
(18) InputIteratorTransformer
Input [2]: [l_orderkey#X, l_suppkey#X]
-(19) Scan parquet
+(19) ScanTransformer parquet
Output [2]: [l_orderkey#X, l_suppkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -231,7 +231,7 @@ Left keys [1]: [l_orderkey#X]
Right keys [1]: [l_orderkey#X]
Join condition: NOT (l_suppkey#X = l_suppkey#X)
-(28) Scan parquet
+(28) ScanTransformer parquet
Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -330,7 +330,7 @@ Input [3]: [s_name#X, s_nationkey#X, l_orderkey#X]
(52) InputIteratorTransformer
Input [3]: [s_name#X, s_nationkey#X, l_orderkey#X]
-(53) Scan parquet
+(53) ScanTransformer parquet
Output [2]: [o_orderkey#X, o_orderstatus#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -398,7 +398,7 @@ Input [2]: [s_name#X, s_nationkey#X]
(69) InputIteratorTransformer
Input [2]: [s_name#X, s_nationkey#X]
-(70) Scan parquet
+(70) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 674e0b96983f..81af91c0ee2a 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
@@ -22,13 +22,13 @@ AdaptiveSparkPlan (52)
: +- VeloxResizeBatches (5)
: +- ^ ProjectExecTransformer (3)
: +- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
+ : +- ^ ScanTransformer parquet (1)
+- ^ InputIteratorTransformer (17)
+- ShuffleQueryStage (15)
+- ColumnarExchange (14)
+- VeloxResizeBatches (13)
+- ^ ProjectExecTransformer (11)
- +- ^ Scan parquet (10)
+ +- ^ ScanTransformer parquet (10)
+- == Initial Plan ==
Sort (51)
+- Exchange (50)
@@ -46,7 +46,7 @@ AdaptiveSparkPlan (52)
+- Scan parquet (42)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [3]: [c_custkey#X, c_phone#X, c_acctbal#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -83,7 +83,7 @@ Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X]
(9) InputIteratorTransformer
Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [1]: [o_custkey#X]
Batched: true
Location: InMemoryFileIndex [*]
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 3b9b7fd93b96..2e427b6bf3ca 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
@@ -1,67 +1,66 @@
== Physical Plan ==
-AdaptiveSparkPlan (67)
+AdaptiveSparkPlan (66)
+- == Final Plan ==
- VeloxColumnarToRow (43)
- +- TakeOrderedAndProjectExecTransformer (42)
- +- ^ ProjectExecTransformer (40)
- +- ^ RegularHashAggregateExecTransformer (39)
- +- ^ RegularHashAggregateExecTransformer (38)
- +- ^ ProjectExecTransformer (37)
- +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (36)
- :- ^ InputIteratorTransformer (26)
- : +- ShuffleQueryStage (24)
- : +- ColumnarExchange (23)
- : +- VeloxResizeBatches (22)
- : +- ^ ProjectExecTransformer (20)
- : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19)
- : :- ^ InputIteratorTransformer (9)
- : : +- ShuffleQueryStage (7)
- : : +- ColumnarExchange (6)
- : : +- VeloxResizeBatches (5)
- : : +- ^ ProjectExecTransformer (3)
- : : +- ^ FilterExecTransformer (2)
- : : +- ^ Scan parquet (1)
- : +- ^ InputIteratorTransformer (18)
- : +- ShuffleQueryStage (16)
- : +- ColumnarExchange (15)
- : +- VeloxResizeBatches (14)
- : +- ^ ProjectExecTransformer (12)
- : +- ^ FilterExecTransformer (11)
- : +- ^ Scan parquet (10)
- +- ^ InputIteratorTransformer (35)
- +- ShuffleQueryStage (33)
- +- ColumnarExchange (32)
- +- VeloxResizeBatches (31)
- +- ^ ProjectExecTransformer (29)
- +- ^ FilterExecTransformer (28)
- +- ^ Scan parquet (27)
+ VeloxColumnarToRow (42)
+ +- TakeOrderedAndProjectExecTransformer (41)
+ +- ^ ProjectExecTransformer (39)
+ +- ^ RegularHashAggregateExecTransformer (38)
+ +- ^ ProjectExecTransformer (37)
+ +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (36)
+ :- ^ InputIteratorTransformer (26)
+ : +- ShuffleQueryStage (24)
+ : +- ColumnarExchange (23)
+ : +- VeloxResizeBatches (22)
+ : +- ^ ProjectExecTransformer (20)
+ : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19)
+ : :- ^ InputIteratorTransformer (9)
+ : : +- ShuffleQueryStage (7)
+ : : +- ColumnarExchange (6)
+ : : +- VeloxResizeBatches (5)
+ : : +- ^ ProjectExecTransformer (3)
+ : : +- ^ FilterExecTransformer (2)
+ : : +- ^ ScanTransformer parquet (1)
+ : +- ^ InputIteratorTransformer (18)
+ : +- ShuffleQueryStage (16)
+ : +- ColumnarExchange (15)
+ : +- VeloxResizeBatches (14)
+ : +- ^ ProjectExecTransformer (12)
+ : +- ^ FilterExecTransformer (11)
+ : +- ^ ScanTransformer parquet (10)
+ +- ^ InputIteratorTransformer (35)
+ +- ShuffleQueryStage (33)
+ +- ColumnarExchange (32)
+ +- VeloxResizeBatches (31)
+ +- ^ ProjectExecTransformer (29)
+ +- ^ FilterExecTransformer (28)
+ +- ^ ScanTransformer parquet (27)
+- == Initial Plan ==
- TakeOrderedAndProject (66)
- +- HashAggregate (65)
- +- HashAggregate (64)
- +- Project (63)
- +- SortMergeJoin Inner (62)
- :- Sort (56)
- : +- Exchange (55)
- : +- Project (54)
- : +- SortMergeJoin Inner (53)
- : :- Sort (48)
- : : +- Exchange (47)
- : : +- Project (46)
- : : +- Filter (45)
- : : +- Scan parquet (44)
- : +- Sort (52)
- : +- Exchange (51)
- : +- Filter (50)
- : +- Scan parquet (49)
- +- Sort (61)
- +- Exchange (60)
- +- Project (59)
- +- Filter (58)
- +- Scan parquet (57)
-
-
-(1) Scan parquet
+ TakeOrderedAndProject (65)
+ +- HashAggregate (64)
+ +- HashAggregate (63)
+ +- Project (62)
+ +- SortMergeJoin Inner (61)
+ :- Sort (55)
+ : +- Exchange (54)
+ : +- Project (53)
+ : +- SortMergeJoin Inner (52)
+ : :- Sort (47)
+ : : +- Exchange (46)
+ : : +- Project (45)
+ : : +- Filter (44)
+ : : +- Scan parquet (43)
+ : +- Sort (51)
+ : +- Exchange (50)
+ : +- Filter (49)
+ : +- Scan parquet (48)
+ +- Sort (60)
+ +- Exchange (59)
+ +- Project (58)
+ +- Filter (57)
+ +- Scan parquet (56)
+
+
+(1) ScanTransformer parquet
Output [2]: [c_custkey#X, c_mktsegment#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -98,7 +97,7 @@ Input [1]: [c_custkey#X]
(9) InputIteratorTransformer
Input [1]: [c_custkey#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -166,7 +165,7 @@ Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X]
(26) InputIteratorTransformer
Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X]
-(27) Scan parquet
+(27) ScanTransformer parquet
Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -215,141 +214,134 @@ Input [6]: [o_orderkey#X, o_orderdate#X, o_shippriority#X, l_orderkey#X, l_exten
(38) RegularHashAggregateExecTransformer
Input [6]: [o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X, _pre_X#X]
Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X]
-Functions [1]: [partial_sum(_pre_X#X)]
-Aggregate Attributes [2]: [sum#X, isEmpty#X]
-Results [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X]
-
-(39) RegularHashAggregateExecTransformer
-Input [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X]
-Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X]
-Functions [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))]
+Functions [1]: [sum(_pre_X#X)]
Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X]
Results [4]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X]
-(40) ProjectExecTransformer
+(39) ProjectExecTransformer
Output [4]: [l_orderkey#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X AS revenue#X, o_orderdate#X, o_shippriority#X]
Input [4]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X]
-(41) WholeStageCodegenTransformer (X)
+(40) WholeStageCodegenTransformer (X)
Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X]
Arguments: false
-(42) TakeOrderedAndProjectExecTransformer
+(41) TakeOrderedAndProjectExecTransformer
Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X]
Arguments: X, [revenue#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X], 0
-(43) VeloxColumnarToRow
+(42) VeloxColumnarToRow
Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X]
-(44) Scan parquet
+(43) Scan parquet
Output [2]: [c_custkey#X, c_mktsegment#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(c_mktsegment), EqualTo(c_mktsegment,BUILDING), IsNotNull(c_custkey)]
ReadSchema: struct
-(45) Filter
+(44) Filter
Input [2]: [c_custkey#X, c_mktsegment#X]
Condition : ((isnotnull(c_mktsegment#X) AND (c_mktsegment#X = BUILDING)) AND isnotnull(c_custkey#X))
-(46) Project
+(45) Project
Output [1]: [c_custkey#X]
Input [2]: [c_custkey#X, c_mktsegment#X]
-(47) Exchange
+(46) Exchange
Input [1]: [c_custkey#X]
Arguments: hashpartitioning(c_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(48) Sort
+(47) Sort
Input [1]: [c_custkey#X]
Arguments: [c_custkey#X ASC NULLS FIRST], false, 0
-(49) Scan parquet
+(48) Scan parquet
Output [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(o_orderdate), LessThan(o_orderdate,1995-03-15), IsNotNull(o_custkey), IsNotNull(o_orderkey)]
ReadSchema: struct
-(50) Filter
+(49) Filter
Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X]
Condition : (((isnotnull(o_orderdate#X) AND (o_orderdate#X < 1995-03-15)) AND isnotnull(o_custkey#X)) AND isnotnull(o_orderkey#X))
-(51) Exchange
+(50) Exchange
Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X]
Arguments: hashpartitioning(o_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(52) Sort
+(51) Sort
Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X]
Arguments: [o_custkey#X ASC NULLS FIRST], false, 0
-(53) SortMergeJoin
+(52) SortMergeJoin
Left keys [1]: [c_custkey#X]
Right keys [1]: [o_custkey#X]
Join condition: None
-(54) Project
+(53) Project
Output [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X]
Input [5]: [c_custkey#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X]
-(55) Exchange
+(54) Exchange
Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X]
Arguments: hashpartitioning(o_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(56) Sort
+(55) Sort
Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X]
Arguments: [o_orderkey#X ASC NULLS FIRST], false, 0
-(57) Scan parquet
+(56) Scan parquet
Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(l_shipdate), GreaterThan(l_shipdate,1995-03-15), IsNotNull(l_orderkey)]
ReadSchema: struct
-(58) Filter
+(57) Filter
Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Condition : ((isnotnull(l_shipdate#X) AND (l_shipdate#X > 1995-03-15)) AND isnotnull(l_orderkey#X))
-(59) Project
+(58) Project
Output [3]: [l_orderkey#X, l_extendedprice#X, l_discount#X]
Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
-(60) Exchange
+(59) Exchange
Input [3]: [l_orderkey#X, l_extendedprice#X, l_discount#X]
Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(61) Sort
+(60) Sort
Input [3]: [l_orderkey#X, l_extendedprice#X, l_discount#X]
Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0
-(62) SortMergeJoin
+(61) SortMergeJoin
Left keys [1]: [o_orderkey#X]
Right keys [1]: [l_orderkey#X]
Join condition: None
-(63) Project
+(62) Project
Output [5]: [o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X]
Input [6]: [o_orderkey#X, o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X]
-(64) HashAggregate
+(63) HashAggregate
Input [5]: [o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X]
Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X]
Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))]
Aggregate Attributes [2]: [sum#X, isEmpty#X]
Results [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X]
-(65) HashAggregate
+(64) HashAggregate
Input [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X]
Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X]
Functions [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))]
Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X]
Results [4]: [l_orderkey#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X AS revenue#X, o_orderdate#X, o_shippriority#X]
-(66) TakeOrderedAndProject
+(65) TakeOrderedAndProject
Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X]
Arguments: X, [revenue#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X]
-(67) AdaptiveSparkPlan
+(66) AdaptiveSparkPlan
Output [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X]
Arguments: isFinalPlan=true
\ No newline at end of file
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 ffb35d1813ae..40cfa020ce73 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
@@ -22,14 +22,14 @@ AdaptiveSparkPlan (56)
: +- VeloxResizeBatches (5)
: +- ^ ProjectExecTransformer (3)
: +- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
+ : +- ^ ScanTransformer parquet (1)
+- ^ InputIteratorTransformer (18)
+- ShuffleQueryStage (16)
+- ColumnarExchange (15)
+- VeloxResizeBatches (14)
+- ^ ProjectExecTransformer (12)
+- ^ FilterExecTransformer (11)
- +- ^ Scan parquet (10)
+ +- ^ ScanTransformer parquet (10)
+- == Initial Plan ==
Sort (55)
+- Exchange (54)
@@ -50,7 +50,7 @@ AdaptiveSparkPlan (56)
+- Scan parquet (44)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -87,7 +87,7 @@ Input [2]: [o_orderkey#X, o_orderpriority#X]
(9) InputIteratorTransformer
Input [2]: [o_orderkey#X, o_orderpriority#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [3]: [l_orderkey#X, l_commitdate#X, l_receiptdate#X]
Batched: true
Location: InMemoryFileIndex [*]
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 0e48f73c7795..37faa880d8af 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
@@ -46,42 +46,42 @@ AdaptiveSparkPlan (156)
: : : : : +- VeloxResizeBatches (5)
: : : : : +- ^ ProjectExecTransformer (3)
: : : : : +- ^ FilterExecTransformer (2)
- : : : : : +- ^ Scan parquet (1)
+ : : : : : +- ^ ScanTransformer parquet (1)
: : : : +- ^ InputIteratorTransformer (18)
: : : : +- ShuffleQueryStage (16)
: : : : +- ColumnarExchange (15)
: : : : +- VeloxResizeBatches (14)
: : : : +- ^ ProjectExecTransformer (12)
: : : : +- ^ FilterExecTransformer (11)
- : : : : +- ^ Scan parquet (10)
+ : : : : +- ^ ScanTransformer parquet (10)
: : : +- ^ InputIteratorTransformer (35)
: : : +- ShuffleQueryStage (33)
: : : +- ColumnarExchange (32)
: : : +- VeloxResizeBatches (31)
: : : +- ^ ProjectExecTransformer (29)
: : : +- ^ FilterExecTransformer (28)
- : : : +- ^ Scan parquet (27)
+ : : : +- ^ ScanTransformer parquet (27)
: : +- ^ InputIteratorTransformer (52)
: : +- ShuffleQueryStage (50)
: : +- ColumnarExchange (49)
: : +- VeloxResizeBatches (48)
: : +- ^ ProjectExecTransformer (46)
: : +- ^ FilterExecTransformer (45)
- : : +- ^ Scan parquet (44)
+ : : +- ^ ScanTransformer parquet (44)
: +- ^ InputIteratorTransformer (69)
: +- ShuffleQueryStage (67)
: +- ColumnarExchange (66)
: +- VeloxResizeBatches (65)
: +- ^ ProjectExecTransformer (63)
: +- ^ FilterExecTransformer (62)
- : +- ^ Scan parquet (61)
+ : +- ^ ScanTransformer parquet (61)
+- ^ InputIteratorTransformer (86)
+- ShuffleQueryStage (84)
+- ColumnarExchange (83)
+- VeloxResizeBatches (82)
+- ^ ProjectExecTransformer (80)
+- ^ FilterExecTransformer (79)
- +- ^ Scan parquet (78)
+ +- ^ ScanTransformer parquet (78)
+- == Initial Plan ==
Sort (155)
+- Exchange (154)
@@ -134,7 +134,7 @@ AdaptiveSparkPlan (156)
+- Scan parquet (144)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [c_custkey#X, c_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -171,7 +171,7 @@ Input [2]: [c_custkey#X, c_nationkey#X]
(9) InputIteratorTransformer
Input [2]: [c_custkey#X, c_nationkey#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -239,7 +239,7 @@ Input [2]: [c_nationkey#X, o_orderkey#X]
(26) InputIteratorTransformer
Input [2]: [c_nationkey#X, o_orderkey#X]
-(27) Scan parquet
+(27) ScanTransformer parquet
Output [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -307,7 +307,7 @@ Input [4]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
(43) InputIteratorTransformer
Input [4]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
-(44) Scan parquet
+(44) ScanTransformer parquet
Output [2]: [s_suppkey#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -375,7 +375,7 @@ Input [3]: [l_extendedprice#X, l_discount#X, s_nationkey#X]
(60) InputIteratorTransformer
Input [3]: [l_extendedprice#X, l_discount#X, s_nationkey#X]
-(61) Scan parquet
+(61) ScanTransformer parquet
Output [3]: [n_nationkey#X, n_name#X, n_regionkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -443,7 +443,7 @@ Input [4]: [l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X]
(77) InputIteratorTransformer
Input [4]: [l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X]
-(78) Scan parquet
+(78) ScanTransformer parquet
Output [2]: [r_regionkey#X, r_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 a1830678d0de..73d9d734e78a 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
@@ -10,7 +10,7 @@ AdaptiveSparkPlan (20)
+- ^ FlushableHashAggregateExecTransformer (4)
+- ^ ProjectExecTransformer (3)
+- ^ FilterExecTransformer (2)
- +- ^ Scan parquet (1)
+ +- ^ ScanTransformer parquet (1)
+- == Initial Plan ==
HashAggregate (19)
+- Exchange (18)
@@ -20,7 +20,7 @@ AdaptiveSparkPlan (20)
+- Scan parquet (14)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [4]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
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 f089996aa4e3..f3ec7ef3a620 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
@@ -46,35 +46,35 @@ AdaptiveSparkPlan (149)
: : : : : +- VeloxResizeBatches (5)
: : : : : +- ^ ProjectExecTransformer (3)
: : : : : +- ^ FilterExecTransformer (2)
- : : : : : +- ^ Scan parquet (1)
+ : : : : : +- ^ ScanTransformer parquet (1)
: : : : +- ^ InputIteratorTransformer (18)
: : : : +- ShuffleQueryStage (16)
: : : : +- ColumnarExchange (15)
: : : : +- VeloxResizeBatches (14)
: : : : +- ^ ProjectExecTransformer (12)
: : : : +- ^ FilterExecTransformer (11)
- : : : : +- ^ Scan parquet (10)
+ : : : : +- ^ ScanTransformer parquet (10)
: : : +- ^ InputIteratorTransformer (35)
: : : +- ShuffleQueryStage (33)
: : : +- ColumnarExchange (32)
: : : +- VeloxResizeBatches (31)
: : : +- ^ ProjectExecTransformer (29)
: : : +- ^ FilterExecTransformer (28)
- : : : +- ^ Scan parquet (27)
+ : : : +- ^ ScanTransformer parquet (27)
: : +- ^ InputIteratorTransformer (52)
: : +- ShuffleQueryStage (50)
: : +- ColumnarExchange (49)
: : +- VeloxResizeBatches (48)
: : +- ^ ProjectExecTransformer (46)
: : +- ^ FilterExecTransformer (45)
- : : +- ^ Scan parquet (44)
+ : : +- ^ ScanTransformer parquet (44)
: +- ^ InputIteratorTransformer (69)
: +- ShuffleQueryStage (67)
: +- ColumnarExchange (66)
: +- VeloxResizeBatches (65)
: +- ^ ProjectExecTransformer (63)
: +- ^ FilterExecTransformer (62)
- : +- ^ Scan parquet (61)
+ : +- ^ ScanTransformer parquet (61)
+- ^ InputIteratorTransformer (81)
+- ShuffleQueryStage (79)
+- ReusedExchange (78)
@@ -128,7 +128,7 @@ AdaptiveSparkPlan (149)
+- Scan parquet (138)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [s_suppkey#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -165,7 +165,7 @@ Input [2]: [s_suppkey#X, s_nationkey#X]
(9) InputIteratorTransformer
Input [2]: [s_suppkey#X, s_nationkey#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [5]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -233,7 +233,7 @@ Input [5]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_ship
(26) InputIteratorTransformer
Input [5]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
-(27) Scan parquet
+(27) ScanTransformer parquet
Output [2]: [o_orderkey#X, o_custkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -301,7 +301,7 @@ Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_cust
(43) InputIteratorTransformer
Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X]
-(44) Scan parquet
+(44) ScanTransformer parquet
Output [2]: [c_custkey#X, c_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -369,7 +369,7 @@ Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nati
(60) InputIteratorTransformer
Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X]
-(61) Scan parquet
+(61) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 80b0d02a6cb2..8bd2945e4859 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
@@ -59,56 +59,56 @@ AdaptiveSparkPlan (207)
: : : : : : : +- VeloxResizeBatches (5)
: : : : : : : +- ^ ProjectExecTransformer (3)
: : : : : : : +- ^ FilterExecTransformer (2)
- : : : : : : : +- ^ Scan parquet (1)
+ : : : : : : : +- ^ ScanTransformer parquet (1)
: : : : : : +- ^ InputIteratorTransformer (18)
: : : : : : +- ShuffleQueryStage (16)
: : : : : : +- ColumnarExchange (15)
: : : : : : +- VeloxResizeBatches (14)
: : : : : : +- ^ ProjectExecTransformer (12)
: : : : : : +- ^ FilterExecTransformer (11)
- : : : : : : +- ^ Scan parquet (10)
+ : : : : : : +- ^ ScanTransformer parquet (10)
: : : : : +- ^ InputIteratorTransformer (35)
: : : : : +- ShuffleQueryStage (33)
: : : : : +- ColumnarExchange (32)
: : : : : +- VeloxResizeBatches (31)
: : : : : +- ^ ProjectExecTransformer (29)
: : : : : +- ^ FilterExecTransformer (28)
- : : : : : +- ^ Scan parquet (27)
+ : : : : : +- ^ ScanTransformer parquet (27)
: : : : +- ^ InputIteratorTransformer (52)
: : : : +- ShuffleQueryStage (50)
: : : : +- ColumnarExchange (49)
: : : : +- VeloxResizeBatches (48)
: : : : +- ^ ProjectExecTransformer (46)
: : : : +- ^ FilterExecTransformer (45)
- : : : : +- ^ Scan parquet (44)
+ : : : : +- ^ ScanTransformer parquet (44)
: : : +- ^ InputIteratorTransformer (69)
: : : +- ShuffleQueryStage (67)
: : : +- ColumnarExchange (66)
: : : +- VeloxResizeBatches (65)
: : : +- ^ ProjectExecTransformer (63)
: : : +- ^ FilterExecTransformer (62)
- : : : +- ^ Scan parquet (61)
+ : : : +- ^ ScanTransformer parquet (61)
: : +- ^ InputIteratorTransformer (86)
: : +- ShuffleQueryStage (84)
: : +- ColumnarExchange (83)
: : +- VeloxResizeBatches (82)
: : +- ^ ProjectExecTransformer (80)
: : +- ^ FilterExecTransformer (79)
- : : +- ^ Scan parquet (78)
+ : : +- ^ ScanTransformer parquet (78)
: +- ^ InputIteratorTransformer (103)
: +- ShuffleQueryStage (101)
: +- ColumnarExchange (100)
: +- VeloxResizeBatches (99)
: +- ^ ProjectExecTransformer (97)
: +- ^ FilterExecTransformer (96)
- : +- ^ Scan parquet (95)
+ : +- ^ ScanTransformer parquet (95)
+- ^ InputIteratorTransformer (120)
+- ShuffleQueryStage (118)
+- ColumnarExchange (117)
+- VeloxResizeBatches (116)
+- ^ ProjectExecTransformer (114)
+- ^ FilterExecTransformer (113)
- +- ^ Scan parquet (112)
+ +- ^ ScanTransformer parquet (112)
+- == Initial Plan ==
Sort (206)
+- Exchange (205)
@@ -177,7 +177,7 @@ AdaptiveSparkPlan (207)
+- Scan parquet (195)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [p_partkey#X, p_type#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -214,7 +214,7 @@ Input [1]: [p_partkey#X]
(9) InputIteratorTransformer
Input [1]: [p_partkey#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [5]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -282,7 +282,7 @@ Input [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
(26) InputIteratorTransformer
Input [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
-(27) Scan parquet
+(27) ScanTransformer parquet
Output [2]: [s_suppkey#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -350,7 +350,7 @@ Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X]
(43) InputIteratorTransformer
Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X]
-(44) Scan parquet
+(44) ScanTransformer parquet
Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -418,7 +418,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_order
(60) InputIteratorTransformer
Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X]
-(61) Scan parquet
+(61) ScanTransformer parquet
Output [2]: [c_custkey#X, c_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -486,7 +486,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nat
(77) InputIteratorTransformer
Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X]
-(78) Scan parquet
+(78) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_regionkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -554,7 +554,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_reg
(94) InputIteratorTransformer
Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X]
-(95) Scan parquet
+(95) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -622,7 +622,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_nam
(111) InputIteratorTransformer
Input [5]: [l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X]
-(112) Scan parquet
+(112) ScanTransformer parquet
Output [2]: [r_regionkey#X, r_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 a41822a60a19..6cca8ba5a479 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
@@ -46,42 +46,42 @@ AdaptiveSparkPlan (155)
: : : : : +- VeloxResizeBatches (5)
: : : : : +- ^ ProjectExecTransformer (3)
: : : : : +- ^ FilterExecTransformer (2)
- : : : : : +- ^ Scan parquet (1)
+ : : : : : +- ^ ScanTransformer parquet (1)
: : : : +- ^ InputIteratorTransformer (18)
: : : : +- ShuffleQueryStage (16)
: : : : +- ColumnarExchange (15)
: : : : +- VeloxResizeBatches (14)
: : : : +- ^ ProjectExecTransformer (12)
: : : : +- ^ FilterExecTransformer (11)
- : : : : +- ^ Scan parquet (10)
+ : : : : +- ^ ScanTransformer parquet (10)
: : : +- ^ InputIteratorTransformer (35)
: : : +- ShuffleQueryStage (33)
: : : +- ColumnarExchange (32)
: : : +- VeloxResizeBatches (31)
: : : +- ^ ProjectExecTransformer (29)
: : : +- ^ FilterExecTransformer (28)
- : : : +- ^ Scan parquet (27)
+ : : : +- ^ ScanTransformer parquet (27)
: : +- ^ InputIteratorTransformer (52)
: : +- ShuffleQueryStage (50)
: : +- ColumnarExchange (49)
: : +- VeloxResizeBatches (48)
: : +- ^ ProjectExecTransformer (46)
: : +- ^ FilterExecTransformer (45)
- : : +- ^ Scan parquet (44)
+ : : +- ^ ScanTransformer parquet (44)
: +- ^ InputIteratorTransformer (69)
: +- ShuffleQueryStage (67)
: +- ColumnarExchange (66)
: +- VeloxResizeBatches (65)
: +- ^ ProjectExecTransformer (63)
: +- ^ FilterExecTransformer (62)
- : +- ^ Scan parquet (61)
+ : +- ^ ScanTransformer parquet (61)
+- ^ InputIteratorTransformer (86)
+- ShuffleQueryStage (84)
+- ColumnarExchange (83)
+- VeloxResizeBatches (82)
+- ^ ProjectExecTransformer (80)
+- ^ FilterExecTransformer (79)
- +- ^ Scan parquet (78)
+ +- ^ ScanTransformer parquet (78)
+- == Initial Plan ==
Sort (154)
+- Exchange (153)
@@ -133,7 +133,7 @@ AdaptiveSparkPlan (155)
+- Scan parquet (144)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [p_partkey#X, p_name#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -170,7 +170,7 @@ Input [1]: [p_partkey#X]
(9) InputIteratorTransformer
Input [1]: [p_partkey#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -238,7 +238,7 @@ Input [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpric
(26) InputIteratorTransformer
Input [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X]
-(27) Scan parquet
+(27) ScanTransformer parquet
Output [2]: [s_suppkey#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -306,7 +306,7 @@ Input [7]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpric
(43) InputIteratorTransformer
Input [7]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X]
-(44) Scan parquet
+(44) ScanTransformer parquet
Output [3]: [ps_partkey#X, ps_suppkey#X, ps_supplycost#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -374,7 +374,7 @@ Input [6]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_natio
(60) InputIteratorTransformer
Input [6]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X]
-(61) Scan parquet
+(61) ScanTransformer parquet
Output [2]: [o_orderkey#X, o_orderdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -442,7 +442,7 @@ Input [6]: [l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_sup
(77) InputIteratorTransformer
Input [6]: [l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X]
-(78) Scan parquet
+(78) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 0851fa2447ea..ca3b50ab05c5 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
@@ -16,7 +16,7 @@ AdaptiveSparkPlan (30)
+- ^ FlushableHashAggregateExecTransformer (4)
+- ^ ProjectExecTransformer (3)
+- ^ FilterExecTransformer (2)
- +- ^ Scan parquet (1)
+ +- ^ ScanTransformer parquet (1)
+- == Initial Plan ==
Sort (29)
+- Exchange (28)
@@ -28,7 +28,7 @@ AdaptiveSparkPlan (30)
+- Scan parquet (22)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [7]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_tax#X, l_returnflag#X, l_linestatus#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
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 1b89e64860f3..a14969cbc44b 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
@@ -31,28 +31,28 @@ AdaptiveSparkPlan (100)
: : : +- VeloxResizeBatches (5)
: : : +- ^ ProjectExecTransformer (3)
: : : +- ^ FilterExecTransformer (2)
- : : : +- ^ Scan parquet (1)
+ : : : +- ^ ScanTransformer parquet (1)
: : +- ^ InputIteratorTransformer (18)
: : +- ShuffleQueryStage (16), Statistics(X)
: : +- ColumnarExchange (15)
: : +- VeloxResizeBatches (14)
: : +- ^ ProjectExecTransformer (12)
: : +- ^ FilterExecTransformer (11)
- : : +- ^ Scan parquet (10)
+ : : +- ^ ScanTransformer parquet (10)
: +- ^ InputIteratorTransformer (35)
: +- ShuffleQueryStage (33), Statistics(X)
: +- ColumnarExchange (32)
: +- VeloxResizeBatches (31)
: +- ^ ProjectExecTransformer (29)
: +- ^ FilterExecTransformer (28)
- : +- ^ Scan parquet (27)
+ : +- ^ ScanTransformer parquet (27)
+- ^ InputIteratorTransformer (52)
+- ShuffleQueryStage (50), Statistics(X)
+- ColumnarExchange (49)
+- VeloxResizeBatches (48)
+- ^ ProjectExecTransformer (46)
+- ^ FilterExecTransformer (45)
- +- ^ Scan parquet (44)
+ +- ^ ScanTransformer parquet (44)
+- == Initial Plan ==
TakeOrderedAndProject (99)
+- HashAggregate (98)
@@ -88,7 +88,7 @@ AdaptiveSparkPlan (100)
+- Scan parquet (90)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -125,7 +125,7 @@ Input [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct
(9) InputIteratorTransformer
Input [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -193,7 +193,7 @@ Input [8]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct
(26) InputIteratorTransformer
Input [8]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, o_orderkey#X]
-(27) Scan parquet
+(27) ScanTransformer parquet
Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_returnflag#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -261,7 +261,7 @@ Input [9]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct
(43) InputIteratorTransformer
Input [9]: [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]
-(44) Scan parquet
+(44) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 6ca992b70988..55d544b898c6 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
@@ -29,21 +29,21 @@ AdaptiveSparkPlan (82)
: : +- VeloxResizeBatches (5)
: : +- ^ ProjectExecTransformer (3)
: : +- ^ FilterExecTransformer (2)
- : : +- ^ Scan parquet (1)
+ : : +- ^ ScanTransformer parquet (1)
: +- ^ InputIteratorTransformer (18)
: +- ShuffleQueryStage (16), Statistics(X)
: +- ColumnarExchange (15)
: +- VeloxResizeBatches (14)
: +- ^ ProjectExecTransformer (12)
: +- ^ FilterExecTransformer (11)
- : +- ^ Scan parquet (10)
+ : +- ^ ScanTransformer parquet (10)
+- ^ InputIteratorTransformer (35)
+- ShuffleQueryStage (33), Statistics(X)
+- ColumnarExchange (32)
+- VeloxResizeBatches (31)
+- ^ ProjectExecTransformer (29)
+- ^ FilterExecTransformer (28)
- +- ^ Scan parquet (27)
+ +- ^ ScanTransformer parquet (27)
+- == Initial Plan ==
Sort (81)
+- Exchange (80)
@@ -72,7 +72,7 @@ AdaptiveSparkPlan (82)
+- Scan parquet (69)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -109,7 +109,7 @@ Input [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X]
(9) InputIteratorTransformer
Input [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [2]: [s_suppkey#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -177,7 +177,7 @@ Input [4]: [ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X]
(26) InputIteratorTransformer
Input [4]: [ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X]
-(27) Scan parquet
+(27) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -424,58 +424,57 @@ Arguments: isFinalPlan=true
===== Subqueries =====
Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery subquery#X, [id=#X]
-AdaptiveSparkPlan (136)
+AdaptiveSparkPlan (135)
+- == Final Plan ==
- VeloxColumnarToRow (114)
- +- ^ ProjectExecTransformer (112)
- +- ^ RegularHashAggregateExecTransformer (111)
- +- ^ RegularHashAggregateExecTransformer (110)
- +- ^ ProjectExecTransformer (109)
- +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (108)
- :- ^ InputIteratorTransformer (103)
- : +- ShuffleQueryStage (101), Statistics(X)
- : +- ColumnarExchange (100)
- : +- VeloxResizeBatches (99)
- : +- ^ ProjectExecTransformer (97)
- : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (96)
- : :- ^ InputIteratorTransformer (91)
- : : +- ShuffleQueryStage (89), Statistics(X)
- : : +- ColumnarExchange (88)
- : : +- VeloxResizeBatches (87)
- : : +- ^ ProjectExecTransformer (85)
- : : +- ^ FilterExecTransformer (84)
- : : +- ^ Scan parquet (83)
- : +- ^ InputIteratorTransformer (95)
- : +- ShuffleQueryStage (93), Statistics(X)
- : +- ReusedExchange (92)
- +- ^ InputIteratorTransformer (107)
- +- ShuffleQueryStage (105), Statistics(X)
- +- ReusedExchange (104)
+ VeloxColumnarToRow (113)
+ +- ^ ProjectExecTransformer (111)
+ +- ^ RegularHashAggregateExecTransformer (110)
+ +- ^ ProjectExecTransformer (109)
+ +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (108)
+ :- ^ InputIteratorTransformer (103)
+ : +- ShuffleQueryStage (101), Statistics(X)
+ : +- ColumnarExchange (100)
+ : +- VeloxResizeBatches (99)
+ : +- ^ ProjectExecTransformer (97)
+ : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (96)
+ : :- ^ InputIteratorTransformer (91)
+ : : +- ShuffleQueryStage (89), Statistics(X)
+ : : +- ColumnarExchange (88)
+ : : +- VeloxResizeBatches (87)
+ : : +- ^ ProjectExecTransformer (85)
+ : : +- ^ FilterExecTransformer (84)
+ : : +- ^ ScanTransformer parquet (83)
+ : +- ^ InputIteratorTransformer (95)
+ : +- ShuffleQueryStage (93), Statistics(X)
+ : +- ReusedExchange (92)
+ +- ^ InputIteratorTransformer (107)
+ +- ShuffleQueryStage (105), Statistics(X)
+ +- ReusedExchange (104)
+- == Initial Plan ==
- HashAggregate (135)
- +- HashAggregate (134)
- +- Project (133)
- +- SortMergeJoin Inner (132)
- :- Sort (126)
- : +- Exchange (125)
- : +- Project (124)
- : +- SortMergeJoin Inner (123)
- : :- Sort (118)
- : : +- Exchange (117)
- : : +- Filter (116)
- : : +- Scan parquet (115)
- : +- Sort (122)
- : +- Exchange (121)
- : +- Filter (120)
- : +- Scan parquet (119)
- +- Sort (131)
- +- Exchange (130)
- +- Project (129)
- +- Filter (128)
- +- Scan parquet (127)
-
-
-(83) Scan parquet
+ HashAggregate (134)
+ +- HashAggregate (133)
+ +- Project (132)
+ +- SortMergeJoin Inner (131)
+ :- Sort (125)
+ : +- Exchange (124)
+ : +- Project (123)
+ : +- SortMergeJoin Inner (122)
+ : :- Sort (117)
+ : : +- Exchange (116)
+ : : +- Filter (115)
+ : : +- Scan parquet (114)
+ : +- Sort (121)
+ : +- Exchange (120)
+ : +- Filter (119)
+ : +- Scan parquet (118)
+ +- Sort (130)
+ +- Exchange (129)
+ +- Project (128)
+ +- Filter (127)
+ +- Scan parquet (126)
+
+
+(83) ScanTransformer parquet
Output [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -581,129 +580,122 @@ Input [4]: [ps_availqty#X, ps_supplycost#X, s_nationkey#X, n_nationkey#X]
(110) RegularHashAggregateExecTransformer
Input [3]: [ps_availqty#X, ps_supplycost#X, _pre_X#X]
Keys: []
-Functions [1]: [partial_sum(_pre_X#X)]
-Aggregate Attributes [2]: [sum#X, isEmpty#X]
-Results [2]: [sum#X, isEmpty#X]
-
-(111) RegularHashAggregateExecTransformer
-Input [2]: [sum#X, isEmpty#X]
-Keys: []
-Functions [1]: [sum(CheckOverflow((promote_precision(ps_supplycost#X) * promote_precision(cast(ps_availqty#X as decimal(12,2)))), DecimalType(23,2)))]
+Functions [1]: [sum(_pre_X#X)]
Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(ps_supplycost#X) * promote_precision(cast(ps_availqty#X as decimal(12,2)))), DecimalType(23,2)))#X]
Results [1]: [sum(CheckOverflow((promote_precision(ps_supplycost#X) * promote_precision(cast(ps_availqty#X as decimal(12,2)))), DecimalType(23,2)))#X]
-(112) ProjectExecTransformer
+(111) ProjectExecTransformer
Output [1]: [CheckOverflow((promote_precision(cast(sum(CheckOverflow((promote_precision(ps_supplycost#X) * promote_precision(cast(ps_availqty#X as decimal(12,2)))), DecimalType(23,2)))#X as decimal(38,10))) * 0.0001000000), DecimalType(38,6)) AS (sum((ps_supplycost * ps_availqty)) * 0.0001000000)#X]
Input [1]: [sum(CheckOverflow((promote_precision(ps_supplycost#X) * promote_precision(cast(ps_availqty#X as decimal(12,2)))), DecimalType(23,2)))#X]
-(113) WholeStageCodegenTransformer (X)
+(112) WholeStageCodegenTransformer (X)
Input [1]: [(sum((ps_supplycost * ps_availqty)) * 0.0001000000)#X]
Arguments: false
-(114) VeloxColumnarToRow
+(113) VeloxColumnarToRow
Input [1]: [(sum((ps_supplycost * ps_availqty)) * 0.0001000000)#X]
-(115) Scan parquet
+(114) Scan parquet
Output [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(ps_suppkey)]
ReadSchema: struct
-(116) Filter
+(115) Filter
Input [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X]
Condition : isnotnull(ps_suppkey#X)
-(117) Exchange
+(116) Exchange
Input [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X]
Arguments: hashpartitioning(ps_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(118) Sort
+(117) Sort
Input [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X]
Arguments: [ps_suppkey#X ASC NULLS FIRST], false, 0
-(119) Scan parquet
+(118) Scan parquet
Output [2]: [s_suppkey#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(s_suppkey), IsNotNull(s_nationkey)]
ReadSchema: struct
-(120) Filter
+(119) Filter
Input [2]: [s_suppkey#X, s_nationkey#X]
Condition : (isnotnull(s_suppkey#X) AND isnotnull(s_nationkey#X))
-(121) Exchange
+(120) Exchange
Input [2]: [s_suppkey#X, s_nationkey#X]
Arguments: hashpartitioning(s_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(122) Sort
+(121) Sort
Input [2]: [s_suppkey#X, s_nationkey#X]
Arguments: [s_suppkey#X ASC NULLS FIRST], false, 0
-(123) SortMergeJoin
+(122) SortMergeJoin
Left keys [1]: [ps_suppkey#X]
Right keys [1]: [s_suppkey#X]
Join condition: None
-(124) Project
+(123) Project
Output [3]: [ps_availqty#X, ps_supplycost#X, s_nationkey#X]
Input [5]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X, s_suppkey#X, s_nationkey#X]
-(125) Exchange
+(124) Exchange
Input [3]: [ps_availqty#X, ps_supplycost#X, s_nationkey#X]
Arguments: hashpartitioning(s_nationkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(126) Sort
+(125) Sort
Input [3]: [ps_availqty#X, ps_supplycost#X, s_nationkey#X]
Arguments: [s_nationkey#X ASC NULLS FIRST], false, 0
-(127) Scan parquet
+(126) Scan parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(n_name), EqualTo(n_name,GERMANY), IsNotNull(n_nationkey)]
ReadSchema: struct
-(128) Filter
+(127) Filter
Input [2]: [n_nationkey#X, n_name#X]
Condition : ((isnotnull(n_name#X) AND (n_name#X = GERMANY)) AND isnotnull(n_nationkey#X))
-(129) Project
+(128) Project
Output [1]: [n_nationkey#X]
Input [2]: [n_nationkey#X, n_name#X]
-(130) Exchange
+(129) Exchange
Input [1]: [n_nationkey#X]
Arguments: hashpartitioning(n_nationkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(131) Sort
+(130) Sort
Input [1]: [n_nationkey#X]
Arguments: [n_nationkey#X ASC NULLS FIRST], false, 0
-(132) SortMergeJoin
+(131) SortMergeJoin
Left keys [1]: [s_nationkey#X]
Right keys [1]: [n_nationkey#X]
Join condition: None
-(133) Project
+(132) Project
Output [2]: [ps_availqty#X, ps_supplycost#X]
Input [4]: [ps_availqty#X, ps_supplycost#X, s_nationkey#X, n_nationkey#X]
-(134) HashAggregate
+(133) HashAggregate
Input [2]: [ps_availqty#X, ps_supplycost#X]
Keys: []
Functions [1]: [partial_sum(CheckOverflow((promote_precision(ps_supplycost#X) * promote_precision(cast(ps_availqty#X as decimal(12,2)))), DecimalType(23,2)))]
Aggregate Attributes [2]: [sum#X, isEmpty#X]
Results [2]: [sum#X, isEmpty#X]
-(135) HashAggregate
+(134) HashAggregate
Input [2]: [sum#X, isEmpty#X]
Keys: []
Functions [1]: [sum(CheckOverflow((promote_precision(ps_supplycost#X) * promote_precision(cast(ps_availqty#X as decimal(12,2)))), DecimalType(23,2)))]
Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(ps_supplycost#X) * promote_precision(cast(ps_availqty#X as decimal(12,2)))), DecimalType(23,2)))#X]
Results [1]: [CheckOverflow((promote_precision(cast(sum(CheckOverflow((promote_precision(ps_supplycost#X) * promote_precision(cast(ps_availqty#X as decimal(12,2)))), DecimalType(23,2)))#X as decimal(38,10))) * 0.0001000000), DecimalType(38,6)) AS (sum((ps_supplycost * ps_availqty)) * 0.0001000000)#X]
-(136) AdaptiveSparkPlan
+(135) AdaptiveSparkPlan
Output [1]: [(sum((ps_supplycost * ps_availqty)) * 0.0001000000)#X]
Arguments: isFinalPlan=true
\ No newline at end of file
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 346a6866342e..f998361cddc5 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
@@ -22,14 +22,14 @@ AdaptiveSparkPlan (55)
: +- VeloxResizeBatches (5)
: +- ^ ProjectExecTransformer (3)
: +- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
+ : +- ^ ScanTransformer parquet (1)
+- ^ InputIteratorTransformer (18)
+- ShuffleQueryStage (16), Statistics(X)
+- ColumnarExchange (15)
+- VeloxResizeBatches (14)
+- ^ ProjectExecTransformer (12)
+- ^ FilterExecTransformer (11)
- +- ^ Scan parquet (10)
+ +- ^ ScanTransformer parquet (10)
+- == Initial Plan ==
Sort (54)
+- Exchange (53)
@@ -49,7 +49,7 @@ AdaptiveSparkPlan (55)
+- Scan parquet (43)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [o_orderkey#X, o_orderpriority#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -86,7 +86,7 @@ Input [2]: [o_orderkey#X, o_orderpriority#X]
(9) InputIteratorTransformer
Input [2]: [o_orderkey#X, o_orderpriority#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [5]: [l_orderkey#X, l_shipdate#X, l_commitdate#X, l_receiptdate#X, l_shipmode#X]
Batched: true
Location: InMemoryFileIndex [*]
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 9e927782f6c3..7c3509c3e726 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
@@ -1,58 +1,57 @@
== Physical Plan ==
-AdaptiveSparkPlan (58)
+AdaptiveSparkPlan (57)
+- == Final Plan ==
- VeloxColumnarToRow (40)
- +- ^ SortExecTransformer (38)
- +- ^ InputIteratorTransformer (37)
- +- ShuffleQueryStage (35), Statistics(X)
- +- ColumnarExchange (34)
- +- VeloxResizeBatches (33)
- +- ^ RegularHashAggregateExecTransformer (31)
- +- ^ InputIteratorTransformer (30)
- +- ShuffleQueryStage (28), Statistics(X)
- +- ColumnarExchange (27)
- +- VeloxResizeBatches (26)
- +- ^ ProjectExecTransformer (24)
- +- ^ FlushableHashAggregateExecTransformer (23)
- +- ^ ProjectExecTransformer (22)
- +- ^ RegularHashAggregateExecTransformer (21)
- +- ^ RegularHashAggregateExecTransformer (20)
- +- ^ ProjectExecTransformer (19)
- +- ^ ShuffledHashJoinExecTransformer LeftOuter BuildLeft (18)
- :- ^ InputIteratorTransformer (8)
- : +- ShuffleQueryStage (6), Statistics(X)
- : +- ColumnarExchange (5)
- : +- VeloxResizeBatches (4)
- : +- ^ ProjectExecTransformer (2)
- : +- ^ Scan parquet (1)
- +- ^ InputIteratorTransformer (17)
- +- ShuffleQueryStage (15), Statistics(X)
- +- ColumnarExchange (14)
- +- VeloxResizeBatches (13)
- +- ^ ProjectExecTransformer (11)
- +- ^ FilterExecTransformer (10)
- +- ^ Scan parquet (9)
+ VeloxColumnarToRow (39)
+ +- ^ SortExecTransformer (37)
+ +- ^ InputIteratorTransformer (36)
+ +- ShuffleQueryStage (34), Statistics(X)
+ +- ColumnarExchange (33)
+ +- VeloxResizeBatches (32)
+ +- ^ RegularHashAggregateExecTransformer (30)
+ +- ^ InputIteratorTransformer (29)
+ +- ShuffleQueryStage (27), Statistics(X)
+ +- ColumnarExchange (26)
+ +- VeloxResizeBatches (25)
+ +- ^ ProjectExecTransformer (23)
+ +- ^ FlushableHashAggregateExecTransformer (22)
+ +- ^ ProjectExecTransformer (21)
+ +- ^ RegularHashAggregateExecTransformer (20)
+ +- ^ ProjectExecTransformer (19)
+ +- ^ ShuffledHashJoinExecTransformer LeftOuter BuildLeft (18)
+ :- ^ InputIteratorTransformer (8)
+ : +- ShuffleQueryStage (6), Statistics(X)
+ : +- ColumnarExchange (5)
+ : +- VeloxResizeBatches (4)
+ : +- ^ ProjectExecTransformer (2)
+ : +- ^ ScanTransformer parquet (1)
+ +- ^ InputIteratorTransformer (17)
+ +- ShuffleQueryStage (15), Statistics(X)
+ +- ColumnarExchange (14)
+ +- VeloxResizeBatches (13)
+ +- ^ ProjectExecTransformer (11)
+ +- ^ FilterExecTransformer (10)
+ +- ^ ScanTransformer parquet (9)
+- == Initial Plan ==
- Sort (57)
- +- Exchange (56)
- +- HashAggregate (55)
- +- Exchange (54)
- +- HashAggregate (53)
- +- HashAggregate (52)
- +- HashAggregate (51)
- +- Project (50)
- +- SortMergeJoin LeftOuter (49)
- :- Sort (43)
- : +- Exchange (42)
- : +- Scan parquet (41)
- +- Sort (48)
- +- Exchange (47)
- +- Project (46)
- +- Filter (45)
- +- Scan parquet (44)
-
-
-(1) Scan parquet
+ Sort (56)
+ +- Exchange (55)
+ +- HashAggregate (54)
+ +- Exchange (53)
+ +- HashAggregate (52)
+ +- HashAggregate (51)
+ +- HashAggregate (50)
+ +- Project (49)
+ +- SortMergeJoin LeftOuter (48)
+ :- Sort (42)
+ : +- Exchange (41)
+ : +- Scan parquet (40)
+ +- Sort (47)
+ +- Exchange (46)
+ +- Project (45)
+ +- Filter (44)
+ +- Scan parquet (43)
+
+
+(1) ScanTransformer parquet
Output [1]: [c_custkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -84,7 +83,7 @@ Input [1]: [c_custkey#X]
(8) InputIteratorTransformer
Input [1]: [c_custkey#X]
-(9) Scan parquet
+(9) ScanTransformer parquet
Output [3]: [o_orderkey#X, o_custkey#X, o_comment#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -133,180 +132,173 @@ Input [3]: [c_custkey#X, o_orderkey#X, o_custkey#X]
(20) RegularHashAggregateExecTransformer
Input [2]: [c_custkey#X, o_orderkey#X]
Keys [1]: [c_custkey#X]
-Functions [1]: [partial_count(o_orderkey#X)]
-Aggregate Attributes [1]: [count#X]
-Results [2]: [c_custkey#X, count#X]
-
-(21) RegularHashAggregateExecTransformer
-Input [2]: [c_custkey#X, count#X]
-Keys [1]: [c_custkey#X]
Functions [1]: [count(o_orderkey#X)]
Aggregate Attributes [1]: [count(o_orderkey#X)#X]
Results [2]: [c_custkey#X, count(o_orderkey#X)#X]
-(22) ProjectExecTransformer
+(21) ProjectExecTransformer
Output [1]: [count(o_orderkey#X)#X AS c_count#X]
Input [2]: [c_custkey#X, count(o_orderkey#X)#X]
-(23) FlushableHashAggregateExecTransformer
+(22) FlushableHashAggregateExecTransformer
Input [1]: [c_count#X]
Keys [1]: [c_count#X]
Functions [1]: [partial_count(1)]
Aggregate Attributes [1]: [count#X]
Results [2]: [c_count#X, count#X]
-(24) ProjectExecTransformer
+(23) ProjectExecTransformer
Output [3]: [hash(c_count#X, 42) AS hash_partition_key#X, c_count#X, count#X]
Input [2]: [c_count#X, count#X]
-(25) WholeStageCodegenTransformer (X)
+(24) WholeStageCodegenTransformer (X)
Input [3]: [hash_partition_key#X, c_count#X, count#X]
Arguments: false
-(26) VeloxResizeBatches
+(25) VeloxResizeBatches
Input [3]: [hash_partition_key#X, c_count#X, count#X]
Arguments: X, X
-(27) ColumnarExchange
+(26) ColumnarExchange
Input [3]: [hash_partition_key#X, c_count#X, count#X]
Arguments: hashpartitioning(c_count#X, 1), ENSURE_REQUIREMENTS, [c_count#X, count#X], [plan_id=X], [shuffle_writer_type=hash]
-(28) ShuffleQueryStage
+(27) ShuffleQueryStage
Output [2]: [c_count#X, count#X]
Arguments: X
-(29) InputAdapter
+(28) InputAdapter
Input [2]: [c_count#X, count#X]
-(30) InputIteratorTransformer
+(29) InputIteratorTransformer
Input [2]: [c_count#X, count#X]
-(31) RegularHashAggregateExecTransformer
+(30) RegularHashAggregateExecTransformer
Input [2]: [c_count#X, count#X]
Keys [1]: [c_count#X]
Functions [1]: [count(1)]
Aggregate Attributes [1]: [count(1)#X]
Results [2]: [c_count#X, count(1)#X AS custdist#X]
-(32) WholeStageCodegenTransformer (X)
+(31) WholeStageCodegenTransformer (X)
Input [2]: [c_count#X, custdist#X]
Arguments: false
-(33) VeloxResizeBatches
+(32) VeloxResizeBatches
Input [2]: [c_count#X, custdist#X]
Arguments: X, X
-(34) ColumnarExchange
+(33) ColumnarExchange
Input [2]: [c_count#X, custdist#X]
Arguments: rangepartitioning(custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST, 1), ENSURE_REQUIREMENTS, [plan_id=X], [shuffle_writer_type=hash]
-(35) ShuffleQueryStage
+(34) ShuffleQueryStage
Output [2]: [c_count#X, custdist#X]
Arguments: X
-(36) InputAdapter
+(35) InputAdapter
Input [2]: [c_count#X, custdist#X]
-(37) InputIteratorTransformer
+(36) InputIteratorTransformer
Input [2]: [c_count#X, custdist#X]
-(38) SortExecTransformer
+(37) SortExecTransformer
Input [2]: [c_count#X, custdist#X]
Arguments: [custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST], true, 0
-(39) WholeStageCodegenTransformer (X)
+(38) WholeStageCodegenTransformer (X)
Input [2]: [c_count#X, custdist#X]
Arguments: false
-(40) VeloxColumnarToRow
+(39) VeloxColumnarToRow
Input [2]: [c_count#X, custdist#X]
-(41) Scan parquet
+(40) Scan parquet
Output [1]: [c_custkey#X]
Batched: true
Location: InMemoryFileIndex [*]
ReadSchema: struct
-(42) Exchange
+(41) Exchange
Input [1]: [c_custkey#X]
Arguments: hashpartitioning(c_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(43) Sort
+(42) Sort
Input [1]: [c_custkey#X]
Arguments: [c_custkey#X ASC NULLS FIRST], false, 0
-(44) Scan parquet
+(43) Scan parquet
Output [3]: [o_orderkey#X, o_custkey#X, o_comment#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(o_comment), IsNotNull(o_custkey)]
ReadSchema: struct
-(45) Filter
+(44) Filter
Input [3]: [o_orderkey#X, o_custkey#X, o_comment#X]
Condition : ((isnotnull(o_comment#X) AND NOT o_comment#X LIKE %special%requests%) AND isnotnull(o_custkey#X))
-(46) Project
+(45) Project
Output [2]: [o_orderkey#X, o_custkey#X]
Input [3]: [o_orderkey#X, o_custkey#X, o_comment#X]
-(47) Exchange
+(46) Exchange
Input [2]: [o_orderkey#X, o_custkey#X]
Arguments: hashpartitioning(o_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(48) Sort
+(47) Sort
Input [2]: [o_orderkey#X, o_custkey#X]
Arguments: [o_custkey#X ASC NULLS FIRST], false, 0
-(49) SortMergeJoin
+(48) SortMergeJoin
Left keys [1]: [c_custkey#X]
Right keys [1]: [o_custkey#X]
Join condition: None
-(50) Project
+(49) Project
Output [2]: [c_custkey#X, o_orderkey#X]
Input [3]: [c_custkey#X, o_orderkey#X, o_custkey#X]
-(51) HashAggregate
+(50) HashAggregate
Input [2]: [c_custkey#X, o_orderkey#X]
Keys [1]: [c_custkey#X]
Functions [1]: [partial_count(o_orderkey#X)]
Aggregate Attributes [1]: [count#X]
Results [2]: [c_custkey#X, count#X]
-(52) HashAggregate
+(51) HashAggregate
Input [2]: [c_custkey#X, count#X]
Keys [1]: [c_custkey#X]
Functions [1]: [count(o_orderkey#X)]
Aggregate Attributes [1]: [count(o_orderkey#X)#X]
Results [1]: [count(o_orderkey#X)#X AS c_count#X]
-(53) HashAggregate
+(52) HashAggregate
Input [1]: [c_count#X]
Keys [1]: [c_count#X]
Functions [1]: [partial_count(1)]
Aggregate Attributes [1]: [count#X]
Results [2]: [c_count#X, count#X]
-(54) Exchange
+(53) Exchange
Input [2]: [c_count#X, count#X]
Arguments: hashpartitioning(c_count#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(55) HashAggregate
+(54) HashAggregate
Input [2]: [c_count#X, count#X]
Keys [1]: [c_count#X]
Functions [1]: [count(1)]
Aggregate Attributes [1]: [count(1)#X]
Results [2]: [c_count#X, count(1)#X AS custdist#X]
-(56) Exchange
+(55) Exchange
Input [2]: [c_count#X, custdist#X]
Arguments: rangepartitioning(custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(57) Sort
+(56) Sort
Input [2]: [c_count#X, custdist#X]
Arguments: [custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST], true, 0
-(58) AdaptiveSparkPlan
+(57) AdaptiveSparkPlan
Output [2]: [c_count#X, custdist#X]
Arguments: isFinalPlan=true
\ No newline at end of file
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 d82e4f2b21db..1161b32937bc 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
@@ -1,43 +1,42 @@
== Physical Plan ==
-AdaptiveSparkPlan (39)
+AdaptiveSparkPlan (38)
+- == Final Plan ==
- VeloxColumnarToRow (25)
- +- ^ ProjectExecTransformer (23)
- +- ^ RegularHashAggregateExecTransformer (22)
- +- ^ RegularHashAggregateExecTransformer (21)
- +- ^ ProjectExecTransformer (20)
- +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19)
- :- ^ InputIteratorTransformer (9)
- : +- ShuffleQueryStage (7), Statistics(X)
- : +- ColumnarExchange (6)
- : +- VeloxResizeBatches (5)
- : +- ^ ProjectExecTransformer (3)
- : +- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
- +- ^ InputIteratorTransformer (18)
- +- ShuffleQueryStage (16), Statistics(X)
- +- ColumnarExchange (15)
- +- VeloxResizeBatches (14)
- +- ^ ProjectExecTransformer (12)
- +- ^ FilterExecTransformer (11)
- +- ^ Scan parquet (10)
+ VeloxColumnarToRow (24)
+ +- ^ ProjectExecTransformer (22)
+ +- ^ RegularHashAggregateExecTransformer (21)
+ +- ^ ProjectExecTransformer (20)
+ +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19)
+ :- ^ InputIteratorTransformer (9)
+ : +- ShuffleQueryStage (7), Statistics(X)
+ : +- ColumnarExchange (6)
+ : +- VeloxResizeBatches (5)
+ : +- ^ ProjectExecTransformer (3)
+ : +- ^ FilterExecTransformer (2)
+ : +- ^ ScanTransformer parquet (1)
+ +- ^ InputIteratorTransformer (18)
+ +- ShuffleQueryStage (16), Statistics(X)
+ +- ColumnarExchange (15)
+ +- VeloxResizeBatches (14)
+ +- ^ ProjectExecTransformer (12)
+ +- ^ FilterExecTransformer (11)
+ +- ^ ScanTransformer parquet (10)
+- == Initial Plan ==
- HashAggregate (38)
- +- HashAggregate (37)
- +- Project (36)
- +- SortMergeJoin Inner (35)
- :- Sort (30)
- : +- Exchange (29)
- : +- Project (28)
- : +- Filter (27)
- : +- Scan parquet (26)
- +- Sort (34)
- +- Exchange (33)
- +- Filter (32)
- +- Scan parquet (31)
-
-
-(1) Scan parquet
+ HashAggregate (37)
+ +- HashAggregate (36)
+ +- Project (35)
+ +- SortMergeJoin Inner (34)
+ :- Sort (29)
+ : +- Exchange (28)
+ : +- Project (27)
+ : +- Filter (26)
+ : +- Scan parquet (25)
+ +- Sort (33)
+ +- Exchange (32)
+ +- Filter (31)
+ +- Scan parquet (30)
+
+
+(1) ScanTransformer parquet
Output [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -74,7 +73,7 @@ Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X]
(9) InputIteratorTransformer
Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [2]: [p_partkey#X, p_type#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -123,93 +122,86 @@ Input [5]: [l_partkey#X, l_extendedprice#X, l_discount#X, p_partkey#X, p_type#X]
(21) RegularHashAggregateExecTransformer
Input [5]: [l_extendedprice#X, l_discount#X, p_type#X, _pre_X#X, _pre_X#X]
Keys: []
-Functions [2]: [partial_sum(_pre_X#X), partial_sum(_pre_X#X)]
-Aggregate Attributes [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X]
-Results [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X]
-
-(22) RegularHashAggregateExecTransformer
-Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X]
-Keys: []
-Functions [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)) ELSE 0.0000 END), sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))]
+Functions [2]: [sum(_pre_X#X), sum(_pre_X#X)]
Aggregate Attributes [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)) ELSE 0.0000 END)#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X]
Results [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)) ELSE 0.0000 END)#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X]
-(23) ProjectExecTransformer
+(22) ProjectExecTransformer
Output [1]: [CheckOverflow((promote_precision(CheckOverflow((100.0000 * promote_precision(sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)) ELSE 0.0000 END)#X)), DecimalType(38,6))) / promote_precision(cast(sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X as decimal(38,6)))), DecimalType(38,6)) AS promo_revenue#X]
Input [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)) ELSE 0.0000 END)#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X]
-(24) WholeStageCodegenTransformer (X)
+(23) WholeStageCodegenTransformer (X)
Input [1]: [promo_revenue#X]
Arguments: false
-(25) VeloxColumnarToRow
+(24) VeloxColumnarToRow
Input [1]: [promo_revenue#X]
-(26) Scan parquet
+(25) Scan parquet
Output [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1995-09-01), LessThan(l_shipdate,1995-10-01), IsNotNull(l_partkey)]
ReadSchema: struct
-(27) Filter
+(26) Filter
Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Condition : (((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1995-09-01)) AND (l_shipdate#X < 1995-10-01)) AND isnotnull(l_partkey#X))
-(28) Project
+(27) Project
Output [3]: [l_partkey#X, l_extendedprice#X, l_discount#X]
Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
-(29) Exchange
+(28) Exchange
Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X]
Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(30) Sort
+(29) Sort
Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X]
Arguments: [l_partkey#X ASC NULLS FIRST], false, 0
-(31) Scan parquet
+(30) Scan parquet
Output [2]: [p_partkey#X, p_type#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(p_partkey)]
ReadSchema: struct
-(32) Filter
+(31) Filter
Input [2]: [p_partkey#X, p_type#X]
Condition : isnotnull(p_partkey#X)
-(33) Exchange
+(32) Exchange
Input [2]: [p_partkey#X, p_type#X]
Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(34) Sort
+(33) Sort
Input [2]: [p_partkey#X, p_type#X]
Arguments: [p_partkey#X ASC NULLS FIRST], false, 0
-(35) SortMergeJoin
+(34) SortMergeJoin
Left keys [1]: [l_partkey#X]
Right keys [1]: [p_partkey#X]
Join condition: None
-(36) Project
+(35) Project
Output [3]: [l_extendedprice#X, l_discount#X, p_type#X]
Input [5]: [l_partkey#X, l_extendedprice#X, l_discount#X, p_partkey#X, p_type#X]
-(37) HashAggregate
+(36) HashAggregate
Input [3]: [l_extendedprice#X, l_discount#X, p_type#X]
Keys: []
Functions [2]: [partial_sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)) ELSE 0.0000 END), partial_sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))]
Aggregate Attributes [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X]
Results [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X]
-(38) HashAggregate
+(37) HashAggregate
Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X]
Keys: []
Functions [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)) ELSE 0.0000 END), sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))]
Aggregate Attributes [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)) ELSE 0.0000 END)#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X]
Results [1]: [CheckOverflow((promote_precision(CheckOverflow((100.0000 * promote_precision(sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)) ELSE 0.0000 END)#X)), DecimalType(38,6))) / promote_precision(cast(sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X as decimal(38,6)))), DecimalType(38,6)) AS promo_revenue#X]
-(39) AdaptiveSparkPlan
+(38) AdaptiveSparkPlan
Output [1]: [promo_revenue#X]
Arguments: isFinalPlan=true
\ No newline at end of file
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 950441af70e9..398cc1f5b8b9 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
@@ -14,7 +14,7 @@ AdaptiveSparkPlan (47)
: +- VeloxResizeBatches (5)
: +- ^ ProjectExecTransformer (3)
: +- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
+ : +- ^ ScanTransformer parquet (1)
+- ^ FilterExecTransformer (22)
+- ^ RegularHashAggregateExecTransformer (21)
+- ^ InputIteratorTransformer (20)
@@ -25,7 +25,7 @@ AdaptiveSparkPlan (47)
+- ^ FlushableHashAggregateExecTransformer (13)
+- ^ ProjectExecTransformer (12)
+- ^ FilterExecTransformer (11)
- +- ^ Scan parquet (10)
+ +- ^ ScanTransformer parquet (10)
+- == Initial Plan ==
Sort (46)
+- Exchange (45)
@@ -45,7 +45,7 @@ AdaptiveSparkPlan (47)
+- Scan parquet (35)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -82,7 +82,7 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X]
(9) InputIteratorTransformer
Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -257,34 +257,33 @@ Arguments: isFinalPlan=true
===== Subqueries =====
Subquery:1 Hosting operator id = 22 Hosting Expression = Subquery subquery#X, [id=#X]
-AdaptiveSparkPlan (73)
+AdaptiveSparkPlan (72)
+- == Final Plan ==
- VeloxColumnarToRow (64)
- +- ^ RegularHashAggregateExecTransformer (62)
- +- ^ RegularHashAggregateExecTransformer (61)
- +- ^ ProjectExecTransformer (60)
- +- ^ RegularHashAggregateExecTransformer (59)
- +- ^ InputIteratorTransformer (58)
- +- ShuffleQueryStage (56), Statistics(X)
- +- ColumnarExchange (55)
- +- VeloxResizeBatches (54)
- +- ^ ProjectExecTransformer (52)
- +- ^ FlushableHashAggregateExecTransformer (51)
- +- ^ ProjectExecTransformer (50)
- +- ^ FilterExecTransformer (49)
- +- ^ Scan parquet (48)
+ VeloxColumnarToRow (63)
+ +- ^ RegularHashAggregateExecTransformer (61)
+ +- ^ ProjectExecTransformer (60)
+ +- ^ RegularHashAggregateExecTransformer (59)
+ +- ^ InputIteratorTransformer (58)
+ +- ShuffleQueryStage (56), Statistics(X)
+ +- ColumnarExchange (55)
+ +- VeloxResizeBatches (54)
+ +- ^ ProjectExecTransformer (52)
+ +- ^ FlushableHashAggregateExecTransformer (51)
+ +- ^ ProjectExecTransformer (50)
+ +- ^ FilterExecTransformer (49)
+ +- ^ ScanTransformer parquet (48)
+- == Initial Plan ==
- HashAggregate (72)
- +- HashAggregate (71)
- +- HashAggregate (70)
- +- Exchange (69)
- +- HashAggregate (68)
- +- Project (67)
- +- Filter (66)
- +- Scan parquet (65)
+ HashAggregate (71)
+ +- HashAggregate (70)
+ +- HashAggregate (69)
+ +- Exchange (68)
+ +- HashAggregate (67)
+ +- Project (66)
+ +- Filter (65)
+ +- Scan parquet (64)
-(48) Scan parquet
+(48) ScanTransformer parquet
Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -346,71 +345,64 @@ Input [2]: [l_suppkey#X, sum(CheckOverflow((promote_precision(cast(l_extendedpri
(61) RegularHashAggregateExecTransformer
Input [1]: [total_revenue#X]
Keys: []
-Functions [1]: [partial_max(total_revenue#X)]
-Aggregate Attributes [1]: [max#X]
-Results [1]: [max#X]
-
-(62) RegularHashAggregateExecTransformer
-Input [1]: [max#X]
-Keys: []
Functions [1]: [max(total_revenue#X)]
Aggregate Attributes [1]: [max(total_revenue#X)#X]
Results [1]: [max(total_revenue#X)#X AS max(total_revenue)#X]
-(63) WholeStageCodegenTransformer (X)
+(62) WholeStageCodegenTransformer (X)
Input [1]: [max(total_revenue)#X]
Arguments: false
-(64) VeloxColumnarToRow
+(63) VeloxColumnarToRow
Input [1]: [max(total_revenue)#X]
-(65) Scan parquet
+(64) Scan parquet
Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1996-01-01), LessThan(l_shipdate,1996-04-01)]
ReadSchema: struct
-(66) Filter
+(65) Filter
Input [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Condition : ((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1996-01-01)) AND (l_shipdate#X < 1996-04-01))
-(67) Project
+(66) Project
Output [3]: [l_suppkey#X, l_extendedprice#X, l_discount#X]
Input [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
-(68) HashAggregate
+(67) HashAggregate
Input [3]: [l_suppkey#X, l_extendedprice#X, l_discount#X]
Keys [1]: [l_suppkey#X]
Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))]
Aggregate Attributes [2]: [sum#X, isEmpty#X]
Results [3]: [l_suppkey#X, sum#X, isEmpty#X]
-(69) Exchange
+(68) Exchange
Input [3]: [l_suppkey#X, sum#X, isEmpty#X]
Arguments: hashpartitioning(l_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(70) HashAggregate
+(69) HashAggregate
Input [3]: [l_suppkey#X, sum#X, isEmpty#X]
Keys [1]: [l_suppkey#X]
Functions [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))]
Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X]
Results [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X AS total_revenue#X]
-(71) HashAggregate
+(70) HashAggregate
Input [1]: [total_revenue#X]
Keys: []
Functions [1]: [partial_max(total_revenue#X)]
Aggregate Attributes [1]: [max#X]
Results [1]: [max#X]
-(72) HashAggregate
+(71) HashAggregate
Input [1]: [max#X]
Keys: []
Functions [1]: [max(total_revenue#X)]
Aggregate Attributes [1]: [max(total_revenue#X)#X]
Results [1]: [max(total_revenue#X)#X AS max(total_revenue)#X]
-(73) AdaptiveSparkPlan
+(72) AdaptiveSparkPlan
Output [1]: [max(total_revenue)#X]
Arguments: isFinalPlan=true
\ No newline at end of file
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 aa7aa005a9b2..cac61db4a6bd 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
@@ -29,14 +29,14 @@ AdaptiveSparkPlan (71)
: +- VeloxResizeBatches (5)
: +- ^ ProjectExecTransformer (3)
: +- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
+ : +- ^ ScanTransformer parquet (1)
+- ^ InputIteratorTransformer (18)
+- ShuffleQueryStage (16), Statistics(X)
+- ColumnarExchange (15)
+- VeloxResizeBatches (14)
+- ^ ProjectExecTransformer (12)
+- ^ FilterExecTransformer (11)
- +- ^ Scan parquet (10)
+ +- ^ ScanTransformer parquet (10)
+- == Initial Plan ==
Sort (70)
+- Exchange (69)
@@ -63,7 +63,7 @@ AdaptiveSparkPlan (71)
+- Scan parquet (57)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [ps_partkey#X, ps_suppkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -100,7 +100,7 @@ Input [2]: [ps_partkey#X, ps_suppkey#X]
(9) InputIteratorTransformer
Input [2]: [ps_partkey#X, ps_suppkey#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [4]: [p_partkey#X, p_brand#X, p_type#X, p_size#X]
Batched: true
Location: InMemoryFileIndex [*]
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 e48379edc9fa..843ce9b39eb6 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
@@ -1,65 +1,64 @@
== Physical Plan ==
-AdaptiveSparkPlan (63)
+AdaptiveSparkPlan (62)
+- == Final Plan ==
- VeloxColumnarToRow (40)
- +- ^ ProjectExecTransformer (38)
- +- ^ RegularHashAggregateExecTransformer (37)
- +- ^ RegularHashAggregateExecTransformer (36)
- +- ^ ProjectExecTransformer (35)
- +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (34)
- :- ^ ProjectExecTransformer (20)
- : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19)
- : :- ^ InputIteratorTransformer (9)
- : : +- ShuffleQueryStage (7), Statistics(X)
- : : +- ColumnarExchange (6)
- : : +- VeloxResizeBatches (5)
- : : +- ^ ProjectExecTransformer (3)
- : : +- ^ FilterExecTransformer (2)
- : : +- ^ Scan parquet (1)
- : +- ^ InputIteratorTransformer (18)
- : +- ShuffleQueryStage (16), Statistics(X)
- : +- ColumnarExchange (15)
- : +- VeloxResizeBatches (14)
- : +- ^ ProjectExecTransformer (12)
- : +- ^ FilterExecTransformer (11)
- : +- ^ Scan parquet (10)
- +- ^ FilterExecTransformer (33)
- +- ^ ProjectExecTransformer (32)
- +- ^ RegularHashAggregateExecTransformer (31)
- +- ^ InputIteratorTransformer (30)
- +- ShuffleQueryStage (28), Statistics(X)
- +- ColumnarExchange (27)
- +- VeloxResizeBatches (26)
- +- ^ ProjectExecTransformer (24)
- +- ^ FlushableHashAggregateExecTransformer (23)
- +- ^ FilterExecTransformer (22)
- +- ^ Scan parquet (21)
+ VeloxColumnarToRow (39)
+ +- ^ ProjectExecTransformer (37)
+ +- ^ RegularHashAggregateExecTransformer (36)
+ +- ^ ProjectExecTransformer (35)
+ +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (34)
+ :- ^ ProjectExecTransformer (20)
+ : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19)
+ : :- ^ InputIteratorTransformer (9)
+ : : +- ShuffleQueryStage (7), Statistics(X)
+ : : +- ColumnarExchange (6)
+ : : +- VeloxResizeBatches (5)
+ : : +- ^ ProjectExecTransformer (3)
+ : : +- ^ FilterExecTransformer (2)
+ : : +- ^ ScanTransformer parquet (1)
+ : +- ^ InputIteratorTransformer (18)
+ : +- ShuffleQueryStage (16), Statistics(X)
+ : +- ColumnarExchange (15)
+ : +- VeloxResizeBatches (14)
+ : +- ^ ProjectExecTransformer (12)
+ : +- ^ FilterExecTransformer (11)
+ : +- ^ ScanTransformer parquet (10)
+ +- ^ FilterExecTransformer (33)
+ +- ^ ProjectExecTransformer (32)
+ +- ^ RegularHashAggregateExecTransformer (31)
+ +- ^ InputIteratorTransformer (30)
+ +- ShuffleQueryStage (28), Statistics(X)
+ +- ColumnarExchange (27)
+ +- VeloxResizeBatches (26)
+ +- ^ ProjectExecTransformer (24)
+ +- ^ FlushableHashAggregateExecTransformer (23)
+ +- ^ FilterExecTransformer (22)
+ +- ^ ScanTransformer parquet (21)
+- == Initial Plan ==
- HashAggregate (62)
- +- HashAggregate (61)
- +- Project (60)
- +- SortMergeJoin Inner (59)
- :- Project (51)
- : +- SortMergeJoin Inner (50)
- : :- Sort (44)
- : : +- Exchange (43)
- : : +- Filter (42)
- : : +- Scan parquet (41)
- : +- Sort (49)
- : +- Exchange (48)
- : +- Project (47)
- : +- Filter (46)
- : +- Scan parquet (45)
- +- Sort (58)
- +- Filter (57)
- +- HashAggregate (56)
- +- Exchange (55)
- +- HashAggregate (54)
- +- Filter (53)
- +- Scan parquet (52)
-
-
-(1) Scan parquet
+ HashAggregate (61)
+ +- HashAggregate (60)
+ +- Project (59)
+ +- SortMergeJoin Inner (58)
+ :- Project (50)
+ : +- SortMergeJoin Inner (49)
+ : :- Sort (43)
+ : : +- Exchange (42)
+ : : +- Filter (41)
+ : : +- Scan parquet (40)
+ : +- Sort (48)
+ : +- Exchange (47)
+ : +- Project (46)
+ : +- Filter (45)
+ : +- Scan parquet (44)
+ +- Sort (57)
+ +- Filter (56)
+ +- HashAggregate (55)
+ +- Exchange (54)
+ +- HashAggregate (53)
+ +- Filter (52)
+ +- Scan parquet (51)
+
+
+(1) ScanTransformer parquet
Output [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -96,7 +95,7 @@ Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X]
(9) InputIteratorTransformer
Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [3]: [p_partkey#X, p_brand#X, p_container#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -142,7 +141,7 @@ Join condition: None
Output [3]: [l_quantity#X, l_extendedprice#X, p_partkey#X]
Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, p_partkey#X]
-(21) Scan parquet
+(21) ScanTransformer parquet
Output [2]: [l_partkey#X, l_quantity#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -213,139 +212,132 @@ Input [5]: [l_quantity#X, l_extendedprice#X, p_partkey#X, (0.2 * avg(l_quantity)
(36) RegularHashAggregateExecTransformer
Input [1]: [l_extendedprice#X]
Keys: []
-Functions [1]: [partial_sum(l_extendedprice#X)]
-Aggregate Attributes [2]: [sum#X, isEmpty#X]
-Results [2]: [sum#X, isEmpty#X]
-
-(37) RegularHashAggregateExecTransformer
-Input [2]: [sum#X, isEmpty#X]
-Keys: []
Functions [1]: [sum(l_extendedprice#X)]
Aggregate Attributes [1]: [sum(l_extendedprice#X)#X]
Results [1]: [sum(l_extendedprice#X)#X]
-(38) ProjectExecTransformer
+(37) ProjectExecTransformer
Output [1]: [CheckOverflow((promote_precision(sum(l_extendedprice#X)#X) / 7.00), DecimalType(27,6)) AS avg_yearly#X]
Input [1]: [sum(l_extendedprice#X)#X]
-(39) WholeStageCodegenTransformer (X)
+(38) WholeStageCodegenTransformer (X)
Input [1]: [avg_yearly#X]
Arguments: false
-(40) VeloxColumnarToRow
+(39) VeloxColumnarToRow
Input [1]: [avg_yearly#X]
-(41) Scan parquet
+(40) Scan parquet
Output [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(l_partkey), IsNotNull(l_quantity)]
ReadSchema: struct
-(42) Filter
+(41) Filter
Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X]
Condition : (isnotnull(l_partkey#X) AND isnotnull(l_quantity#X))
-(43) Exchange
+(42) Exchange
Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X]
Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(44) Sort
+(43) Sort
Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X]
Arguments: [l_partkey#X ASC NULLS FIRST], false, 0
-(45) Scan parquet
+(44) Scan parquet
Output [3]: [p_partkey#X, p_brand#X, p_container#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(p_brand), IsNotNull(p_container), EqualTo(p_brand,Brand#X), EqualTo(p_container,MED BOX), IsNotNull(p_partkey)]
ReadSchema: struct
-(46) Filter
+(45) Filter
Input [3]: [p_partkey#X, p_brand#X, p_container#X]
Condition : ((((isnotnull(p_brand#X) AND isnotnull(p_container#X)) AND (p_brand#X = Brand#X)) AND (p_container#X = MED BOX)) AND isnotnull(p_partkey#X))
-(47) Project
+(46) Project
Output [1]: [p_partkey#X]
Input [3]: [p_partkey#X, p_brand#X, p_container#X]
-(48) Exchange
+(47) Exchange
Input [1]: [p_partkey#X]
Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(49) Sort
+(48) Sort
Input [1]: [p_partkey#X]
Arguments: [p_partkey#X ASC NULLS FIRST], false, 0
-(50) SortMergeJoin
+(49) SortMergeJoin
Left keys [1]: [l_partkey#X]
Right keys [1]: [p_partkey#X]
Join condition: None
-(51) Project
+(50) Project
Output [3]: [l_quantity#X, l_extendedprice#X, p_partkey#X]
Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, p_partkey#X]
-(52) Scan parquet
+(51) Scan parquet
Output [2]: [l_partkey#X, l_quantity#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(l_partkey)]
ReadSchema: struct
-(53) Filter
+(52) Filter
Input [2]: [l_partkey#X, l_quantity#X]
Condition : isnotnull(l_partkey#X)
-(54) HashAggregate
+(53) HashAggregate
Input [2]: [l_partkey#X, l_quantity#X]
Keys [1]: [l_partkey#X]
Functions [1]: [partial_avg(l_quantity#X)]
Aggregate Attributes [2]: [sum#X, count#X]
Results [3]: [l_partkey#X, sum#X, count#X]
-(55) Exchange
+(54) Exchange
Input [3]: [l_partkey#X, sum#X, count#X]
Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(56) HashAggregate
+(55) HashAggregate
Input [3]: [l_partkey#X, sum#X, count#X]
Keys [1]: [l_partkey#X]
Functions [1]: [avg(l_quantity#X)]
Aggregate Attributes [1]: [avg(l_quantity#X)#X]
Results [2]: [CheckOverflow((0.200000 * promote_precision(avg(l_quantity#X)#X)), DecimalType(18,7)) AS (0.2 * avg(l_quantity))#X, l_partkey#X]
-(57) Filter
+(56) Filter
Input [2]: [(0.2 * avg(l_quantity))#X, l_partkey#X]
Condition : isnotnull((0.2 * avg(l_quantity))#X)
-(58) Sort
+(57) Sort
Input [2]: [(0.2 * avg(l_quantity))#X, l_partkey#X]
Arguments: [l_partkey#X ASC NULLS FIRST], false, 0
-(59) SortMergeJoin
+(58) SortMergeJoin
Left keys [1]: [p_partkey#X]
Right keys [1]: [l_partkey#X]
Join condition: (cast(l_quantity#X as decimal(18,7)) < (0.2 * avg(l_quantity))#X)
-(60) Project
+(59) Project
Output [1]: [l_extendedprice#X]
Input [5]: [l_quantity#X, l_extendedprice#X, p_partkey#X, (0.2 * avg(l_quantity))#X, l_partkey#X]
-(61) HashAggregate
+(60) HashAggregate
Input [1]: [l_extendedprice#X]
Keys: []
Functions [1]: [partial_sum(l_extendedprice#X)]
Aggregate Attributes [2]: [sum#X, isEmpty#X]
Results [2]: [sum#X, isEmpty#X]
-(62) HashAggregate
+(61) HashAggregate
Input [2]: [sum#X, isEmpty#X]
Keys: []
Functions [1]: [sum(l_extendedprice#X)]
Aggregate Attributes [1]: [sum(l_extendedprice#X)#X]
Results [1]: [CheckOverflow((promote_precision(sum(l_extendedprice#X)#X) / 7.00), DecimalType(27,6)) AS avg_yearly#X]
-(63) AdaptiveSparkPlan
+(62) AdaptiveSparkPlan
Output [1]: [avg_yearly#X]
Arguments: isFinalPlan=true
\ No newline at end of file
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 8dbf9429961a..1119ddcf80be 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
@@ -1,105 +1,104 @@
== Physical Plan ==
-AdaptiveSparkPlan (110)
+AdaptiveSparkPlan (109)
+- == Final Plan ==
- VeloxColumnarToRow (70)
- +- TakeOrderedAndProjectExecTransformer (69)
- +- ^ RegularHashAggregateExecTransformer (67)
- +- ^ RegularHashAggregateExecTransformer (66)
- +- ^ ProjectExecTransformer (65)
- +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (64)
- :- ^ InputIteratorTransformer (46)
- : +- ShuffleQueryStage (44), Statistics(X)
- : +- ColumnarExchange (43)
- : +- VeloxResizeBatches (42)
- : +- ^ ProjectExecTransformer (40)
- : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (39)
- : :- ^ InputIteratorTransformer (9)
- : : +- ShuffleQueryStage (7), Statistics(X)
- : : +- ColumnarExchange (6)
- : : +- VeloxResizeBatches (5)
- : : +- ^ ProjectExecTransformer (3)
- : : +- ^ FilterExecTransformer (2)
- : : +- ^ Scan parquet (1)
- : +- ^ InputIteratorTransformer (38)
- : +- ShuffleQueryStage (36), Statistics(X)
- : +- ColumnarExchange (35)
- : +- VeloxResizeBatches (34)
- : +- ^ ProjectExecTransformer (32)
- : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (31)
- : :- ^ InputIteratorTransformer (18)
- : : +- ShuffleQueryStage (16), Statistics(X)
- : : +- ColumnarExchange (15)
- : : +- VeloxResizeBatches (14)
- : : +- ^ ProjectExecTransformer (12)
- : : +- ^ FilterExecTransformer (11)
- : : +- ^ Scan parquet (10)
- : +- ^ ProjectExecTransformer (30)
- : +- ^ FilterExecTransformer (29)
- : +- ^ RegularHashAggregateExecTransformer (28)
- : +- ^ InputIteratorTransformer (27)
- : +- ShuffleQueryStage (25), Statistics(X)
- : +- ColumnarExchange (24)
- : +- VeloxResizeBatches (23)
- : +- ^ ProjectExecTransformer (21)
- : +- ^ FlushableHashAggregateExecTransformer (20)
- : +- ^ Scan parquet (19)
- +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (63)
- :- ^ InputIteratorTransformer (55)
- : +- ShuffleQueryStage (53), Statistics(X)
- : +- ColumnarExchange (52)
- : +- VeloxResizeBatches (51)
- : +- ^ ProjectExecTransformer (49)
- : +- ^ FilterExecTransformer (48)
- : +- ^ Scan parquet (47)
- +- ^ ProjectExecTransformer (62)
- +- ^ FilterExecTransformer (61)
- +- ^ RegularHashAggregateExecTransformer (60)
- +- ^ InputIteratorTransformer (59)
- +- ShuffleQueryStage (57), Statistics(X)
- +- ReusedExchange (56)
+ VeloxColumnarToRow (69)
+ +- TakeOrderedAndProjectExecTransformer (68)
+ +- ^ RegularHashAggregateExecTransformer (66)
+ +- ^ ProjectExecTransformer (65)
+ +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (64)
+ :- ^ InputIteratorTransformer (46)
+ : +- ShuffleQueryStage (44), Statistics(X)
+ : +- ColumnarExchange (43)
+ : +- VeloxResizeBatches (42)
+ : +- ^ ProjectExecTransformer (40)
+ : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (39)
+ : :- ^ InputIteratorTransformer (9)
+ : : +- ShuffleQueryStage (7), Statistics(X)
+ : : +- ColumnarExchange (6)
+ : : +- VeloxResizeBatches (5)
+ : : +- ^ ProjectExecTransformer (3)
+ : : +- ^ FilterExecTransformer (2)
+ : : +- ^ ScanTransformer parquet (1)
+ : +- ^ InputIteratorTransformer (38)
+ : +- ShuffleQueryStage (36), Statistics(X)
+ : +- ColumnarExchange (35)
+ : +- VeloxResizeBatches (34)
+ : +- ^ ProjectExecTransformer (32)
+ : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (31)
+ : :- ^ InputIteratorTransformer (18)
+ : : +- ShuffleQueryStage (16), Statistics(X)
+ : : +- ColumnarExchange (15)
+ : : +- VeloxResizeBatches (14)
+ : : +- ^ ProjectExecTransformer (12)
+ : : +- ^ FilterExecTransformer (11)
+ : : +- ^ ScanTransformer parquet (10)
+ : +- ^ ProjectExecTransformer (30)
+ : +- ^ FilterExecTransformer (29)
+ : +- ^ RegularHashAggregateExecTransformer (28)
+ : +- ^ InputIteratorTransformer (27)
+ : +- ShuffleQueryStage (25), Statistics(X)
+ : +- ColumnarExchange (24)
+ : +- VeloxResizeBatches (23)
+ : +- ^ ProjectExecTransformer (21)
+ : +- ^ FlushableHashAggregateExecTransformer (20)
+ : +- ^ ScanTransformer parquet (19)
+ +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (63)
+ :- ^ InputIteratorTransformer (55)
+ : +- ShuffleQueryStage (53), Statistics(X)
+ : +- ColumnarExchange (52)
+ : +- VeloxResizeBatches (51)
+ : +- ^ ProjectExecTransformer (49)
+ : +- ^ FilterExecTransformer (48)
+ : +- ^ ScanTransformer parquet (47)
+ +- ^ ProjectExecTransformer (62)
+ +- ^ FilterExecTransformer (61)
+ +- ^ RegularHashAggregateExecTransformer (60)
+ +- ^ InputIteratorTransformer (59)
+ +- ShuffleQueryStage (57), Statistics(X)
+ +- ReusedExchange (56)
+- == Initial Plan ==
- TakeOrderedAndProject (109)
- +- HashAggregate (108)
- +- HashAggregate (107)
- +- Project (106)
- +- SortMergeJoin Inner (105)
- :- Sort (92)
- : +- Exchange (91)
- : +- Project (90)
- : +- SortMergeJoin Inner (89)
- : :- Sort (74)
- : : +- Exchange (73)
- : : +- Filter (72)
- : : +- Scan parquet (71)
- : +- Sort (88)
- : +- Exchange (87)
- : +- SortMergeJoin LeftSemi (86)
- : :- Sort (78)
- : : +- Exchange (77)
- : : +- Filter (76)
- : : +- Scan parquet (75)
- : +- Sort (85)
- : +- Project (84)
- : +- Filter (83)
- : +- HashAggregate (82)
- : +- Exchange (81)
- : +- HashAggregate (80)
- : +- Scan parquet (79)
- +- SortMergeJoin LeftSemi (104)
- :- Sort (96)
- : +- Exchange (95)
- : +- Filter (94)
- : +- Scan parquet (93)
- +- Sort (103)
- +- Project (102)
- +- Filter (101)
- +- HashAggregate (100)
- +- Exchange (99)
- +- HashAggregate (98)
- +- Scan parquet (97)
-
-
-(1) Scan parquet
+ TakeOrderedAndProject (108)
+ +- HashAggregate (107)
+ +- HashAggregate (106)
+ +- Project (105)
+ +- SortMergeJoin Inner (104)
+ :- Sort (91)
+ : +- Exchange (90)
+ : +- Project (89)
+ : +- SortMergeJoin Inner (88)
+ : :- Sort (73)
+ : : +- Exchange (72)
+ : : +- Filter (71)
+ : : +- Scan parquet (70)
+ : +- Sort (87)
+ : +- Exchange (86)
+ : +- SortMergeJoin LeftSemi (85)
+ : :- Sort (77)
+ : : +- Exchange (76)
+ : : +- Filter (75)
+ : : +- Scan parquet (74)
+ : +- Sort (84)
+ : +- Project (83)
+ : +- Filter (82)
+ : +- HashAggregate (81)
+ : +- Exchange (80)
+ : +- HashAggregate (79)
+ : +- Scan parquet (78)
+ +- SortMergeJoin LeftSemi (103)
+ :- Sort (95)
+ : +- Exchange (94)
+ : +- Filter (93)
+ : +- Scan parquet (92)
+ +- Sort (102)
+ +- Project (101)
+ +- Filter (100)
+ +- HashAggregate (99)
+ +- Exchange (98)
+ +- HashAggregate (97)
+ +- Scan parquet (96)
+
+
+(1) ScanTransformer parquet
Output [2]: [c_custkey#X, c_name#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -136,7 +135,7 @@ Input [2]: [c_custkey#X, c_name#X]
(9) InputIteratorTransformer
Input [2]: [c_custkey#X, c_name#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -173,7 +172,7 @@ Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X]
(18) InputIteratorTransformer
Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X]
-(19) Scan parquet
+(19) ScanTransformer parquet
Output [2]: [l_orderkey#X, l_quantity#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -289,7 +288,7 @@ Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X]
(46) InputIteratorTransformer
Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X]
-(47) Scan parquet
+(47) ScanTransformer parquet
Output [2]: [l_orderkey#X, l_quantity#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -371,219 +370,212 @@ Input [7]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X,
(66) RegularHashAggregateExecTransformer
Input [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_quantity#X]
Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X]
-Functions [1]: [partial_sum(l_quantity#X)]
-Aggregate Attributes [2]: [sum#X, isEmpty#X]
-Results [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X]
-
-(67) RegularHashAggregateExecTransformer
-Input [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X]
-Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X]
Functions [1]: [sum(l_quantity#X)]
Aggregate Attributes [1]: [sum(l_quantity#X)#X]
Results [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity#X)#X AS sum(l_quantity)#X]
-(68) WholeStageCodegenTransformer (X)
+(67) WholeStageCodegenTransformer (X)
Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X]
Arguments: false
-(69) TakeOrderedAndProjectExecTransformer
+(68) TakeOrderedAndProjectExecTransformer
Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X]
Arguments: X, [o_totalprice#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X], 0
-(70) VeloxColumnarToRow
+(69) VeloxColumnarToRow
Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X]
-(71) Scan parquet
+(70) Scan parquet
Output [2]: [c_custkey#X, c_name#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(c_custkey)]
ReadSchema: struct
-(72) Filter
+(71) Filter
Input [2]: [c_custkey#X, c_name#X]
Condition : isnotnull(c_custkey#X)
-(73) Exchange
+(72) Exchange
Input [2]: [c_custkey#X, c_name#X]
Arguments: hashpartitioning(c_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(74) Sort
+(73) Sort
Input [2]: [c_custkey#X, c_name#X]
Arguments: [c_custkey#X ASC NULLS FIRST], false, 0
-(75) Scan parquet
+(74) Scan parquet
Output [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(o_custkey), IsNotNull(o_orderkey)]
ReadSchema: struct
-(76) Filter
+(75) Filter
Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X]
Condition : (isnotnull(o_custkey#X) AND isnotnull(o_orderkey#X))
-(77) Exchange
+(76) Exchange
Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X]
Arguments: hashpartitioning(o_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(78) Sort
+(77) Sort
Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X]
Arguments: [o_orderkey#X ASC NULLS FIRST], false, 0
-(79) Scan parquet
+(78) Scan parquet
Output [2]: [l_orderkey#X, l_quantity#X]
Batched: true
Location: InMemoryFileIndex [*]
ReadSchema: struct
-(80) HashAggregate
+(79) HashAggregate
Input [2]: [l_orderkey#X, l_quantity#X]
Keys [1]: [l_orderkey#X]
Functions [1]: [partial_sum(l_quantity#X)]
Aggregate Attributes [2]: [sum#X, isEmpty#X]
Results [3]: [l_orderkey#X, sum#X, isEmpty#X]
-(81) Exchange
+(80) Exchange
Input [3]: [l_orderkey#X, sum#X, isEmpty#X]
Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(82) HashAggregate
+(81) HashAggregate
Input [3]: [l_orderkey#X, sum#X, isEmpty#X]
Keys [1]: [l_orderkey#X]
Functions [1]: [sum(l_quantity#X)]
Aggregate Attributes [1]: [sum(l_quantity#X)#X]
Results [2]: [l_orderkey#X, sum(l_quantity#X)#X AS sum(l_quantity#X)#X]
-(83) Filter
+(82) Filter
Input [2]: [l_orderkey#X, sum(l_quantity#X)#X]
Condition : (isnotnull(sum(l_quantity#X)#X) AND (sum(l_quantity#X)#X > 300.00))
-(84) Project
+(83) Project
Output [1]: [l_orderkey#X]
Input [2]: [l_orderkey#X, sum(l_quantity#X)#X]
-(85) Sort
+(84) Sort
Input [1]: [l_orderkey#X]
Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0
-(86) SortMergeJoin
+(85) SortMergeJoin
Left keys [1]: [o_orderkey#X]
Right keys [1]: [l_orderkey#X]
Join condition: None
-(87) Exchange
+(86) Exchange
Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X]
Arguments: hashpartitioning(o_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(88) Sort
+(87) Sort
Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X]
Arguments: [o_custkey#X ASC NULLS FIRST], false, 0
-(89) SortMergeJoin
+(88) SortMergeJoin
Left keys [1]: [c_custkey#X]
Right keys [1]: [o_custkey#X]
Join condition: None
-(90) Project
+(89) Project
Output [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X]
Input [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X]
-(91) Exchange
+(90) Exchange
Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X]
Arguments: hashpartitioning(o_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(92) Sort
+(91) Sort
Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X]
Arguments: [o_orderkey#X ASC NULLS FIRST], false, 0
-(93) Scan parquet
+(92) Scan parquet
Output [2]: [l_orderkey#X, l_quantity#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(l_orderkey)]
ReadSchema: struct
-(94) Filter
+(93) Filter
Input [2]: [l_orderkey#X, l_quantity#X]
Condition : isnotnull(l_orderkey#X)
-(95) Exchange
+(94) Exchange
Input [2]: [l_orderkey#X, l_quantity#X]
Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(96) Sort
+(95) Sort
Input [2]: [l_orderkey#X, l_quantity#X]
Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0
-(97) Scan parquet
+(96) Scan parquet
Output [2]: [l_orderkey#X, l_quantity#X]
Batched: true
Location: InMemoryFileIndex [*]
ReadSchema: struct
-(98) HashAggregate
+(97) HashAggregate
Input [2]: [l_orderkey#X, l_quantity#X]
Keys [1]: [l_orderkey#X]
Functions [1]: [partial_sum(l_quantity#X)]
Aggregate Attributes [2]: [sum#X, isEmpty#X]
Results [3]: [l_orderkey#X, sum#X, isEmpty#X]
-(99) Exchange
+(98) Exchange
Input [3]: [l_orderkey#X, sum#X, isEmpty#X]
Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(100) HashAggregate
+(99) HashAggregate
Input [3]: [l_orderkey#X, sum#X, isEmpty#X]
Keys [1]: [l_orderkey#X]
Functions [1]: [sum(l_quantity#X)]
Aggregate Attributes [1]: [sum(l_quantity#X)#X]
Results [2]: [l_orderkey#X, sum(l_quantity#X)#X AS sum(l_quantity#X)#X]
-(101) Filter
+(100) Filter
Input [2]: [l_orderkey#X, sum(l_quantity#X)#X]
Condition : (isnotnull(sum(l_quantity#X)#X) AND (sum(l_quantity#X)#X > 300.00))
-(102) Project
+(101) Project
Output [1]: [l_orderkey#X]
Input [2]: [l_orderkey#X, sum(l_quantity#X)#X]
-(103) Sort
+(102) Sort
Input [1]: [l_orderkey#X]
Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0
-(104) SortMergeJoin
+(103) SortMergeJoin
Left keys [1]: [l_orderkey#X]
Right keys [1]: [l_orderkey#X]
Join condition: None
-(105) SortMergeJoin
+(104) SortMergeJoin
Left keys [1]: [o_orderkey#X]
Right keys [1]: [l_orderkey#X]
Join condition: None
-(106) Project
+(105) Project
Output [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_quantity#X]
Input [7]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_orderkey#X, l_quantity#X]
-(107) HashAggregate
+(106) HashAggregate
Input [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_quantity#X]
Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X]
Functions [1]: [partial_sum(l_quantity#X)]
Aggregate Attributes [2]: [sum#X, isEmpty#X]
Results [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X]
-(108) HashAggregate
+(107) HashAggregate
Input [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X]
Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X]
Functions [1]: [sum(l_quantity#X)]
Aggregate Attributes [1]: [sum(l_quantity#X)#X]
Results [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity#X)#X AS sum(l_quantity)#X]
-(109) TakeOrderedAndProject
+(108) TakeOrderedAndProject
Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X]
Arguments: X, [o_totalprice#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X]
-(110) AdaptiveSparkPlan
+(109) AdaptiveSparkPlan
Output [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X]
Arguments: isFinalPlan=true
\ No newline at end of file
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 41da1d28e987..31341db339e1 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
@@ -1,42 +1,41 @@
== Physical Plan ==
-AdaptiveSparkPlan (38)
+AdaptiveSparkPlan (37)
+- == Final Plan ==
- VeloxColumnarToRow (24)
- +- ^ RegularHashAggregateExecTransformer (22)
- +- ^ RegularHashAggregateExecTransformer (21)
- +- ^ ProjectExecTransformer (20)
- +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19)
- :- ^ InputIteratorTransformer (9)
- : +- ShuffleQueryStage (7), Statistics(X)
- : +- ColumnarExchange (6)
- : +- VeloxResizeBatches (5)
- : +- ^ ProjectExecTransformer (3)
- : +- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
- +- ^ InputIteratorTransformer (18)
- +- ShuffleQueryStage (16), Statistics(X)
- +- ColumnarExchange (15)
- +- VeloxResizeBatches (14)
- +- ^ ProjectExecTransformer (12)
- +- ^ FilterExecTransformer (11)
- +- ^ Scan parquet (10)
+ VeloxColumnarToRow (23)
+ +- ^ RegularHashAggregateExecTransformer (21)
+ +- ^ ProjectExecTransformer (20)
+ +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19)
+ :- ^ InputIteratorTransformer (9)
+ : +- ShuffleQueryStage (7), Statistics(X)
+ : +- ColumnarExchange (6)
+ : +- VeloxResizeBatches (5)
+ : +- ^ ProjectExecTransformer (3)
+ : +- ^ FilterExecTransformer (2)
+ : +- ^ ScanTransformer parquet (1)
+ +- ^ InputIteratorTransformer (18)
+ +- ShuffleQueryStage (16), Statistics(X)
+ +- ColumnarExchange (15)
+ +- VeloxResizeBatches (14)
+ +- ^ ProjectExecTransformer (12)
+ +- ^ FilterExecTransformer (11)
+ +- ^ ScanTransformer parquet (10)
+- == Initial Plan ==
- HashAggregate (37)
- +- HashAggregate (36)
- +- Project (35)
- +- SortMergeJoin Inner (34)
- :- Sort (29)
- : +- Exchange (28)
- : +- Project (27)
- : +- Filter (26)
- : +- Scan parquet (25)
- +- Sort (33)
- +- Exchange (32)
- +- Filter (31)
- +- Scan parquet (30)
-
-
-(1) Scan parquet
+ HashAggregate (36)
+ +- HashAggregate (35)
+ +- Project (34)
+ +- SortMergeJoin Inner (33)
+ :- Sort (28)
+ : +- Exchange (27)
+ : +- Project (26)
+ : +- Filter (25)
+ : +- Scan parquet (24)
+ +- Sort (32)
+ +- Exchange (31)
+ +- Filter (30)
+ +- Scan parquet (29)
+
+
+(1) ScanTransformer parquet
Output [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -73,7 +72,7 @@ Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X]
(9) InputIteratorTransformer
Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -122,89 +121,82 @@ Input [8]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, p_partke
(21) RegularHashAggregateExecTransformer
Input [3]: [l_extendedprice#X, l_discount#X, _pre_X#X]
Keys: []
-Functions [1]: [partial_sum(_pre_X#X)]
-Aggregate Attributes [2]: [sum#X, isEmpty#X]
-Results [2]: [sum#X, isEmpty#X]
-
-(22) RegularHashAggregateExecTransformer
-Input [2]: [sum#X, isEmpty#X]
-Keys: []
-Functions [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))]
+Functions [1]: [sum(_pre_X#X)]
Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X]
Results [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X AS revenue#X]
-(23) WholeStageCodegenTransformer (X)
+(22) WholeStageCodegenTransformer (X)
Input [1]: [revenue#X]
Arguments: false
-(24) VeloxColumnarToRow
+(23) VeloxColumnarToRow
Input [1]: [revenue#X]
-(25) Scan parquet
+(24) Scan parquet
Output [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(l_shipinstruct), In(l_shipmode, [AIR,AIR REG]), EqualTo(l_shipinstruct,DELIVER IN PERSON), IsNotNull(l_partkey), Or(Or(And(GreaterThanOrEqual(l_quantity,1.00),LessThanOrEqual(l_quantity,11.00)),And(GreaterThanOrEqual(l_quantity,10.00),LessThanOrEqual(l_quantity,20.00))),And(GreaterThanOrEqual(l_quantity,20.00),LessThanOrEqual(l_quantity,30.00)))]
ReadSchema: struct
-(26) Filter
+(25) Filter
Input [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X]
Condition : ((((isnotnull(l_shipinstruct#X) AND l_shipmode#X IN (AIR,AIR REG)) AND (l_shipinstruct#X = DELIVER IN PERSON)) AND isnotnull(l_partkey#X)) AND ((((l_quantity#X >= 1.00) AND (l_quantity#X <= 11.00)) OR ((l_quantity#X >= 10.00) AND (l_quantity#X <= 20.00))) OR ((l_quantity#X >= 20.00) AND (l_quantity#X <= 30.00))))
-(27) Project
+(26) Project
Output [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X]
Input [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X]
-(28) Exchange
+(27) Exchange
Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X]
Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(29) Sort
+(28) Sort
Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X]
Arguments: [l_partkey#X ASC NULLS FIRST], false, 0
-(30) Scan parquet
+(29) Scan parquet
Output [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(p_size), GreaterThanOrEqual(p_size,1), IsNotNull(p_partkey), Or(Or(And(And(EqualTo(p_brand,Brand#X),In(p_container, [SM BOX,SM CASE,SM PACK,SM PKG])),LessThanOrEqual(p_size,5)),And(And(EqualTo(p_brand,Brand#X),In(p_container, [MED BAG,MED BOX,MED PACK,MED PKG])),LessThanOrEqual(p_size,10))),And(And(EqualTo(p_brand,Brand#X),In(p_container, [LG BOX,LG CASE,LG PACK,LG PKG])),LessThanOrEqual(p_size,15)))]
ReadSchema: struct
-(31) Filter
+(30) Filter
Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X]
Condition : (((isnotnull(p_size#X) AND (p_size#X >= 1)) AND isnotnull(p_partkey#X)) AND (((((p_brand#X = Brand#X) AND p_container#X IN (SM CASE,SM BOX,SM PACK,SM PKG)) AND (p_size#X <= 5)) OR (((p_brand#X = Brand#X) AND p_container#X IN (MED BAG,MED BOX,MED PKG,MED PACK)) AND (p_size#X <= 10))) OR (((p_brand#X = Brand#X) AND p_container#X IN (LG CASE,LG BOX,LG PACK,LG PKG)) AND (p_size#X <= 15))))
-(32) Exchange
+(31) Exchange
Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X]
Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(33) Sort
+(32) Sort
Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X]
Arguments: [p_partkey#X ASC NULLS FIRST], false, 0
-(34) SortMergeJoin
+(33) SortMergeJoin
Left keys [1]: [l_partkey#X]
Right keys [1]: [p_partkey#X]
Join condition: (((((((p_brand#X = Brand#X) AND p_container#X IN (SM CASE,SM BOX,SM PACK,SM PKG)) AND (l_quantity#X >= 1.00)) AND (l_quantity#X <= 11.00)) AND (p_size#X <= 5)) OR (((((p_brand#X = Brand#X) AND p_container#X IN (MED BAG,MED BOX,MED PKG,MED PACK)) AND (l_quantity#X >= 10.00)) AND (l_quantity#X <= 20.00)) AND (p_size#X <= 10))) OR (((((p_brand#X = Brand#X) AND p_container#X IN (LG CASE,LG BOX,LG PACK,LG PKG)) AND (l_quantity#X >= 20.00)) AND (l_quantity#X <= 30.00)) AND (p_size#X <= 15)))
-(35) Project
+(34) Project
Output [2]: [l_extendedprice#X, l_discount#X]
Input [8]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, p_partkey#X, p_brand#X, p_size#X, p_container#X]
-(36) HashAggregate
+(35) HashAggregate
Input [2]: [l_extendedprice#X, l_discount#X]
Keys: []
Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))]
Aggregate Attributes [2]: [sum#X, isEmpty#X]
Results [2]: [sum#X, isEmpty#X]
-(37) HashAggregate
+(36) HashAggregate
Input [2]: [sum#X, isEmpty#X]
Keys: []
Functions [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))]
Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X]
Results [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X AS revenue#X]
-(38) AdaptiveSparkPlan
+(37) AdaptiveSparkPlan
Output [1]: [revenue#X]
Arguments: isFinalPlan=true
\ No newline at end of file
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 1fad7160bf64..9a2c56040b38 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
@@ -1,32 +1,32 @@
== Physical Plan ==
-AdaptiveSparkPlan (143)
+AdaptiveSparkPlan (142)
+- == Final Plan ==
- VeloxColumnarToRow (93)
- +- AQEShuffleRead (92)
- +- ShuffleQueryStage (91), Statistics(X)
- +- ColumnarExchange (90)
- +- VeloxResizeBatches (89)
- +- ^ ProjectExecTransformer (87)
- +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (86)
- :- ^ InputIteratorTransformer (76)
- : +- ShuffleQueryStage (74), Statistics(X)
- : +- ColumnarExchange (73)
- : +- VeloxResizeBatches (72)
- : +- ^ ProjectExecTransformer (70)
- : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (69)
+ VeloxColumnarToRow (92)
+ +- AQEShuffleRead (91)
+ +- ShuffleQueryStage (90), Statistics(X)
+ +- ColumnarExchange (89)
+ +- VeloxResizeBatches (88)
+ +- ^ ProjectExecTransformer (86)
+ +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (85)
+ :- ^ InputIteratorTransformer (75)
+ : +- ShuffleQueryStage (73), Statistics(X)
+ : +- ColumnarExchange (72)
+ : +- VeloxResizeBatches (71)
+ : +- ^ ProjectExecTransformer (69)
+ : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (68)
: :- ^ InputIteratorTransformer (9)
: : +- ShuffleQueryStage (7), Statistics(X)
: : +- ColumnarExchange (6)
: : +- VeloxResizeBatches (5)
: : +- ^ ProjectExecTransformer (3)
: : +- ^ FilterExecTransformer (2)
- : : +- ^ Scan parquet (1)
- : +- ^ InputIteratorTransformer (68)
- : +- ShuffleQueryStage (66), Statistics(X)
- : +- ColumnarExchange (65)
- : +- VeloxResizeBatches (64)
- : +- ^ ProjectExecTransformer (62)
- : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (61)
+ : : +- ^ ScanTransformer parquet (1)
+ : +- ^ InputIteratorTransformer (67)
+ : +- ShuffleQueryStage (65), Statistics(X)
+ : +- ColumnarExchange (64)
+ : +- VeloxResizeBatches (63)
+ : +- ^ ProjectExecTransformer (61)
+ : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (60)
: :- ^ InputIteratorTransformer (35)
: : +- ShuffleQueryStage (33), Statistics(X)
: : +- ColumnarExchange (32)
@@ -39,94 +39,93 @@ AdaptiveSparkPlan (143)
: : : +- VeloxResizeBatches (14)
: : : +- ^ ProjectExecTransformer (12)
: : : +- ^ FilterExecTransformer (11)
- : : : +- ^ Scan parquet (10)
+ : : : +- ^ ScanTransformer parquet (10)
: : +- ^ InputIteratorTransformer (27)
: : +- ShuffleQueryStage (25), Statistics(X)
: : +- ColumnarExchange (24)
: : +- VeloxResizeBatches (23)
: : +- ^ ProjectExecTransformer (21)
: : +- ^ FilterExecTransformer (20)
- : : +- ^ Scan parquet (19)
- : +- ^ InputIteratorTransformer (60)
- : +- ShuffleQueryStage (58), Statistics(X)
- : +- ColumnarExchange (57)
- : +- VeloxResizeBatches (56)
- : +- ^ ProjectExecTransformer (54)
- : +- ^ FilterExecTransformer (53)
- : +- ^ ProjectExecTransformer (52)
- : +- ^ RegularHashAggregateExecTransformer (51)
- : +- ^ RegularHashAggregateExecTransformer (50)
- : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (49)
- : :- ^ InputIteratorTransformer (44)
- : : +- ShuffleQueryStage (42), Statistics(X)
- : : +- ColumnarExchange (41)
- : : +- VeloxResizeBatches (40)
- : : +- ^ ProjectExecTransformer (38)
- : : +- ^ FilterExecTransformer (37)
- : : +- ^ Scan parquet (36)
- : +- ^ InputIteratorTransformer (48)
- : +- ShuffleQueryStage (46), Statistics(X)
- : +- ReusedExchange (45)
- +- ^ InputIteratorTransformer (85)
- +- ShuffleQueryStage (83), Statistics(X)
- +- ColumnarExchange (82)
- +- VeloxResizeBatches (81)
- +- ^ ProjectExecTransformer (79)
- +- ^ FilterExecTransformer (78)
- +- ^ Scan parquet (77)
+ : : +- ^ ScanTransformer parquet (19)
+ : +- ^ InputIteratorTransformer (59)
+ : +- ShuffleQueryStage (57), Statistics(X)
+ : +- ColumnarExchange (56)
+ : +- VeloxResizeBatches (55)
+ : +- ^ ProjectExecTransformer (53)
+ : +- ^ FilterExecTransformer (52)
+ : +- ^ ProjectExecTransformer (51)
+ : +- ^ RegularHashAggregateExecTransformer (50)
+ : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (49)
+ : :- ^ InputIteratorTransformer (44)
+ : : +- ShuffleQueryStage (42), Statistics(X)
+ : : +- ColumnarExchange (41)
+ : : +- VeloxResizeBatches (40)
+ : : +- ^ ProjectExecTransformer (38)
+ : : +- ^ FilterExecTransformer (37)
+ : : +- ^ ScanTransformer parquet (36)
+ : +- ^ InputIteratorTransformer (48)
+ : +- ShuffleQueryStage (46), Statistics(X)
+ : +- ReusedExchange (45)
+ +- ^ InputIteratorTransformer (84)
+ +- ShuffleQueryStage (82), Statistics(X)
+ +- ColumnarExchange (81)
+ +- VeloxResizeBatches (80)
+ +- ^ ProjectExecTransformer (78)
+ +- ^ FilterExecTransformer (77)
+ +- ^ ScanTransformer parquet (76)
+- == Initial Plan ==
- Sort (142)
- +- Exchange (141)
- +- Project (140)
- +- SortMergeJoin Inner (139)
- :- Sort (133)
- : +- Exchange (132)
- : +- Project (131)
- : +- SortMergeJoin LeftSemi (130)
- : :- Sort (97)
- : : +- Exchange (96)
- : : +- Filter (95)
- : : +- Scan parquet (94)
- : +- Sort (129)
- : +- Exchange (128)
- : +- Project (127)
- : +- SortMergeJoin Inner (126)
- : :- Sort (109)
- : : +- Exchange (108)
- : : +- SortMergeJoin LeftSemi (107)
- : : :- Sort (101)
- : : : +- Exchange (100)
- : : : +- Filter (99)
- : : : +- Scan parquet (98)
- : : +- Sort (106)
- : : +- Exchange (105)
- : : +- Project (104)
- : : +- Filter (103)
- : : +- Scan parquet (102)
- : +- Sort (125)
- : +- Exchange (124)
- : +- Filter (123)
- : +- HashAggregate (122)
- : +- HashAggregate (121)
- : +- SortMergeJoin LeftSemi (120)
- : :- Sort (114)
- : : +- Exchange (113)
- : : +- Project (112)
- : : +- Filter (111)
- : : +- Scan parquet (110)
- : +- Sort (119)
- : +- Exchange (118)
- : +- Project (117)
- : +- Filter (116)
- : +- Scan parquet (115)
- +- Sort (138)
- +- Exchange (137)
- +- Project (136)
- +- Filter (135)
- +- Scan parquet (134)
-
-
-(1) Scan parquet
+ Sort (141)
+ +- Exchange (140)
+ +- Project (139)
+ +- SortMergeJoin Inner (138)
+ :- Sort (132)
+ : +- Exchange (131)
+ : +- Project (130)
+ : +- SortMergeJoin LeftSemi (129)
+ : :- Sort (96)
+ : : +- Exchange (95)
+ : : +- Filter (94)
+ : : +- Scan parquet (93)
+ : +- Sort (128)
+ : +- Exchange (127)
+ : +- Project (126)
+ : +- SortMergeJoin Inner (125)
+ : :- Sort (108)
+ : : +- Exchange (107)
+ : : +- SortMergeJoin LeftSemi (106)
+ : : :- Sort (100)
+ : : : +- Exchange (99)
+ : : : +- Filter (98)
+ : : : +- Scan parquet (97)
+ : : +- Sort (105)
+ : : +- Exchange (104)
+ : : +- Project (103)
+ : : +- Filter (102)
+ : : +- Scan parquet (101)
+ : +- Sort (124)
+ : +- Exchange (123)
+ : +- Filter (122)
+ : +- HashAggregate (121)
+ : +- HashAggregate (120)
+ : +- SortMergeJoin LeftSemi (119)
+ : :- Sort (113)
+ : : +- Exchange (112)
+ : : +- Project (111)
+ : : +- Filter (110)
+ : : +- Scan parquet (109)
+ : +- Sort (118)
+ : +- Exchange (117)
+ : +- Project (116)
+ : +- Filter (115)
+ : +- Scan parquet (114)
+ +- Sort (137)
+ +- Exchange (136)
+ +- Project (135)
+ +- Filter (134)
+ +- Scan parquet (133)
+
+
+(1) ScanTransformer parquet
Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -163,7 +162,7 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X]
(9) InputIteratorTransformer
Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -200,7 +199,7 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
(18) InputIteratorTransformer
Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
-(19) Scan parquet
+(19) ScanTransformer parquet
Output [2]: [p_partkey#X, p_name#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -268,7 +267,7 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
(35) InputIteratorTransformer
Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
-(36) Scan parquet
+(36) ScanTransformer parquet
Output [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -326,407 +325,400 @@ Join condition: None
(50) RegularHashAggregateExecTransformer
Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X]
Keys [2]: [l_partkey#X, l_suppkey#X]
-Functions [1]: [partial_sum(l_quantity#X)]
-Aggregate Attributes [2]: [sum#X, isEmpty#X]
-Results [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X]
-
-(51) RegularHashAggregateExecTransformer
-Input [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X]
-Keys [2]: [l_partkey#X, l_suppkey#X]
Functions [1]: [sum(l_quantity#X)]
Aggregate Attributes [1]: [sum(l_quantity#X)#X]
Results [3]: [l_partkey#X, l_suppkey#X, sum(l_quantity#X)#X]
-(52) ProjectExecTransformer
+(51) ProjectExecTransformer
Output [3]: [CheckOverflow((0.50 * promote_precision(sum(l_quantity#X)#X)), DecimalType(24,3)) AS (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
Input [3]: [l_partkey#X, l_suppkey#X, sum(l_quantity#X)#X]
-(53) FilterExecTransformer
+(52) FilterExecTransformer
Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
Arguments: isnotnull((0.5 * sum(l_quantity))#X)
-(54) ProjectExecTransformer
+(53) ProjectExecTransformer
Output [4]: [hash(l_partkey#X, l_suppkey#X, 42) AS hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
-(55) WholeStageCodegenTransformer (X)
+(54) WholeStageCodegenTransformer (X)
Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
Arguments: false
-(56) VeloxResizeBatches
+(55) VeloxResizeBatches
Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
Arguments: X, X
-(57) ColumnarExchange
+(56) ColumnarExchange
Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
Arguments: hashpartitioning(l_partkey#X, l_suppkey#X, 1), ENSURE_REQUIREMENTS, [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X], [plan_id=X], [shuffle_writer_type=hash]
-(58) ShuffleQueryStage
+(57) ShuffleQueryStage
Output [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
Arguments: X
-(59) InputAdapter
+(58) InputAdapter
Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
-(60) InputIteratorTransformer
+(59) InputIteratorTransformer
Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
-(61) ShuffledHashJoinExecTransformer
+(60) ShuffledHashJoinExecTransformer
Left keys [2]: [ps_partkey#X, ps_suppkey#X]
Right keys [2]: [l_partkey#X, l_suppkey#X]
Join condition: (cast(ps_availqty#X as decimal(24,3)) > (0.5 * sum(l_quantity))#X)
-(62) ProjectExecTransformer
+(61) ProjectExecTransformer
Output [2]: [hash(ps_suppkey#X, 42) AS hash_partition_key#X, ps_suppkey#X]
Input [6]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
-(63) WholeStageCodegenTransformer (X)
+(62) WholeStageCodegenTransformer (X)
Input [2]: [hash_partition_key#X, ps_suppkey#X]
Arguments: false
-(64) VeloxResizeBatches
+(63) VeloxResizeBatches
Input [2]: [hash_partition_key#X, ps_suppkey#X]
Arguments: X, X
-(65) ColumnarExchange
+(64) ColumnarExchange
Input [2]: [hash_partition_key#X, ps_suppkey#X]
Arguments: hashpartitioning(ps_suppkey#X, 1), ENSURE_REQUIREMENTS, [ps_suppkey#X], [plan_id=X], [shuffle_writer_type=hash]
-(66) ShuffleQueryStage
+(65) ShuffleQueryStage
Output [1]: [ps_suppkey#X]
Arguments: X
-(67) InputAdapter
+(66) InputAdapter
Input [1]: [ps_suppkey#X]
-(68) InputIteratorTransformer
+(67) InputIteratorTransformer
Input [1]: [ps_suppkey#X]
-(69) ShuffledHashJoinExecTransformer
+(68) ShuffledHashJoinExecTransformer
Left keys [1]: [s_suppkey#X]
Right keys [1]: [ps_suppkey#X]
Join condition: None
-(70) ProjectExecTransformer
+(69) ProjectExecTransformer
Output [4]: [hash(s_nationkey#X, 42) AS hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X]
Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X]
-(71) WholeStageCodegenTransformer (X)
+(70) WholeStageCodegenTransformer (X)
Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X]
Arguments: false
-(72) VeloxResizeBatches
+(71) VeloxResizeBatches
Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X]
Arguments: X, X
-(73) ColumnarExchange
+(72) ColumnarExchange
Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X]
Arguments: hashpartitioning(s_nationkey#X, 1), ENSURE_REQUIREMENTS, [s_name#X, s_address#X, s_nationkey#X], [plan_id=X], [shuffle_writer_type=hash]
-(74) ShuffleQueryStage
+(73) ShuffleQueryStage
Output [3]: [s_name#X, s_address#X, s_nationkey#X]
Arguments: X
-(75) InputAdapter
+(74) InputAdapter
Input [3]: [s_name#X, s_address#X, s_nationkey#X]
-(76) InputIteratorTransformer
+(75) InputIteratorTransformer
Input [3]: [s_name#X, s_address#X, s_nationkey#X]
-(77) Scan parquet
+(76) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(n_name), EqualTo(n_name,CANADA), IsNotNull(n_nationkey)]
ReadSchema: struct
-(78) FilterExecTransformer
+(77) FilterExecTransformer
Input [2]: [n_nationkey#X, n_name#X]
Arguments: ((isnotnull(n_name#X) AND (n_name#X = CANADA)) AND isnotnull(n_nationkey#X))
-(79) ProjectExecTransformer
+(78) ProjectExecTransformer
Output [2]: [hash(n_nationkey#X, 42) AS hash_partition_key#X, n_nationkey#X]
Input [2]: [n_nationkey#X, n_name#X]
-(80) WholeStageCodegenTransformer (X)
+(79) WholeStageCodegenTransformer (X)
Input [2]: [hash_partition_key#X, n_nationkey#X]
Arguments: false
-(81) VeloxResizeBatches
+(80) VeloxResizeBatches
Input [2]: [hash_partition_key#X, n_nationkey#X]
Arguments: X, X
-(82) ColumnarExchange
+(81) ColumnarExchange
Input [2]: [hash_partition_key#X, n_nationkey#X]
Arguments: hashpartitioning(n_nationkey#X, 1), ENSURE_REQUIREMENTS, [n_nationkey#X], [plan_id=X], [shuffle_writer_type=hash]
-(83) ShuffleQueryStage
+(82) ShuffleQueryStage
Output [1]: [n_nationkey#X]
Arguments: X
-(84) InputAdapter
+(83) InputAdapter
Input [1]: [n_nationkey#X]
-(85) InputIteratorTransformer
+(84) InputIteratorTransformer
Input [1]: [n_nationkey#X]
-(86) ShuffledHashJoinExecTransformer
+(85) ShuffledHashJoinExecTransformer
Left keys [1]: [s_nationkey#X]
Right keys [1]: [n_nationkey#X]
Join condition: None
-(87) ProjectExecTransformer
+(86) ProjectExecTransformer
Output [2]: [s_name#X, s_address#X]
Input [4]: [s_name#X, s_address#X, s_nationkey#X, n_nationkey#X]
-(88) WholeStageCodegenTransformer (X)
+(87) WholeStageCodegenTransformer (X)
Input [2]: [s_name#X, s_address#X]
Arguments: false
-(89) VeloxResizeBatches
+(88) VeloxResizeBatches
Input [2]: [s_name#X, s_address#X]
Arguments: X, X
-(90) ColumnarExchange
+(89) ColumnarExchange
Input [2]: [s_name#X, s_address#X]
Arguments: rangepartitioning(s_name#X ASC NULLS FIRST, 1), ENSURE_REQUIREMENTS, [plan_id=X], [shuffle_writer_type=hash]
-(91) ShuffleQueryStage
+(90) ShuffleQueryStage
Output [2]: [s_name#X, s_address#X]
Arguments: X
-(92) AQEShuffleRead
+(91) AQEShuffleRead
Input [2]: [s_name#X, s_address#X]
Arguments: local
-(93) VeloxColumnarToRow
+(92) VeloxColumnarToRow
Input [2]: [s_name#X, s_address#X]
-(94) Scan parquet
+(93) Scan parquet
Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(s_nationkey)]
ReadSchema: struct
-(95) Filter
+(94) Filter
Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X]
Condition : isnotnull(s_nationkey#X)
-(96) Exchange
+(95) Exchange
Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X]
Arguments: hashpartitioning(s_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(97) Sort
+(96) Sort
Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X]
Arguments: [s_suppkey#X ASC NULLS FIRST], false, 0
-(98) Scan parquet
+(97) Scan parquet
Output [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(ps_availqty), IsNotNull(ps_partkey), IsNotNull(ps_suppkey)]
ReadSchema: struct
-(99) Filter
+(98) Filter
Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
Condition : ((isnotnull(ps_availqty#X) AND isnotnull(ps_partkey#X)) AND isnotnull(ps_suppkey#X))
-(100) Exchange
+(99) Exchange
Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
Arguments: hashpartitioning(ps_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(101) Sort
+(100) Sort
Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
Arguments: [ps_partkey#X ASC NULLS FIRST], false, 0
-(102) Scan parquet
+(101) Scan parquet
Output [2]: [p_partkey#X, p_name#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(p_name), StringStartsWith(p_name,forest)]
ReadSchema: struct
-(103) Filter
+(102) Filter
Input [2]: [p_partkey#X, p_name#X]
Condition : (isnotnull(p_name#X) AND StartsWith(p_name#X, forest))
-(104) Project
+(103) Project
Output [1]: [p_partkey#X]
Input [2]: [p_partkey#X, p_name#X]
-(105) Exchange
+(104) Exchange
Input [1]: [p_partkey#X]
Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(106) Sort
+(105) Sort
Input [1]: [p_partkey#X]
Arguments: [p_partkey#X ASC NULLS FIRST], false, 0
-(107) SortMergeJoin
+(106) SortMergeJoin
Left keys [1]: [ps_partkey#X]
Right keys [1]: [p_partkey#X]
Join condition: None
-(108) Exchange
+(107) Exchange
Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
Arguments: hashpartitioning(ps_partkey#X, ps_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(109) Sort
+(108) Sort
Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
Arguments: [ps_partkey#X ASC NULLS FIRST, ps_suppkey#X ASC NULLS FIRST], false, 0
-(110) Scan parquet
+(109) Scan parquet
Output [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1994-01-01), LessThan(l_shipdate,1995-01-01), IsNotNull(l_partkey), IsNotNull(l_suppkey)]
ReadSchema: struct
-(111) Filter
+(110) Filter
Input [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X]
Condition : ((((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1994-01-01)) AND (l_shipdate#X < 1995-01-01)) AND isnotnull(l_partkey#X)) AND isnotnull(l_suppkey#X))
-(112) Project
+(111) Project
Output [3]: [l_partkey#X, l_suppkey#X, l_quantity#X]
Input [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X]
-(113) Exchange
+(112) Exchange
Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X]
Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(114) Sort
+(113) Sort
Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X]
Arguments: [l_partkey#X ASC NULLS FIRST], false, 0
-(115) Scan parquet
+(114) Scan parquet
Output [2]: [p_partkey#X, p_name#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(p_name), StringStartsWith(p_name,forest)]
ReadSchema: struct
-(116) Filter
+(115) Filter
Input [2]: [p_partkey#X, p_name#X]
Condition : (isnotnull(p_name#X) AND StartsWith(p_name#X, forest))
-(117) Project
+(116) Project
Output [1]: [p_partkey#X]
Input [2]: [p_partkey#X, p_name#X]
-(118) Exchange
+(117) Exchange
Input [1]: [p_partkey#X]
Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(119) Sort
+(118) Sort
Input [1]: [p_partkey#X]
Arguments: [p_partkey#X ASC NULLS FIRST], false, 0
-(120) SortMergeJoin
+(119) SortMergeJoin
Left keys [1]: [l_partkey#X]
Right keys [1]: [p_partkey#X]
Join condition: None
-(121) HashAggregate
+(120) HashAggregate
Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X]
Keys [2]: [l_partkey#X, l_suppkey#X]
Functions [1]: [partial_sum(l_quantity#X)]
Aggregate Attributes [2]: [sum#X, isEmpty#X]
Results [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X]
-(122) HashAggregate
+(121) HashAggregate
Input [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X]
Keys [2]: [l_partkey#X, l_suppkey#X]
Functions [1]: [sum(l_quantity#X)]
Aggregate Attributes [1]: [sum(l_quantity#X)#X]
Results [3]: [CheckOverflow((0.50 * promote_precision(sum(l_quantity#X)#X)), DecimalType(24,3)) AS (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
-(123) Filter
+(122) Filter
Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
Condition : isnotnull((0.5 * sum(l_quantity))#X)
-(124) Exchange
+(123) Exchange
Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
Arguments: hashpartitioning(l_partkey#X, l_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(125) Sort
+(124) Sort
Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
Arguments: [l_partkey#X ASC NULLS FIRST, l_suppkey#X ASC NULLS FIRST], false, 0
-(126) SortMergeJoin
+(125) SortMergeJoin
Left keys [2]: [ps_partkey#X, ps_suppkey#X]
Right keys [2]: [l_partkey#X, l_suppkey#X]
Join condition: (cast(ps_availqty#X as decimal(24,3)) > (0.5 * sum(l_quantity))#X)
-(127) Project
+(126) Project
Output [1]: [ps_suppkey#X]
Input [6]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
-(128) Exchange
+(127) Exchange
Input [1]: [ps_suppkey#X]
Arguments: hashpartitioning(ps_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(129) Sort
+(128) Sort
Input [1]: [ps_suppkey#X]
Arguments: [ps_suppkey#X ASC NULLS FIRST], false, 0
-(130) SortMergeJoin
+(129) SortMergeJoin
Left keys [1]: [s_suppkey#X]
Right keys [1]: [ps_suppkey#X]
Join condition: None
-(131) Project
+(130) Project
Output [3]: [s_name#X, s_address#X, s_nationkey#X]
Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X]
-(132) Exchange
+(131) Exchange
Input [3]: [s_name#X, s_address#X, s_nationkey#X]
Arguments: hashpartitioning(s_nationkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(133) Sort
+(132) Sort
Input [3]: [s_name#X, s_address#X, s_nationkey#X]
Arguments: [s_nationkey#X ASC NULLS FIRST], false, 0
-(134) Scan parquet
+(133) Scan parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(n_name), EqualTo(n_name,CANADA), IsNotNull(n_nationkey)]
ReadSchema: struct
-(135) Filter
+(134) Filter
Input [2]: [n_nationkey#X, n_name#X]
Condition : ((isnotnull(n_name#X) AND (n_name#X = CANADA)) AND isnotnull(n_nationkey#X))
-(136) Project
+(135) Project
Output [1]: [n_nationkey#X]
Input [2]: [n_nationkey#X, n_name#X]
-(137) Exchange
+(136) Exchange
Input [1]: [n_nationkey#X]
Arguments: hashpartitioning(n_nationkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(138) Sort
+(137) Sort
Input [1]: [n_nationkey#X]
Arguments: [n_nationkey#X ASC NULLS FIRST], false, 0
-(139) SortMergeJoin
+(138) SortMergeJoin
Left keys [1]: [s_nationkey#X]
Right keys [1]: [n_nationkey#X]
Join condition: None
-(140) Project
+(139) Project
Output [2]: [s_name#X, s_address#X]
Input [4]: [s_name#X, s_address#X, s_nationkey#X, n_nationkey#X]
-(141) Exchange
+(140) Exchange
Input [2]: [s_name#X, s_address#X]
Arguments: rangepartitioning(s_name#X ASC NULLS FIRST, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(142) Sort
+(141) Sort
Input [2]: [s_name#X, s_address#X]
Arguments: [s_name#X ASC NULLS FIRST], true, 0
-(143) AdaptiveSparkPlan
+(142) AdaptiveSparkPlan
Output [2]: [s_name#X, s_address#X]
Arguments: isFinalPlan=true
\ No newline at end of file
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 fd61236faaf8..a675841a475a 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
@@ -29,7 +29,7 @@ AdaptiveSparkPlan (137)
: : : +- VeloxResizeBatches (5)
: : : +- ^ ProjectExecTransformer (3)
: : : +- ^ FilterExecTransformer (2)
- : : : +- ^ Scan parquet (1)
+ : : : +- ^ ScanTransformer parquet (1)
: : +- ^ InputIteratorTransformer (44)
: : +- ShuffleQueryStage (42), Statistics(X)
: : +- ColumnarExchange (41)
@@ -43,34 +43,34 @@ AdaptiveSparkPlan (137)
: : : : +- VeloxResizeBatches (14)
: : : : +- ^ ProjectExecTransformer (12)
: : : : +- ^ FilterExecTransformer (11)
- : : : : +- ^ Scan parquet (10)
+ : : : : +- ^ ScanTransformer parquet (10)
: : : +- ^ InputIteratorTransformer (26)
: : : +- ShuffleQueryStage (24), Statistics(X)
: : : +- ColumnarExchange (23)
: : : +- VeloxResizeBatches (22)
: : : +- ^ ProjectExecTransformer (20)
- : : : +- ^ Scan parquet (19)
+ : : : +- ^ ScanTransformer parquet (19)
: : +- ^ InputIteratorTransformer (36)
: : +- ShuffleQueryStage (34), Statistics(X)
: : +- ColumnarExchange (33)
: : +- VeloxResizeBatches (32)
: : +- ^ ProjectExecTransformer (30)
: : +- ^ FilterExecTransformer (29)
- : : +- ^ Scan parquet (28)
+ : : +- ^ ScanTransformer parquet (28)
: +- ^ InputIteratorTransformer (61)
: +- ShuffleQueryStage (59), Statistics(X)
: +- ColumnarExchange (58)
: +- VeloxResizeBatches (57)
: +- ^ ProjectExecTransformer (55)
: +- ^ FilterExecTransformer (54)
- : +- ^ Scan parquet (53)
+ : +- ^ ScanTransformer parquet (53)
+- ^ InputIteratorTransformer (78)
+- ShuffleQueryStage (76), Statistics(X)
+- ColumnarExchange (75)
+- VeloxResizeBatches (74)
+- ^ ProjectExecTransformer (72)
+- ^ FilterExecTransformer (71)
- +- ^ Scan parquet (70)
+ +- ^ ScanTransformer parquet (70)
+- == Initial Plan ==
TakeOrderedAndProject (136)
+- HashAggregate (135)
@@ -119,7 +119,7 @@ AdaptiveSparkPlan (137)
+- Scan parquet (126)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [3]: [s_suppkey#X, s_name#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -156,7 +156,7 @@ Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X]
(9) InputIteratorTransformer
Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -193,7 +193,7 @@ Input [2]: [l_orderkey#X, l_suppkey#X]
(18) InputIteratorTransformer
Input [2]: [l_orderkey#X, l_suppkey#X]
-(19) Scan parquet
+(19) ScanTransformer parquet
Output [2]: [l_orderkey#X, l_suppkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -230,7 +230,7 @@ Left keys [1]: [l_orderkey#X]
Right keys [1]: [l_orderkey#X]
Join condition: NOT (l_suppkey#X = l_suppkey#X)
-(28) Scan parquet
+(28) ScanTransformer parquet
Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -329,7 +329,7 @@ Input [3]: [s_name#X, s_nationkey#X, l_orderkey#X]
(52) InputIteratorTransformer
Input [3]: [s_name#X, s_nationkey#X, l_orderkey#X]
-(53) Scan parquet
+(53) ScanTransformer parquet
Output [2]: [o_orderkey#X, o_orderstatus#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -397,7 +397,7 @@ Input [2]: [s_name#X, s_nationkey#X]
(69) InputIteratorTransformer
Input [2]: [s_name#X, s_nationkey#X]
-(70) Scan parquet
+(70) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 44c57da7482f..c48c13779d64 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
@@ -22,13 +22,13 @@ AdaptiveSparkPlan (52)
: +- VeloxResizeBatches (5)
: +- ^ ProjectExecTransformer (3)
: +- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
+ : +- ^ ScanTransformer parquet (1)
+- ^ InputIteratorTransformer (17)
+- ShuffleQueryStage (15), Statistics(X)
+- ColumnarExchange (14)
+- VeloxResizeBatches (13)
+- ^ ProjectExecTransformer (11)
- +- ^ Scan parquet (10)
+ +- ^ ScanTransformer parquet (10)
+- == Initial Plan ==
Sort (51)
+- Exchange (50)
@@ -46,7 +46,7 @@ AdaptiveSparkPlan (52)
+- Scan parquet (42)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [3]: [c_custkey#X, c_phone#X, c_acctbal#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -83,7 +83,7 @@ Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X]
(9) InputIteratorTransformer
Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [1]: [o_custkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -283,7 +283,7 @@ AdaptiveSparkPlan (72)
+- ^ FlushableHashAggregateExecTransformer (56)
+- ^ ProjectExecTransformer (55)
+- ^ FilterExecTransformer (54)
- +- ^ Scan parquet (53)
+ +- ^ ScanTransformer parquet (53)
+- == Initial Plan ==
HashAggregate (71)
+- Exchange (70)
@@ -293,7 +293,7 @@ AdaptiveSparkPlan (72)
+- Scan parquet (66)
-(53) Scan parquet
+(53) ScanTransformer parquet
Output [2]: [c_phone#X, c_acctbal#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -400,7 +400,7 @@ AdaptiveSparkPlan (72)
+- ^ FlushableHashAggregateExecTransformer (56)
+- ^ ProjectExecTransformer (55)
+- ^ FilterExecTransformer (54)
- +- ^ Scan parquet (53)
+ +- ^ ScanTransformer parquet (53)
+- == Initial Plan ==
HashAggregate (71)
+- Exchange (70)
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 a698975568fb..d09fe3e19a9b 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
@@ -1,67 +1,66 @@
== Physical Plan ==
-AdaptiveSparkPlan (67)
+AdaptiveSparkPlan (66)
+- == Final Plan ==
- VeloxColumnarToRow (43)
- +- TakeOrderedAndProjectExecTransformer (42)
- +- ^ ProjectExecTransformer (40)
- +- ^ RegularHashAggregateExecTransformer (39)
- +- ^ RegularHashAggregateExecTransformer (38)
- +- ^ ProjectExecTransformer (37)
- +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (36)
- :- ^ InputIteratorTransformer (26)
- : +- ShuffleQueryStage (24), Statistics(X)
- : +- ColumnarExchange (23)
- : +- VeloxResizeBatches (22)
- : +- ^ ProjectExecTransformer (20)
- : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19)
- : :- ^ InputIteratorTransformer (9)
- : : +- ShuffleQueryStage (7), Statistics(X)
- : : +- ColumnarExchange (6)
- : : +- VeloxResizeBatches (5)
- : : +- ^ ProjectExecTransformer (3)
- : : +- ^ FilterExecTransformer (2)
- : : +- ^ Scan parquet (1)
- : +- ^ InputIteratorTransformer (18)
- : +- ShuffleQueryStage (16), Statistics(X)
- : +- ColumnarExchange (15)
- : +- VeloxResizeBatches (14)
- : +- ^ ProjectExecTransformer (12)
- : +- ^ FilterExecTransformer (11)
- : +- ^ Scan parquet (10)
- +- ^ InputIteratorTransformer (35)
- +- ShuffleQueryStage (33), Statistics(X)
- +- ColumnarExchange (32)
- +- VeloxResizeBatches (31)
- +- ^ ProjectExecTransformer (29)
- +- ^ FilterExecTransformer (28)
- +- ^ Scan parquet (27)
+ VeloxColumnarToRow (42)
+ +- TakeOrderedAndProjectExecTransformer (41)
+ +- ^ ProjectExecTransformer (39)
+ +- ^ RegularHashAggregateExecTransformer (38)
+ +- ^ ProjectExecTransformer (37)
+ +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (36)
+ :- ^ InputIteratorTransformer (26)
+ : +- ShuffleQueryStage (24), Statistics(X)
+ : +- ColumnarExchange (23)
+ : +- VeloxResizeBatches (22)
+ : +- ^ ProjectExecTransformer (20)
+ : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19)
+ : :- ^ InputIteratorTransformer (9)
+ : : +- ShuffleQueryStage (7), Statistics(X)
+ : : +- ColumnarExchange (6)
+ : : +- VeloxResizeBatches (5)
+ : : +- ^ ProjectExecTransformer (3)
+ : : +- ^ FilterExecTransformer (2)
+ : : +- ^ ScanTransformer parquet (1)
+ : +- ^ InputIteratorTransformer (18)
+ : +- ShuffleQueryStage (16), Statistics(X)
+ : +- ColumnarExchange (15)
+ : +- VeloxResizeBatches (14)
+ : +- ^ ProjectExecTransformer (12)
+ : +- ^ FilterExecTransformer (11)
+ : +- ^ ScanTransformer parquet (10)
+ +- ^ InputIteratorTransformer (35)
+ +- ShuffleQueryStage (33), Statistics(X)
+ +- ColumnarExchange (32)
+ +- VeloxResizeBatches (31)
+ +- ^ ProjectExecTransformer (29)
+ +- ^ FilterExecTransformer (28)
+ +- ^ ScanTransformer parquet (27)
+- == Initial Plan ==
- TakeOrderedAndProject (66)
- +- HashAggregate (65)
- +- HashAggregate (64)
- +- Project (63)
- +- SortMergeJoin Inner (62)
- :- Sort (56)
- : +- Exchange (55)
- : +- Project (54)
- : +- SortMergeJoin Inner (53)
- : :- Sort (48)
- : : +- Exchange (47)
- : : +- Project (46)
- : : +- Filter (45)
- : : +- Scan parquet (44)
- : +- Sort (52)
- : +- Exchange (51)
- : +- Filter (50)
- : +- Scan parquet (49)
- +- Sort (61)
- +- Exchange (60)
- +- Project (59)
- +- Filter (58)
- +- Scan parquet (57)
-
-
-(1) Scan parquet
+ TakeOrderedAndProject (65)
+ +- HashAggregate (64)
+ +- HashAggregate (63)
+ +- Project (62)
+ +- SortMergeJoin Inner (61)
+ :- Sort (55)
+ : +- Exchange (54)
+ : +- Project (53)
+ : +- SortMergeJoin Inner (52)
+ : :- Sort (47)
+ : : +- Exchange (46)
+ : : +- Project (45)
+ : : +- Filter (44)
+ : : +- Scan parquet (43)
+ : +- Sort (51)
+ : +- Exchange (50)
+ : +- Filter (49)
+ : +- Scan parquet (48)
+ +- Sort (60)
+ +- Exchange (59)
+ +- Project (58)
+ +- Filter (57)
+ +- Scan parquet (56)
+
+
+(1) ScanTransformer parquet
Output [2]: [c_custkey#X, c_mktsegment#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -98,7 +97,7 @@ Input [1]: [c_custkey#X]
(9) InputIteratorTransformer
Input [1]: [c_custkey#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -166,7 +165,7 @@ Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X]
(26) InputIteratorTransformer
Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X]
-(27) Scan parquet
+(27) ScanTransformer parquet
Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -215,141 +214,134 @@ Input [6]: [o_orderkey#X, o_orderdate#X, o_shippriority#X, l_orderkey#X, l_exten
(38) RegularHashAggregateExecTransformer
Input [6]: [o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X, _pre_X#X]
Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X]
-Functions [1]: [partial_sum(_pre_X#X)]
-Aggregate Attributes [2]: [sum#X, isEmpty#X]
-Results [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X]
-
-(39) RegularHashAggregateExecTransformer
-Input [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X]
-Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X]
-Functions [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))]
+Functions [1]: [sum(_pre_X#X)]
Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X]
Results [4]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X]
-(40) ProjectExecTransformer
+(39) ProjectExecTransformer
Output [4]: [l_orderkey#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X AS revenue#X, o_orderdate#X, o_shippriority#X]
Input [4]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X]
-(41) WholeStageCodegenTransformer (X)
+(40) WholeStageCodegenTransformer (X)
Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X]
Arguments: false
-(42) TakeOrderedAndProjectExecTransformer
+(41) TakeOrderedAndProjectExecTransformer
Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X]
Arguments: X, [revenue#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X], 0
-(43) VeloxColumnarToRow
+(42) VeloxColumnarToRow
Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X]
-(44) Scan parquet
+(43) Scan parquet
Output [2]: [c_custkey#X, c_mktsegment#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(c_mktsegment), EqualTo(c_mktsegment,BUILDING), IsNotNull(c_custkey)]
ReadSchema: struct
-(45) Filter
+(44) Filter
Input [2]: [c_custkey#X, c_mktsegment#X]
Condition : ((isnotnull(c_mktsegment#X) AND (c_mktsegment#X = BUILDING)) AND isnotnull(c_custkey#X))
-(46) Project
+(45) Project
Output [1]: [c_custkey#X]
Input [2]: [c_custkey#X, c_mktsegment#X]
-(47) Exchange
+(46) Exchange
Input [1]: [c_custkey#X]
Arguments: hashpartitioning(c_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(48) Sort
+(47) Sort
Input [1]: [c_custkey#X]
Arguments: [c_custkey#X ASC NULLS FIRST], false, 0
-(49) Scan parquet
+(48) Scan parquet
Output [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(o_orderdate), LessThan(o_orderdate,1995-03-15), IsNotNull(o_custkey), IsNotNull(o_orderkey)]
ReadSchema: struct
-(50) Filter
+(49) Filter
Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X]
Condition : (((isnotnull(o_orderdate#X) AND (o_orderdate#X < 1995-03-15)) AND isnotnull(o_custkey#X)) AND isnotnull(o_orderkey#X))
-(51) Exchange
+(50) Exchange
Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X]
Arguments: hashpartitioning(o_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(52) Sort
+(51) Sort
Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X]
Arguments: [o_custkey#X ASC NULLS FIRST], false, 0
-(53) SortMergeJoin
+(52) SortMergeJoin
Left keys [1]: [c_custkey#X]
Right keys [1]: [o_custkey#X]
Join condition: None
-(54) Project
+(53) Project
Output [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X]
Input [5]: [c_custkey#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X]
-(55) Exchange
+(54) Exchange
Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X]
Arguments: hashpartitioning(o_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(56) Sort
+(55) Sort
Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X]
Arguments: [o_orderkey#X ASC NULLS FIRST], false, 0
-(57) Scan parquet
+(56) Scan parquet
Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(l_shipdate), GreaterThan(l_shipdate,1995-03-15), IsNotNull(l_orderkey)]
ReadSchema: struct
-(58) Filter
+(57) Filter
Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Condition : ((isnotnull(l_shipdate#X) AND (l_shipdate#X > 1995-03-15)) AND isnotnull(l_orderkey#X))
-(59) Project
+(58) Project
Output [3]: [l_orderkey#X, l_extendedprice#X, l_discount#X]
Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
-(60) Exchange
+(59) Exchange
Input [3]: [l_orderkey#X, l_extendedprice#X, l_discount#X]
Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(61) Sort
+(60) Sort
Input [3]: [l_orderkey#X, l_extendedprice#X, l_discount#X]
Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0
-(62) SortMergeJoin
+(61) SortMergeJoin
Left keys [1]: [o_orderkey#X]
Right keys [1]: [l_orderkey#X]
Join condition: None
-(63) Project
+(62) Project
Output [5]: [o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X]
Input [6]: [o_orderkey#X, o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X]
-(64) HashAggregate
+(63) HashAggregate
Input [5]: [o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X]
Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X]
Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))]
Aggregate Attributes [2]: [sum#X, isEmpty#X]
Results [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X]
-(65) HashAggregate
+(64) HashAggregate
Input [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X]
Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X]
Functions [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))]
Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X]
Results [4]: [l_orderkey#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X AS revenue#X, o_orderdate#X, o_shippriority#X]
-(66) TakeOrderedAndProject
+(65) TakeOrderedAndProject
Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X]
Arguments: X, [revenue#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X]
-(67) AdaptiveSparkPlan
+(66) AdaptiveSparkPlan
Output [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X]
Arguments: isFinalPlan=true
\ No newline at end of file
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 8b00d867cd70..23fb123956dc 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
@@ -22,14 +22,14 @@ AdaptiveSparkPlan (56)
: +- VeloxResizeBatches (5)
: +- ^ ProjectExecTransformer (3)
: +- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
+ : +- ^ ScanTransformer parquet (1)
+- ^ InputIteratorTransformer (18)
+- ShuffleQueryStage (16), Statistics(X)
+- ColumnarExchange (15)
+- VeloxResizeBatches (14)
+- ^ ProjectExecTransformer (12)
+- ^ FilterExecTransformer (11)
- +- ^ Scan parquet (10)
+ +- ^ ScanTransformer parquet (10)
+- == Initial Plan ==
Sort (55)
+- Exchange (54)
@@ -50,7 +50,7 @@ AdaptiveSparkPlan (56)
+- Scan parquet (44)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -87,7 +87,7 @@ Input [2]: [o_orderkey#X, o_orderpriority#X]
(9) InputIteratorTransformer
Input [2]: [o_orderkey#X, o_orderpriority#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [3]: [l_orderkey#X, l_commitdate#X, l_receiptdate#X]
Batched: true
Location: InMemoryFileIndex [*]
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 dd869d25d416..ea8f7dbd3371 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
@@ -46,42 +46,42 @@ AdaptiveSparkPlan (156)
: : : : : +- VeloxResizeBatches (5)
: : : : : +- ^ ProjectExecTransformer (3)
: : : : : +- ^ FilterExecTransformer (2)
- : : : : : +- ^ Scan parquet (1)
+ : : : : : +- ^ ScanTransformer parquet (1)
: : : : +- ^ InputIteratorTransformer (18)
: : : : +- ShuffleQueryStage (16), Statistics(X)
: : : : +- ColumnarExchange (15)
: : : : +- VeloxResizeBatches (14)
: : : : +- ^ ProjectExecTransformer (12)
: : : : +- ^ FilterExecTransformer (11)
- : : : : +- ^ Scan parquet (10)
+ : : : : +- ^ ScanTransformer parquet (10)
: : : +- ^ InputIteratorTransformer (35)
: : : +- ShuffleQueryStage (33), Statistics(X)
: : : +- ColumnarExchange (32)
: : : +- VeloxResizeBatches (31)
: : : +- ^ ProjectExecTransformer (29)
: : : +- ^ FilterExecTransformer (28)
- : : : +- ^ Scan parquet (27)
+ : : : +- ^ ScanTransformer parquet (27)
: : +- ^ InputIteratorTransformer (52)
: : +- ShuffleQueryStage (50), Statistics(X)
: : +- ColumnarExchange (49)
: : +- VeloxResizeBatches (48)
: : +- ^ ProjectExecTransformer (46)
: : +- ^ FilterExecTransformer (45)
- : : +- ^ Scan parquet (44)
+ : : +- ^ ScanTransformer parquet (44)
: +- ^ InputIteratorTransformer (69)
: +- ShuffleQueryStage (67), Statistics(X)
: +- ColumnarExchange (66)
: +- VeloxResizeBatches (65)
: +- ^ ProjectExecTransformer (63)
: +- ^ FilterExecTransformer (62)
- : +- ^ Scan parquet (61)
+ : +- ^ ScanTransformer parquet (61)
+- ^ InputIteratorTransformer (86)
+- ShuffleQueryStage (84), Statistics(X)
+- ColumnarExchange (83)
+- VeloxResizeBatches (82)
+- ^ ProjectExecTransformer (80)
+- ^ FilterExecTransformer (79)
- +- ^ Scan parquet (78)
+ +- ^ ScanTransformer parquet (78)
+- == Initial Plan ==
Sort (155)
+- Exchange (154)
@@ -134,7 +134,7 @@ AdaptiveSparkPlan (156)
+- Scan parquet (144)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [c_custkey#X, c_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -171,7 +171,7 @@ Input [2]: [c_custkey#X, c_nationkey#X]
(9) InputIteratorTransformer
Input [2]: [c_custkey#X, c_nationkey#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -239,7 +239,7 @@ Input [2]: [c_nationkey#X, o_orderkey#X]
(26) InputIteratorTransformer
Input [2]: [c_nationkey#X, o_orderkey#X]
-(27) Scan parquet
+(27) ScanTransformer parquet
Output [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -307,7 +307,7 @@ Input [4]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
(43) InputIteratorTransformer
Input [4]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
-(44) Scan parquet
+(44) ScanTransformer parquet
Output [2]: [s_suppkey#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -375,7 +375,7 @@ Input [3]: [l_extendedprice#X, l_discount#X, s_nationkey#X]
(60) InputIteratorTransformer
Input [3]: [l_extendedprice#X, l_discount#X, s_nationkey#X]
-(61) Scan parquet
+(61) ScanTransformer parquet
Output [3]: [n_nationkey#X, n_name#X, n_regionkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -443,7 +443,7 @@ Input [4]: [l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X]
(77) InputIteratorTransformer
Input [4]: [l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X]
-(78) Scan parquet
+(78) ScanTransformer parquet
Output [2]: [r_regionkey#X, r_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 5df63675caee..d5569132a41f 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
@@ -10,7 +10,7 @@ AdaptiveSparkPlan (20)
+- ^ FlushableHashAggregateExecTransformer (4)
+- ^ ProjectExecTransformer (3)
+- ^ FilterExecTransformer (2)
- +- ^ Scan parquet (1)
+ +- ^ ScanTransformer parquet (1)
+- == Initial Plan ==
HashAggregate (19)
+- Exchange (18)
@@ -20,7 +20,7 @@ AdaptiveSparkPlan (20)
+- Scan parquet (14)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [4]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
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 ac6dda065bb0..ccf77e181d75 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
@@ -46,35 +46,35 @@ AdaptiveSparkPlan (149)
: : : : : +- VeloxResizeBatches (5)
: : : : : +- ^ ProjectExecTransformer (3)
: : : : : +- ^ FilterExecTransformer (2)
- : : : : : +- ^ Scan parquet (1)
+ : : : : : +- ^ ScanTransformer parquet (1)
: : : : +- ^ InputIteratorTransformer (18)
: : : : +- ShuffleQueryStage (16), Statistics(X)
: : : : +- ColumnarExchange (15)
: : : : +- VeloxResizeBatches (14)
: : : : +- ^ ProjectExecTransformer (12)
: : : : +- ^ FilterExecTransformer (11)
- : : : : +- ^ Scan parquet (10)
+ : : : : +- ^ ScanTransformer parquet (10)
: : : +- ^ InputIteratorTransformer (35)
: : : +- ShuffleQueryStage (33), Statistics(X)
: : : +- ColumnarExchange (32)
: : : +- VeloxResizeBatches (31)
: : : +- ^ ProjectExecTransformer (29)
: : : +- ^ FilterExecTransformer (28)
- : : : +- ^ Scan parquet (27)
+ : : : +- ^ ScanTransformer parquet (27)
: : +- ^ InputIteratorTransformer (52)
: : +- ShuffleQueryStage (50), Statistics(X)
: : +- ColumnarExchange (49)
: : +- VeloxResizeBatches (48)
: : +- ^ ProjectExecTransformer (46)
: : +- ^ FilterExecTransformer (45)
- : : +- ^ Scan parquet (44)
+ : : +- ^ ScanTransformer parquet (44)
: +- ^ InputIteratorTransformer (69)
: +- ShuffleQueryStage (67), Statistics(X)
: +- ColumnarExchange (66)
: +- VeloxResizeBatches (65)
: +- ^ ProjectExecTransformer (63)
: +- ^ FilterExecTransformer (62)
- : +- ^ Scan parquet (61)
+ : +- ^ ScanTransformer parquet (61)
+- ^ InputIteratorTransformer (81)
+- ShuffleQueryStage (79), Statistics(X)
+- ReusedExchange (78)
@@ -128,7 +128,7 @@ AdaptiveSparkPlan (149)
+- Scan parquet (138)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [s_suppkey#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -165,7 +165,7 @@ Input [2]: [s_suppkey#X, s_nationkey#X]
(9) InputIteratorTransformer
Input [2]: [s_suppkey#X, s_nationkey#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [5]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -233,7 +233,7 @@ Input [5]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_ship
(26) InputIteratorTransformer
Input [5]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
-(27) Scan parquet
+(27) ScanTransformer parquet
Output [2]: [o_orderkey#X, o_custkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -301,7 +301,7 @@ Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_cust
(43) InputIteratorTransformer
Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X]
-(44) Scan parquet
+(44) ScanTransformer parquet
Output [2]: [c_custkey#X, c_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -369,7 +369,7 @@ Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nati
(60) InputIteratorTransformer
Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X]
-(61) Scan parquet
+(61) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 6f5ed0d82b80..76e38b48de31 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
@@ -59,56 +59,56 @@ AdaptiveSparkPlan (207)
: : : : : : : +- VeloxResizeBatches (5)
: : : : : : : +- ^ ProjectExecTransformer (3)
: : : : : : : +- ^ FilterExecTransformer (2)
- : : : : : : : +- ^ Scan parquet (1)
+ : : : : : : : +- ^ ScanTransformer parquet (1)
: : : : : : +- ^ InputIteratorTransformer (18)
: : : : : : +- ShuffleQueryStage (16), Statistics(X)
: : : : : : +- ColumnarExchange (15)
: : : : : : +- VeloxResizeBatches (14)
: : : : : : +- ^ ProjectExecTransformer (12)
: : : : : : +- ^ FilterExecTransformer (11)
- : : : : : : +- ^ Scan parquet (10)
+ : : : : : : +- ^ ScanTransformer parquet (10)
: : : : : +- ^ InputIteratorTransformer (35)
: : : : : +- ShuffleQueryStage (33), Statistics(X)
: : : : : +- ColumnarExchange (32)
: : : : : +- VeloxResizeBatches (31)
: : : : : +- ^ ProjectExecTransformer (29)
: : : : : +- ^ FilterExecTransformer (28)
- : : : : : +- ^ Scan parquet (27)
+ : : : : : +- ^ ScanTransformer parquet (27)
: : : : +- ^ InputIteratorTransformer (52)
: : : : +- ShuffleQueryStage (50), Statistics(X)
: : : : +- ColumnarExchange (49)
: : : : +- VeloxResizeBatches (48)
: : : : +- ^ ProjectExecTransformer (46)
: : : : +- ^ FilterExecTransformer (45)
- : : : : +- ^ Scan parquet (44)
+ : : : : +- ^ ScanTransformer parquet (44)
: : : +- ^ InputIteratorTransformer (69)
: : : +- ShuffleQueryStage (67), Statistics(X)
: : : +- ColumnarExchange (66)
: : : +- VeloxResizeBatches (65)
: : : +- ^ ProjectExecTransformer (63)
: : : +- ^ FilterExecTransformer (62)
- : : : +- ^ Scan parquet (61)
+ : : : +- ^ ScanTransformer parquet (61)
: : +- ^ InputIteratorTransformer (86)
: : +- ShuffleQueryStage (84), Statistics(X)
: : +- ColumnarExchange (83)
: : +- VeloxResizeBatches (82)
: : +- ^ ProjectExecTransformer (80)
: : +- ^ FilterExecTransformer (79)
- : : +- ^ Scan parquet (78)
+ : : +- ^ ScanTransformer parquet (78)
: +- ^ InputIteratorTransformer (103)
: +- ShuffleQueryStage (101), Statistics(X)
: +- ColumnarExchange (100)
: +- VeloxResizeBatches (99)
: +- ^ ProjectExecTransformer (97)
: +- ^ FilterExecTransformer (96)
- : +- ^ Scan parquet (95)
+ : +- ^ ScanTransformer parquet (95)
+- ^ InputIteratorTransformer (120)
+- ShuffleQueryStage (118), Statistics(X)
+- ColumnarExchange (117)
+- VeloxResizeBatches (116)
+- ^ ProjectExecTransformer (114)
+- ^ FilterExecTransformer (113)
- +- ^ Scan parquet (112)
+ +- ^ ScanTransformer parquet (112)
+- == Initial Plan ==
Sort (206)
+- Exchange (205)
@@ -177,7 +177,7 @@ AdaptiveSparkPlan (207)
+- Scan parquet (195)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [p_partkey#X, p_type#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -214,7 +214,7 @@ Input [1]: [p_partkey#X]
(9) InputIteratorTransformer
Input [1]: [p_partkey#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [5]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -282,7 +282,7 @@ Input [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
(26) InputIteratorTransformer
Input [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
-(27) Scan parquet
+(27) ScanTransformer parquet
Output [2]: [s_suppkey#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -350,7 +350,7 @@ Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X]
(43) InputIteratorTransformer
Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X]
-(44) Scan parquet
+(44) ScanTransformer parquet
Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -418,7 +418,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_order
(60) InputIteratorTransformer
Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X]
-(61) Scan parquet
+(61) ScanTransformer parquet
Output [2]: [c_custkey#X, c_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -486,7 +486,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nat
(77) InputIteratorTransformer
Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X]
-(78) Scan parquet
+(78) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_regionkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -554,7 +554,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_reg
(94) InputIteratorTransformer
Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X]
-(95) Scan parquet
+(95) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -622,7 +622,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_nam
(111) InputIteratorTransformer
Input [5]: [l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X]
-(112) Scan parquet
+(112) ScanTransformer parquet
Output [2]: [r_regionkey#X, r_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 22cd2b9967fb..3c152f565673 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
@@ -46,42 +46,42 @@ AdaptiveSparkPlan (155)
: : : : : +- VeloxResizeBatches (5)
: : : : : +- ^ ProjectExecTransformer (3)
: : : : : +- ^ FilterExecTransformer (2)
- : : : : : +- ^ Scan parquet (1)
+ : : : : : +- ^ ScanTransformer parquet (1)
: : : : +- ^ InputIteratorTransformer (18)
: : : : +- ShuffleQueryStage (16), Statistics(X)
: : : : +- ColumnarExchange (15)
: : : : +- VeloxResizeBatches (14)
: : : : +- ^ ProjectExecTransformer (12)
: : : : +- ^ FilterExecTransformer (11)
- : : : : +- ^ Scan parquet (10)
+ : : : : +- ^ ScanTransformer parquet (10)
: : : +- ^ InputIteratorTransformer (35)
: : : +- ShuffleQueryStage (33), Statistics(X)
: : : +- ColumnarExchange (32)
: : : +- VeloxResizeBatches (31)
: : : +- ^ ProjectExecTransformer (29)
: : : +- ^ FilterExecTransformer (28)
- : : : +- ^ Scan parquet (27)
+ : : : +- ^ ScanTransformer parquet (27)
: : +- ^ InputIteratorTransformer (52)
: : +- ShuffleQueryStage (50), Statistics(X)
: : +- ColumnarExchange (49)
: : +- VeloxResizeBatches (48)
: : +- ^ ProjectExecTransformer (46)
: : +- ^ FilterExecTransformer (45)
- : : +- ^ Scan parquet (44)
+ : : +- ^ ScanTransformer parquet (44)
: +- ^ InputIteratorTransformer (69)
: +- ShuffleQueryStage (67), Statistics(X)
: +- ColumnarExchange (66)
: +- VeloxResizeBatches (65)
: +- ^ ProjectExecTransformer (63)
: +- ^ FilterExecTransformer (62)
- : +- ^ Scan parquet (61)
+ : +- ^ ScanTransformer parquet (61)
+- ^ InputIteratorTransformer (86)
+- ShuffleQueryStage (84), Statistics(X)
+- ColumnarExchange (83)
+- VeloxResizeBatches (82)
+- ^ ProjectExecTransformer (80)
+- ^ FilterExecTransformer (79)
- +- ^ Scan parquet (78)
+ +- ^ ScanTransformer parquet (78)
+- == Initial Plan ==
Sort (154)
+- Exchange (153)
@@ -133,7 +133,7 @@ AdaptiveSparkPlan (155)
+- Scan parquet (144)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [p_partkey#X, p_name#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -170,7 +170,7 @@ Input [1]: [p_partkey#X]
(9) InputIteratorTransformer
Input [1]: [p_partkey#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -238,7 +238,7 @@ Input [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpric
(26) InputIteratorTransformer
Input [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X]
-(27) Scan parquet
+(27) ScanTransformer parquet
Output [2]: [s_suppkey#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -306,7 +306,7 @@ Input [7]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpric
(43) InputIteratorTransformer
Input [7]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X]
-(44) Scan parquet
+(44) ScanTransformer parquet
Output [3]: [ps_partkey#X, ps_suppkey#X, ps_supplycost#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -374,7 +374,7 @@ Input [6]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_natio
(60) InputIteratorTransformer
Input [6]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X]
-(61) Scan parquet
+(61) ScanTransformer parquet
Output [2]: [o_orderkey#X, o_orderdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -442,7 +442,7 @@ Input [6]: [l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_sup
(77) InputIteratorTransformer
Input [6]: [l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X]
-(78) Scan parquet
+(78) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 6dbee1ad7ffb..4d61f904dbe9 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
@@ -16,7 +16,7 @@ AdaptiveSparkPlan (30)
+- ^ FlushableHashAggregateExecTransformer (4)
+- ^ ProjectExecTransformer (3)
+- ^ FilterExecTransformer (2)
- +- ^ Scan parquet (1)
+ +- ^ ScanTransformer parquet (1)
+- == Initial Plan ==
Sort (29)
+- Exchange (28)
@@ -28,7 +28,7 @@ AdaptiveSparkPlan (30)
+- Scan parquet (22)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [7]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_tax#X, l_returnflag#X, l_linestatus#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
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 a052989ba525..1a64123cc8c0 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
@@ -31,28 +31,28 @@ AdaptiveSparkPlan (100)
: : : +- VeloxResizeBatches (5)
: : : +- ^ ProjectExecTransformer (3)
: : : +- ^ FilterExecTransformer (2)
- : : : +- ^ Scan parquet (1)
+ : : : +- ^ ScanTransformer parquet (1)
: : +- ^ InputIteratorTransformer (18)
: : +- ShuffleQueryStage (16), Statistics(X)
: : +- ColumnarExchange (15)
: : +- VeloxResizeBatches (14)
: : +- ^ ProjectExecTransformer (12)
: : +- ^ FilterExecTransformer (11)
- : : +- ^ Scan parquet (10)
+ : : +- ^ ScanTransformer parquet (10)
: +- ^ InputIteratorTransformer (35)
: +- ShuffleQueryStage (33), Statistics(X)
: +- ColumnarExchange (32)
: +- VeloxResizeBatches (31)
: +- ^ ProjectExecTransformer (29)
: +- ^ FilterExecTransformer (28)
- : +- ^ Scan parquet (27)
+ : +- ^ ScanTransformer parquet (27)
+- ^ InputIteratorTransformer (52)
+- ShuffleQueryStage (50), Statistics(X)
+- ColumnarExchange (49)
+- VeloxResizeBatches (48)
+- ^ ProjectExecTransformer (46)
+- ^ FilterExecTransformer (45)
- +- ^ Scan parquet (44)
+ +- ^ ScanTransformer parquet (44)
+- == Initial Plan ==
TakeOrderedAndProject (99)
+- HashAggregate (98)
@@ -88,7 +88,7 @@ AdaptiveSparkPlan (100)
+- Scan parquet (90)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -125,7 +125,7 @@ Input [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct
(9) InputIteratorTransformer
Input [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -194,7 +194,7 @@ Input [8]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct
(26) InputIteratorTransformer
Input [8]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, o_orderkey#X]
-(27) Scan parquet
+(27) ScanTransformer parquet
Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_returnflag#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -263,7 +263,7 @@ Input [9]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct
(43) InputIteratorTransformer
Input [9]: [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]
-(44) Scan parquet
+(44) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 68e87c39a078..e2d5e58eba47 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
@@ -29,21 +29,21 @@ AdaptiveSparkPlan (82)
: : +- VeloxResizeBatches (5)
: : +- ^ ProjectExecTransformer (3)
: : +- ^ FilterExecTransformer (2)
- : : +- ^ Scan parquet (1)
+ : : +- ^ ScanTransformer parquet (1)
: +- ^ InputIteratorTransformer (18)
: +- ShuffleQueryStage (16), Statistics(X)
: +- ColumnarExchange (15)
: +- VeloxResizeBatches (14)
: +- ^ ProjectExecTransformer (12)
: +- ^ FilterExecTransformer (11)
- : +- ^ Scan parquet (10)
+ : +- ^ ScanTransformer parquet (10)
+- ^ InputIteratorTransformer (35)
+- ShuffleQueryStage (33), Statistics(X)
+- ColumnarExchange (32)
+- VeloxResizeBatches (31)
+- ^ ProjectExecTransformer (29)
+- ^ FilterExecTransformer (28)
- +- ^ Scan parquet (27)
+ +- ^ ScanTransformer parquet (27)
+- == Initial Plan ==
Sort (81)
+- Exchange (80)
@@ -72,7 +72,7 @@ AdaptiveSparkPlan (82)
+- Scan parquet (69)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -109,7 +109,7 @@ Input [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X]
(9) InputIteratorTransformer
Input [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [2]: [s_suppkey#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -178,7 +178,7 @@ Input [4]: [ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X]
(26) InputIteratorTransformer
Input [4]: [ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X]
-(27) Scan parquet
+(27) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -428,58 +428,57 @@ Arguments: isFinalPlan=true
===== Subqueries =====
Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery subquery#X, [id=#X]
-AdaptiveSparkPlan (136)
+AdaptiveSparkPlan (135)
+- == Final Plan ==
- VeloxColumnarToRow (114)
- +- ^ ProjectExecTransformer (112)
- +- ^ RegularHashAggregateExecTransformer (111)
- +- ^ RegularHashAggregateExecTransformer (110)
- +- ^ ProjectExecTransformer (109)
- +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (108)
- :- ^ InputIteratorTransformer (103)
- : +- ShuffleQueryStage (101), Statistics(X)
- : +- ColumnarExchange (100)
- : +- VeloxResizeBatches (99)
- : +- ^ ProjectExecTransformer (97)
- : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (96)
- : :- ^ InputIteratorTransformer (91)
- : : +- ShuffleQueryStage (89), Statistics(X)
- : : +- ColumnarExchange (88)
- : : +- VeloxResizeBatches (87)
- : : +- ^ ProjectExecTransformer (85)
- : : +- ^ FilterExecTransformer (84)
- : : +- ^ Scan parquet (83)
- : +- ^ InputIteratorTransformer (95)
- : +- ShuffleQueryStage (93), Statistics(X)
- : +- ReusedExchange (92)
- +- ^ InputIteratorTransformer (107)
- +- ShuffleQueryStage (105), Statistics(X)
- +- ReusedExchange (104)
+ VeloxColumnarToRow (113)
+ +- ^ ProjectExecTransformer (111)
+ +- ^ RegularHashAggregateExecTransformer (110)
+ +- ^ ProjectExecTransformer (109)
+ +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (108)
+ :- ^ InputIteratorTransformer (103)
+ : +- ShuffleQueryStage (101), Statistics(X)
+ : +- ColumnarExchange (100)
+ : +- VeloxResizeBatches (99)
+ : +- ^ ProjectExecTransformer (97)
+ : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (96)
+ : :- ^ InputIteratorTransformer (91)
+ : : +- ShuffleQueryStage (89), Statistics(X)
+ : : +- ColumnarExchange (88)
+ : : +- VeloxResizeBatches (87)
+ : : +- ^ ProjectExecTransformer (85)
+ : : +- ^ FilterExecTransformer (84)
+ : : +- ^ ScanTransformer parquet (83)
+ : +- ^ InputIteratorTransformer (95)
+ : +- ShuffleQueryStage (93), Statistics(X)
+ : +- ReusedExchange (92)
+ +- ^ InputIteratorTransformer (107)
+ +- ShuffleQueryStage (105), Statistics(X)
+ +- ReusedExchange (104)
+- == Initial Plan ==
- HashAggregate (135)
- +- HashAggregate (134)
- +- Project (133)
- +- SortMergeJoin Inner (132)
- :- Sort (126)
- : +- Exchange (125)
- : +- Project (124)
- : +- SortMergeJoin Inner (123)
- : :- Sort (118)
- : : +- Exchange (117)
- : : +- Filter (116)
- : : +- Scan parquet (115)
- : +- Sort (122)
- : +- Exchange (121)
- : +- Filter (120)
- : +- Scan parquet (119)
- +- Sort (131)
- +- Exchange (130)
- +- Project (129)
- +- Filter (128)
- +- Scan parquet (127)
-
-
-(83) Scan parquet
+ HashAggregate (134)
+ +- HashAggregate (133)
+ +- Project (132)
+ +- SortMergeJoin Inner (131)
+ :- Sort (125)
+ : +- Exchange (124)
+ : +- Project (123)
+ : +- SortMergeJoin Inner (122)
+ : :- Sort (117)
+ : : +- Exchange (116)
+ : : +- Filter (115)
+ : : +- Scan parquet (114)
+ : +- Sort (121)
+ : +- Exchange (120)
+ : +- Filter (119)
+ : +- Scan parquet (118)
+ +- Sort (130)
+ +- Exchange (129)
+ +- Project (128)
+ +- Filter (127)
+ +- Scan parquet (126)
+
+
+(83) ScanTransformer parquet
Output [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -587,131 +586,124 @@ Input [4]: [ps_availqty#X, ps_supplycost#X, s_nationkey#X, n_nationkey#X]
(110) RegularHashAggregateExecTransformer
Input [3]: [ps_availqty#X, ps_supplycost#X, _pre_X#X]
Keys: []
-Functions [1]: [partial_sum(_pre_X#X)]
-Aggregate Attributes [2]: [sum#X, isEmpty#X]
-Results [2]: [sum#X, isEmpty#X]
-
-(111) RegularHashAggregateExecTransformer
-Input [2]: [sum#X, isEmpty#X]
-Keys: []
-Functions [1]: [sum((ps_supplycost#X * cast(ps_availqty#X as decimal(10,0))))]
+Functions [1]: [sum(_pre_X#X)]
Aggregate Attributes [1]: [sum((ps_supplycost#X * cast(ps_availqty#X as decimal(10,0))))#X]
Results [1]: [sum((ps_supplycost#X * cast(ps_availqty#X as decimal(10,0))))#X]
-(112) ProjectExecTransformer
+(111) ProjectExecTransformer
Output [1]: [(sum((ps_supplycost#X * cast(ps_availqty#X as decimal(10,0))))#X * 0.0001000000) AS (sum((ps_supplycost * ps_availqty)) * 0.0001000000)#X]
Input [1]: [sum((ps_supplycost#X * cast(ps_availqty#X as decimal(10,0))))#X]
-(113) WholeStageCodegenTransformer (X)
+(112) WholeStageCodegenTransformer (X)
Input [1]: [(sum((ps_supplycost * ps_availqty)) * 0.0001000000)#X]
Arguments: false
-(114) VeloxColumnarToRow
+(113) VeloxColumnarToRow
Input [1]: [(sum((ps_supplycost * ps_availqty)) * 0.0001000000)#X]
-(115) Scan parquet
+(114) Scan parquet
Output [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(ps_suppkey)]
ReadSchema: struct
-(116) Filter
+(115) Filter
Input [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X]
Condition : isnotnull(ps_suppkey#X)
-(117) Exchange
+(116) Exchange
Input [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X]
Arguments: hashpartitioning(ps_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(118) Sort
+(117) Sort
Input [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X]
Arguments: [ps_suppkey#X ASC NULLS FIRST], false, 0
-(119) Scan parquet
+(118) Scan parquet
Output [2]: [s_suppkey#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(s_suppkey), IsNotNull(s_nationkey)]
ReadSchema: struct
-(120) Filter
+(119) Filter
Input [2]: [s_suppkey#X, s_nationkey#X]
Condition : (isnotnull(s_suppkey#X) AND isnotnull(s_nationkey#X))
-(121) Exchange
+(120) Exchange
Input [2]: [s_suppkey#X, s_nationkey#X]
Arguments: hashpartitioning(s_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(122) Sort
+(121) Sort
Input [2]: [s_suppkey#X, s_nationkey#X]
Arguments: [s_suppkey#X ASC NULLS FIRST], false, 0
-(123) SortMergeJoin
+(122) SortMergeJoin
Left keys [1]: [ps_suppkey#X]
Right keys [1]: [s_suppkey#X]
Join type: Inner
Join condition: None
-(124) Project
+(123) Project
Output [3]: [ps_availqty#X, ps_supplycost#X, s_nationkey#X]
Input [5]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X, s_suppkey#X, s_nationkey#X]
-(125) Exchange
+(124) Exchange
Input [3]: [ps_availqty#X, ps_supplycost#X, s_nationkey#X]
Arguments: hashpartitioning(s_nationkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(126) Sort
+(125) Sort
Input [3]: [ps_availqty#X, ps_supplycost#X, s_nationkey#X]
Arguments: [s_nationkey#X ASC NULLS FIRST], false, 0
-(127) Scan parquet
+(126) Scan parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(n_name), EqualTo(n_name,GERMANY), IsNotNull(n_nationkey)]
ReadSchema: struct
-(128) Filter
+(127) Filter
Input [2]: [n_nationkey#X, n_name#X]
Condition : ((isnotnull(n_name#X) AND (n_name#X = GERMANY)) AND isnotnull(n_nationkey#X))
-(129) Project
+(128) Project
Output [1]: [n_nationkey#X]
Input [2]: [n_nationkey#X, n_name#X]
-(130) Exchange
+(129) Exchange
Input [1]: [n_nationkey#X]
Arguments: hashpartitioning(n_nationkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(131) Sort
+(130) Sort
Input [1]: [n_nationkey#X]
Arguments: [n_nationkey#X ASC NULLS FIRST], false, 0
-(132) SortMergeJoin
+(131) SortMergeJoin
Left keys [1]: [s_nationkey#X]
Right keys [1]: [n_nationkey#X]
Join type: Inner
Join condition: None
-(133) Project
+(132) Project
Output [2]: [ps_availqty#X, ps_supplycost#X]
Input [4]: [ps_availqty#X, ps_supplycost#X, s_nationkey#X, n_nationkey#X]
-(134) HashAggregate
+(133) HashAggregate
Input [2]: [ps_availqty#X, ps_supplycost#X]
Keys: []
Functions [1]: [partial_sum((ps_supplycost#X * cast(ps_availqty#X as decimal(10,0))))]
Aggregate Attributes [2]: [sum#X, isEmpty#X]
Results [2]: [sum#X, isEmpty#X]
-(135) HashAggregate
+(134) HashAggregate
Input [2]: [sum#X, isEmpty#X]
Keys: []
Functions [1]: [sum((ps_supplycost#X * cast(ps_availqty#X as decimal(10,0))))]
Aggregate Attributes [1]: [sum((ps_supplycost#X * cast(ps_availqty#X as decimal(10,0))))#X]
Results [1]: [(sum((ps_supplycost#X * cast(ps_availqty#X as decimal(10,0))))#X * 0.0001000000) AS (sum((ps_supplycost * ps_availqty)) * 0.0001000000)#X]
-(136) AdaptiveSparkPlan
+(135) AdaptiveSparkPlan
Output [1]: [(sum((ps_supplycost * ps_availqty)) * 0.0001000000)#X]
Arguments: isFinalPlan=true
\ No newline at end of file
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 9175845083ca..3bfd39bdaf49 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
@@ -22,14 +22,14 @@ AdaptiveSparkPlan (55)
: +- VeloxResizeBatches (5)
: +- ^ ProjectExecTransformer (3)
: +- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
+ : +- ^ ScanTransformer parquet (1)
+- ^ InputIteratorTransformer (18)
+- ShuffleQueryStage (16), Statistics(X)
+- ColumnarExchange (15)
+- VeloxResizeBatches (14)
+- ^ ProjectExecTransformer (12)
+- ^ FilterExecTransformer (11)
- +- ^ Scan parquet (10)
+ +- ^ ScanTransformer parquet (10)
+- == Initial Plan ==
Sort (54)
+- Exchange (53)
@@ -49,7 +49,7 @@ AdaptiveSparkPlan (55)
+- Scan parquet (43)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [o_orderkey#X, o_orderpriority#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -86,7 +86,7 @@ Input [2]: [o_orderkey#X, o_orderpriority#X]
(9) InputIteratorTransformer
Input [2]: [o_orderkey#X, o_orderpriority#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [5]: [l_orderkey#X, l_shipdate#X, l_commitdate#X, l_receiptdate#X, l_shipmode#X]
Batched: true
Location: InMemoryFileIndex [*]
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 b527c2bc3bc3..59abfb682c42 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
@@ -1,58 +1,57 @@
== Physical Plan ==
-AdaptiveSparkPlan (58)
+AdaptiveSparkPlan (57)
+- == Final Plan ==
- VeloxColumnarToRow (40)
- +- ^ SortExecTransformer (38)
- +- ^ InputIteratorTransformer (37)
- +- ShuffleQueryStage (35), Statistics(X)
- +- ColumnarExchange (34)
- +- VeloxResizeBatches (33)
- +- ^ RegularHashAggregateExecTransformer (31)
- +- ^ InputIteratorTransformer (30)
- +- ShuffleQueryStage (28), Statistics(X)
- +- ColumnarExchange (27)
- +- VeloxResizeBatches (26)
- +- ^ ProjectExecTransformer (24)
- +- ^ FlushableHashAggregateExecTransformer (23)
- +- ^ ProjectExecTransformer (22)
- +- ^ RegularHashAggregateExecTransformer (21)
- +- ^ RegularHashAggregateExecTransformer (20)
- +- ^ ProjectExecTransformer (19)
- +- ^ ShuffledHashJoinExecTransformer LeftOuter BuildLeft (18)
- :- ^ InputIteratorTransformer (8)
- : +- ShuffleQueryStage (6), Statistics(X)
- : +- ColumnarExchange (5)
- : +- VeloxResizeBatches (4)
- : +- ^ ProjectExecTransformer (2)
- : +- ^ Scan parquet (1)
- +- ^ InputIteratorTransformer (17)
- +- ShuffleQueryStage (15), Statistics(X)
- +- ColumnarExchange (14)
- +- VeloxResizeBatches (13)
- +- ^ ProjectExecTransformer (11)
- +- ^ FilterExecTransformer (10)
- +- ^ Scan parquet (9)
+ VeloxColumnarToRow (39)
+ +- ^ SortExecTransformer (37)
+ +- ^ InputIteratorTransformer (36)
+ +- ShuffleQueryStage (34), Statistics(X)
+ +- ColumnarExchange (33)
+ +- VeloxResizeBatches (32)
+ +- ^ RegularHashAggregateExecTransformer (30)
+ +- ^ InputIteratorTransformer (29)
+ +- ShuffleQueryStage (27), Statistics(X)
+ +- ColumnarExchange (26)
+ +- VeloxResizeBatches (25)
+ +- ^ ProjectExecTransformer (23)
+ +- ^ FlushableHashAggregateExecTransformer (22)
+ +- ^ ProjectExecTransformer (21)
+ +- ^ RegularHashAggregateExecTransformer (20)
+ +- ^ ProjectExecTransformer (19)
+ +- ^ ShuffledHashJoinExecTransformer LeftOuter BuildLeft (18)
+ :- ^ InputIteratorTransformer (8)
+ : +- ShuffleQueryStage (6), Statistics(X)
+ : +- ColumnarExchange (5)
+ : +- VeloxResizeBatches (4)
+ : +- ^ ProjectExecTransformer (2)
+ : +- ^ ScanTransformer parquet (1)
+ +- ^ InputIteratorTransformer (17)
+ +- ShuffleQueryStage (15), Statistics(X)
+ +- ColumnarExchange (14)
+ +- VeloxResizeBatches (13)
+ +- ^ ProjectExecTransformer (11)
+ +- ^ FilterExecTransformer (10)
+ +- ^ ScanTransformer parquet (9)
+- == Initial Plan ==
- Sort (57)
- +- Exchange (56)
- +- HashAggregate (55)
- +- Exchange (54)
- +- HashAggregate (53)
- +- HashAggregate (52)
- +- HashAggregate (51)
- +- Project (50)
- +- SortMergeJoin LeftOuter (49)
- :- Sort (43)
- : +- Exchange (42)
- : +- Scan parquet (41)
- +- Sort (48)
- +- Exchange (47)
- +- Project (46)
- +- Filter (45)
- +- Scan parquet (44)
-
-
-(1) Scan parquet
+ Sort (56)
+ +- Exchange (55)
+ +- HashAggregate (54)
+ +- Exchange (53)
+ +- HashAggregate (52)
+ +- HashAggregate (51)
+ +- HashAggregate (50)
+ +- Project (49)
+ +- SortMergeJoin LeftOuter (48)
+ :- Sort (42)
+ : +- Exchange (41)
+ : +- Scan parquet (40)
+ +- Sort (47)
+ +- Exchange (46)
+ +- Project (45)
+ +- Filter (44)
+ +- Scan parquet (43)
+
+
+(1) ScanTransformer parquet
Output [1]: [c_custkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -84,7 +83,7 @@ Input [1]: [c_custkey#X]
(8) InputIteratorTransformer
Input [1]: [c_custkey#X]
-(9) Scan parquet
+(9) ScanTransformer parquet
Output [3]: [o_orderkey#X, o_custkey#X, o_comment#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -134,181 +133,174 @@ Input [3]: [c_custkey#X, o_orderkey#X, o_custkey#X]
(20) RegularHashAggregateExecTransformer
Input [2]: [c_custkey#X, o_orderkey#X]
Keys [1]: [c_custkey#X]
-Functions [1]: [partial_count(o_orderkey#X)]
-Aggregate Attributes [1]: [count#X]
-Results [2]: [c_custkey#X, count#X]
-
-(21) RegularHashAggregateExecTransformer
-Input [2]: [c_custkey#X, count#X]
-Keys [1]: [c_custkey#X]
Functions [1]: [count(o_orderkey#X)]
Aggregate Attributes [1]: [count(o_orderkey#X)#X]
Results [2]: [c_custkey#X, count(o_orderkey#X)#X]
-(22) ProjectExecTransformer
+(21) ProjectExecTransformer
Output [1]: [count(o_orderkey#X)#X AS c_count#X]
Input [2]: [c_custkey#X, count(o_orderkey#X)#X]
-(23) FlushableHashAggregateExecTransformer
+(22) FlushableHashAggregateExecTransformer
Input [1]: [c_count#X]
Keys [1]: [c_count#X]
Functions [1]: [partial_count(1)]
Aggregate Attributes [1]: [count#X]
Results [2]: [c_count#X, count#X]
-(24) ProjectExecTransformer
+(23) ProjectExecTransformer
Output [3]: [hash(c_count#X, 42) AS hash_partition_key#X, c_count#X, count#X]
Input [2]: [c_count#X, count#X]
-(25) WholeStageCodegenTransformer (X)
+(24) WholeStageCodegenTransformer (X)
Input [3]: [hash_partition_key#X, c_count#X, count#X]
Arguments: false
-(26) VeloxResizeBatches
+(25) VeloxResizeBatches
Input [3]: [hash_partition_key#X, c_count#X, count#X]
Arguments: X, X
-(27) ColumnarExchange
+(26) ColumnarExchange
Input [3]: [hash_partition_key#X, c_count#X, count#X]
Arguments: hashpartitioning(c_count#X, 1), ENSURE_REQUIREMENTS, [c_count#X, count#X], [plan_id=X], [shuffle_writer_type=hash]
-(28) ShuffleQueryStage
+(27) ShuffleQueryStage
Output [2]: [c_count#X, count#X]
Arguments: X
-(29) InputAdapter
+(28) InputAdapter
Input [2]: [c_count#X, count#X]
-(30) InputIteratorTransformer
+(29) InputIteratorTransformer
Input [2]: [c_count#X, count#X]
-(31) RegularHashAggregateExecTransformer
+(30) RegularHashAggregateExecTransformer
Input [2]: [c_count#X, count#X]
Keys [1]: [c_count#X]
Functions [1]: [count(1)]
Aggregate Attributes [1]: [count(1)#X]
Results [2]: [c_count#X, count(1)#X AS custdist#X]
-(32) WholeStageCodegenTransformer (X)
+(31) WholeStageCodegenTransformer (X)
Input [2]: [c_count#X, custdist#X]
Arguments: false
-(33) VeloxResizeBatches
+(32) VeloxResizeBatches
Input [2]: [c_count#X, custdist#X]
Arguments: X, X
-(34) ColumnarExchange
+(33) ColumnarExchange
Input [2]: [c_count#X, custdist#X]
Arguments: rangepartitioning(custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST, 1), ENSURE_REQUIREMENTS, [plan_id=X], [shuffle_writer_type=hash]
-(35) ShuffleQueryStage
+(34) ShuffleQueryStage
Output [2]: [c_count#X, custdist#X]
Arguments: X
-(36) InputAdapter
+(35) InputAdapter
Input [2]: [c_count#X, custdist#X]
-(37) InputIteratorTransformer
+(36) InputIteratorTransformer
Input [2]: [c_count#X, custdist#X]
-(38) SortExecTransformer
+(37) SortExecTransformer
Input [2]: [c_count#X, custdist#X]
Arguments: [custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST], true, 0
-(39) WholeStageCodegenTransformer (X)
+(38) WholeStageCodegenTransformer (X)
Input [2]: [c_count#X, custdist#X]
Arguments: false
-(40) VeloxColumnarToRow
+(39) VeloxColumnarToRow
Input [2]: [c_count#X, custdist#X]
-(41) Scan parquet
+(40) Scan parquet
Output [1]: [c_custkey#X]
Batched: true
Location: InMemoryFileIndex [*]
ReadSchema: struct
-(42) Exchange
+(41) Exchange
Input [1]: [c_custkey#X]
Arguments: hashpartitioning(c_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(43) Sort
+(42) Sort
Input [1]: [c_custkey#X]
Arguments: [c_custkey#X ASC NULLS FIRST], false, 0
-(44) Scan parquet
+(43) Scan parquet
Output [3]: [o_orderkey#X, o_custkey#X, o_comment#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(o_comment), IsNotNull(o_custkey)]
ReadSchema: struct
-(45) Filter
+(44) Filter
Input [3]: [o_orderkey#X, o_custkey#X, o_comment#X]
Condition : ((isnotnull(o_comment#X) AND NOT o_comment#X LIKE %special%requests%) AND isnotnull(o_custkey#X))
-(46) Project
+(45) Project
Output [2]: [o_orderkey#X, o_custkey#X]
Input [3]: [o_orderkey#X, o_custkey#X, o_comment#X]
-(47) Exchange
+(46) Exchange
Input [2]: [o_orderkey#X, o_custkey#X]
Arguments: hashpartitioning(o_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(48) Sort
+(47) Sort
Input [2]: [o_orderkey#X, o_custkey#X]
Arguments: [o_custkey#X ASC NULLS FIRST], false, 0
-(49) SortMergeJoin
+(48) SortMergeJoin
Left keys [1]: [c_custkey#X]
Right keys [1]: [o_custkey#X]
Join type: LeftOuter
Join condition: None
-(50) Project
+(49) Project
Output [2]: [c_custkey#X, o_orderkey#X]
Input [3]: [c_custkey#X, o_orderkey#X, o_custkey#X]
-(51) HashAggregate
+(50) HashAggregate
Input [2]: [c_custkey#X, o_orderkey#X]
Keys [1]: [c_custkey#X]
Functions [1]: [partial_count(o_orderkey#X)]
Aggregate Attributes [1]: [count#X]
Results [2]: [c_custkey#X, count#X]
-(52) HashAggregate
+(51) HashAggregate
Input [2]: [c_custkey#X, count#X]
Keys [1]: [c_custkey#X]
Functions [1]: [count(o_orderkey#X)]
Aggregate Attributes [1]: [count(o_orderkey#X)#X]
Results [1]: [count(o_orderkey#X)#X AS c_count#X]
-(53) HashAggregate
+(52) HashAggregate
Input [1]: [c_count#X]
Keys [1]: [c_count#X]
Functions [1]: [partial_count(1)]
Aggregate Attributes [1]: [count#X]
Results [2]: [c_count#X, count#X]
-(54) Exchange
+(53) Exchange
Input [2]: [c_count#X, count#X]
Arguments: hashpartitioning(c_count#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(55) HashAggregate
+(54) HashAggregate
Input [2]: [c_count#X, count#X]
Keys [1]: [c_count#X]
Functions [1]: [count(1)]
Aggregate Attributes [1]: [count(1)#X]
Results [2]: [c_count#X, count(1)#X AS custdist#X]
-(56) Exchange
+(55) Exchange
Input [2]: [c_count#X, custdist#X]
Arguments: rangepartitioning(custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(57) Sort
+(56) Sort
Input [2]: [c_count#X, custdist#X]
Arguments: [custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST], true, 0
-(58) AdaptiveSparkPlan
+(57) AdaptiveSparkPlan
Output [2]: [c_count#X, custdist#X]
Arguments: isFinalPlan=true
\ No newline at end of file
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 cc6d364dea22..531cc5aaab55 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
@@ -1,43 +1,42 @@
== Physical Plan ==
-AdaptiveSparkPlan (39)
+AdaptiveSparkPlan (38)
+- == Final Plan ==
- VeloxColumnarToRow (25)
- +- ^ ProjectExecTransformer (23)
- +- ^ RegularHashAggregateExecTransformer (22)
- +- ^ RegularHashAggregateExecTransformer (21)
- +- ^ ProjectExecTransformer (20)
- +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19)
- :- ^ InputIteratorTransformer (9)
- : +- ShuffleQueryStage (7), Statistics(X)
- : +- ColumnarExchange (6)
- : +- VeloxResizeBatches (5)
- : +- ^ ProjectExecTransformer (3)
- : +- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
- +- ^ InputIteratorTransformer (18)
- +- ShuffleQueryStage (16), Statistics(X)
- +- ColumnarExchange (15)
- +- VeloxResizeBatches (14)
- +- ^ ProjectExecTransformer (12)
- +- ^ FilterExecTransformer (11)
- +- ^ Scan parquet (10)
+ VeloxColumnarToRow (24)
+ +- ^ ProjectExecTransformer (22)
+ +- ^ RegularHashAggregateExecTransformer (21)
+ +- ^ ProjectExecTransformer (20)
+ +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19)
+ :- ^ InputIteratorTransformer (9)
+ : +- ShuffleQueryStage (7), Statistics(X)
+ : +- ColumnarExchange (6)
+ : +- VeloxResizeBatches (5)
+ : +- ^ ProjectExecTransformer (3)
+ : +- ^ FilterExecTransformer (2)
+ : +- ^ ScanTransformer parquet (1)
+ +- ^ InputIteratorTransformer (18)
+ +- ShuffleQueryStage (16), Statistics(X)
+ +- ColumnarExchange (15)
+ +- VeloxResizeBatches (14)
+ +- ^ ProjectExecTransformer (12)
+ +- ^ FilterExecTransformer (11)
+ +- ^ ScanTransformer parquet (10)
+- == Initial Plan ==
- HashAggregate (38)
- +- HashAggregate (37)
- +- Project (36)
- +- SortMergeJoin Inner (35)
- :- Sort (30)
- : +- Exchange (29)
- : +- Project (28)
- : +- Filter (27)
- : +- Scan parquet (26)
- +- Sort (34)
- +- Exchange (33)
- +- Filter (32)
- +- Scan parquet (31)
-
-
-(1) Scan parquet
+ HashAggregate (37)
+ +- HashAggregate (36)
+ +- Project (35)
+ +- SortMergeJoin Inner (34)
+ :- Sort (29)
+ : +- Exchange (28)
+ : +- Project (27)
+ : +- Filter (26)
+ : +- Scan parquet (25)
+ +- Sort (33)
+ +- Exchange (32)
+ +- Filter (31)
+ +- Scan parquet (30)
+
+
+(1) ScanTransformer parquet
Output [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -74,7 +73,7 @@ Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X]
(9) InputIteratorTransformer
Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [2]: [p_partkey#X, p_type#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -124,94 +123,87 @@ Input [5]: [l_partkey#X, l_extendedprice#X, l_discount#X, p_partkey#X, p_type#X]
(21) RegularHashAggregateExecTransformer
Input [5]: [l_extendedprice#X, l_discount#X, p_type#X, _pre_X#X, _pre_X#X]
Keys: []
-Functions [2]: [partial_sum(_pre_X#X), partial_sum(_pre_X#X)]
-Aggregate Attributes [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X]
-Results [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X]
-
-(22) RegularHashAggregateExecTransformer
-Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X]
-Keys: []
-Functions [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN (l_extendedprice#X * (1 - l_discount#X)) ELSE 0.0000 END), sum((l_extendedprice#X * (1 - l_discount#X)))]
+Functions [2]: [sum(_pre_X#X), sum(_pre_X#X)]
Aggregate Attributes [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN (l_extendedprice#X * (1 - l_discount#X)) ELSE 0.0000 END)#X, sum((l_extendedprice#X * (1 - l_discount#X)))#X]
Results [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN (l_extendedprice#X * (1 - l_discount#X)) ELSE 0.0000 END)#X, sum((l_extendedprice#X * (1 - l_discount#X)))#X]
-(23) ProjectExecTransformer
+(22) ProjectExecTransformer
Output [1]: [((100.00 * sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN (l_extendedprice#X * (1 - l_discount#X)) ELSE 0.0000 END)#X) / sum((l_extendedprice#X * (1 - l_discount#X)))#X) AS promo_revenue#X]
Input [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN (l_extendedprice#X * (1 - l_discount#X)) ELSE 0.0000 END)#X, sum((l_extendedprice#X * (1 - l_discount#X)))#X]
-(24) WholeStageCodegenTransformer (X)
+(23) WholeStageCodegenTransformer (X)
Input [1]: [promo_revenue#X]
Arguments: false
-(25) VeloxColumnarToRow
+(24) VeloxColumnarToRow
Input [1]: [promo_revenue#X]
-(26) Scan parquet
+(25) Scan parquet
Output [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1995-09-01), LessThan(l_shipdate,1995-10-01), IsNotNull(l_partkey)]
ReadSchema: struct
-(27) Filter
+(26) Filter
Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Condition : (((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1995-09-01)) AND (l_shipdate#X < 1995-10-01)) AND isnotnull(l_partkey#X))
-(28) Project
+(27) Project
Output [3]: [l_partkey#X, l_extendedprice#X, l_discount#X]
Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
-(29) Exchange
+(28) Exchange
Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X]
Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(30) Sort
+(29) Sort
Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X]
Arguments: [l_partkey#X ASC NULLS FIRST], false, 0
-(31) Scan parquet
+(30) Scan parquet
Output [2]: [p_partkey#X, p_type#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(p_partkey)]
ReadSchema: struct
-(32) Filter
+(31) Filter
Input [2]: [p_partkey#X, p_type#X]
Condition : isnotnull(p_partkey#X)
-(33) Exchange
+(32) Exchange
Input [2]: [p_partkey#X, p_type#X]
Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(34) Sort
+(33) Sort
Input [2]: [p_partkey#X, p_type#X]
Arguments: [p_partkey#X ASC NULLS FIRST], false, 0
-(35) SortMergeJoin
+(34) SortMergeJoin
Left keys [1]: [l_partkey#X]
Right keys [1]: [p_partkey#X]
Join type: Inner
Join condition: None
-(36) Project
+(35) Project
Output [3]: [l_extendedprice#X, l_discount#X, p_type#X]
Input [5]: [l_partkey#X, l_extendedprice#X, l_discount#X, p_partkey#X, p_type#X]
-(37) HashAggregate
+(36) HashAggregate
Input [3]: [l_extendedprice#X, l_discount#X, p_type#X]
Keys: []
Functions [2]: [partial_sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN (l_extendedprice#X * (1 - l_discount#X)) ELSE 0.0000 END), partial_sum((l_extendedprice#X * (1 - l_discount#X)))]
Aggregate Attributes [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X]
Results [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X]
-(38) HashAggregate
+(37) HashAggregate
Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X]
Keys: []
Functions [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN (l_extendedprice#X * (1 - l_discount#X)) ELSE 0.0000 END), sum((l_extendedprice#X * (1 - l_discount#X)))]
Aggregate Attributes [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN (l_extendedprice#X * (1 - l_discount#X)) ELSE 0.0000 END)#X, sum((l_extendedprice#X * (1 - l_discount#X)))#X]
Results [1]: [((100.00 * sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN (l_extendedprice#X * (1 - l_discount#X)) ELSE 0.0000 END)#X) / sum((l_extendedprice#X * (1 - l_discount#X)))#X) AS promo_revenue#X]
-(39) AdaptiveSparkPlan
+(38) AdaptiveSparkPlan
Output [1]: [promo_revenue#X]
Arguments: isFinalPlan=true
\ No newline at end of file
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 8f1a1b4415db..f370146f9206 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
@@ -14,7 +14,7 @@ AdaptiveSparkPlan (47)
: +- VeloxResizeBatches (5)
: +- ^ ProjectExecTransformer (3)
: +- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
+ : +- ^ ScanTransformer parquet (1)
+- ^ FilterExecTransformer (22)
+- ^ RegularHashAggregateExecTransformer (21)
+- ^ InputIteratorTransformer (20)
@@ -25,7 +25,7 @@ AdaptiveSparkPlan (47)
+- ^ FlushableHashAggregateExecTransformer (13)
+- ^ ProjectExecTransformer (12)
+- ^ FilterExecTransformer (11)
- +- ^ Scan parquet (10)
+ +- ^ ScanTransformer parquet (10)
+- == Initial Plan ==
Sort (46)
+- Exchange (45)
@@ -45,7 +45,7 @@ AdaptiveSparkPlan (47)
+- Scan parquet (35)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -82,7 +82,7 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X]
(9) InputIteratorTransformer
Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -259,34 +259,33 @@ Arguments: isFinalPlan=true
===== Subqueries =====
Subquery:1 Hosting operator id = 22 Hosting Expression = Subquery subquery#X, [id=#X]
-AdaptiveSparkPlan (73)
+AdaptiveSparkPlan (72)
+- == Final Plan ==
- VeloxColumnarToRow (64)
- +- ^ RegularHashAggregateExecTransformer (62)
- +- ^ RegularHashAggregateExecTransformer (61)
- +- ^ ProjectExecTransformer (60)
- +- ^ RegularHashAggregateExecTransformer (59)
- +- ^ InputIteratorTransformer (58)
- +- ShuffleQueryStage (56), Statistics(X)
- +- ColumnarExchange (55)
- +- VeloxResizeBatches (54)
- +- ^ ProjectExecTransformer (52)
- +- ^ FlushableHashAggregateExecTransformer (51)
- +- ^ ProjectExecTransformer (50)
- +- ^ FilterExecTransformer (49)
- +- ^ Scan parquet (48)
+ VeloxColumnarToRow (63)
+ +- ^ RegularHashAggregateExecTransformer (61)
+ +- ^ ProjectExecTransformer (60)
+ +- ^ RegularHashAggregateExecTransformer (59)
+ +- ^ InputIteratorTransformer (58)
+ +- ShuffleQueryStage (56), Statistics(X)
+ +- ColumnarExchange (55)
+ +- VeloxResizeBatches (54)
+ +- ^ ProjectExecTransformer (52)
+ +- ^ FlushableHashAggregateExecTransformer (51)
+ +- ^ ProjectExecTransformer (50)
+ +- ^ FilterExecTransformer (49)
+ +- ^ ScanTransformer parquet (48)
+- == Initial Plan ==
- HashAggregate (72)
- +- HashAggregate (71)
- +- HashAggregate (70)
- +- Exchange (69)
- +- HashAggregate (68)
- +- Project (67)
- +- Filter (66)
- +- Scan parquet (65)
+ HashAggregate (71)
+ +- HashAggregate (70)
+ +- HashAggregate (69)
+ +- Exchange (68)
+ +- HashAggregate (67)
+ +- Project (66)
+ +- Filter (65)
+ +- Scan parquet (64)
-(48) Scan parquet
+(48) ScanTransformer parquet
Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -348,71 +347,64 @@ Input [2]: [l_suppkey#X, sum((l_extendedprice#X * (1 - l_discount#X)))#X]
(61) RegularHashAggregateExecTransformer
Input [1]: [total_revenue#X]
Keys: []
-Functions [1]: [partial_max(total_revenue#X)]
-Aggregate Attributes [1]: [max#X]
-Results [1]: [max#X]
-
-(62) RegularHashAggregateExecTransformer
-Input [1]: [max#X]
-Keys: []
Functions [1]: [max(total_revenue#X)]
Aggregate Attributes [1]: [max(total_revenue#X)#X]
Results [1]: [max(total_revenue#X)#X AS max(total_revenue)#X]
-(63) WholeStageCodegenTransformer (X)
+(62) WholeStageCodegenTransformer (X)
Input [1]: [max(total_revenue)#X]
Arguments: false
-(64) VeloxColumnarToRow
+(63) VeloxColumnarToRow
Input [1]: [max(total_revenue)#X]
-(65) Scan parquet
+(64) Scan parquet
Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1996-01-01), LessThan(l_shipdate,1996-04-01)]
ReadSchema: struct
-(66) Filter
+(65) Filter
Input [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Condition : ((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1996-01-01)) AND (l_shipdate#X < 1996-04-01))
-(67) Project
+(66) Project
Output [3]: [l_suppkey#X, l_extendedprice#X, l_discount#X]
Input [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
-(68) HashAggregate
+(67) HashAggregate
Input [3]: [l_suppkey#X, l_extendedprice#X, l_discount#X]
Keys [1]: [l_suppkey#X]
Functions [1]: [partial_sum((l_extendedprice#X * (1 - l_discount#X)))]
Aggregate Attributes [2]: [sum#X, isEmpty#X]
Results [3]: [l_suppkey#X, sum#X, isEmpty#X]
-(69) Exchange
+(68) Exchange
Input [3]: [l_suppkey#X, sum#X, isEmpty#X]
Arguments: hashpartitioning(l_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(70) HashAggregate
+(69) HashAggregate
Input [3]: [l_suppkey#X, sum#X, isEmpty#X]
Keys [1]: [l_suppkey#X]
Functions [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))]
Aggregate Attributes [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))#X]
Results [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))#X AS total_revenue#X]
-(71) HashAggregate
+(70) HashAggregate
Input [1]: [total_revenue#X]
Keys: []
Functions [1]: [partial_max(total_revenue#X)]
Aggregate Attributes [1]: [max#X]
Results [1]: [max#X]
-(72) HashAggregate
+(71) HashAggregate
Input [1]: [max#X]
Keys: []
Functions [1]: [max(total_revenue#X)]
Aggregate Attributes [1]: [max(total_revenue#X)#X]
Results [1]: [max(total_revenue#X)#X AS max(total_revenue)#X]
-(73) AdaptiveSparkPlan
+(72) AdaptiveSparkPlan
Output [1]: [max(total_revenue)#X]
Arguments: isFinalPlan=true
\ No newline at end of file
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 ee7826c8aa7e..3441216f140f 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
@@ -29,14 +29,14 @@ AdaptiveSparkPlan (71)
: +- VeloxResizeBatches (5)
: +- ^ ProjectExecTransformer (3)
: +- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
+ : +- ^ ScanTransformer parquet (1)
+- ^ InputIteratorTransformer (18)
+- ShuffleQueryStage (16), Statistics(X)
+- ColumnarExchange (15)
+- VeloxResizeBatches (14)
+- ^ ProjectExecTransformer (12)
+- ^ FilterExecTransformer (11)
- +- ^ Scan parquet (10)
+ +- ^ ScanTransformer parquet (10)
+- == Initial Plan ==
Sort (70)
+- Exchange (69)
@@ -63,7 +63,7 @@ AdaptiveSparkPlan (71)
+- Scan parquet (57)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [ps_partkey#X, ps_suppkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -100,7 +100,7 @@ Input [2]: [ps_partkey#X, ps_suppkey#X]
(9) InputIteratorTransformer
Input [2]: [ps_partkey#X, ps_suppkey#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [4]: [p_partkey#X, p_brand#X, p_type#X, p_size#X]
Batched: true
Location: InMemoryFileIndex [*]
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 eaad0429ab40..c1b43b7fc421 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
@@ -1,65 +1,64 @@
== Physical Plan ==
-AdaptiveSparkPlan (63)
+AdaptiveSparkPlan (62)
+- == Final Plan ==
- VeloxColumnarToRow (40)
- +- ^ ProjectExecTransformer (38)
- +- ^ RegularHashAggregateExecTransformer (37)
- +- ^ RegularHashAggregateExecTransformer (36)
- +- ^ ProjectExecTransformer (35)
- +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (34)
- :- ^ ProjectExecTransformer (20)
- : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19)
- : :- ^ InputIteratorTransformer (9)
- : : +- ShuffleQueryStage (7), Statistics(X)
- : : +- ColumnarExchange (6)
- : : +- VeloxResizeBatches (5)
- : : +- ^ ProjectExecTransformer (3)
- : : +- ^ FilterExecTransformer (2)
- : : +- ^ Scan parquet (1)
- : +- ^ InputIteratorTransformer (18)
- : +- ShuffleQueryStage (16), Statistics(X)
- : +- ColumnarExchange (15)
- : +- VeloxResizeBatches (14)
- : +- ^ ProjectExecTransformer (12)
- : +- ^ FilterExecTransformer (11)
- : +- ^ Scan parquet (10)
- +- ^ FilterExecTransformer (33)
- +- ^ ProjectExecTransformer (32)
- +- ^ RegularHashAggregateExecTransformer (31)
- +- ^ InputIteratorTransformer (30)
- +- ShuffleQueryStage (28), Statistics(X)
- +- ColumnarExchange (27)
- +- VeloxResizeBatches (26)
- +- ^ ProjectExecTransformer (24)
- +- ^ FlushableHashAggregateExecTransformer (23)
- +- ^ FilterExecTransformer (22)
- +- ^ Scan parquet (21)
+ VeloxColumnarToRow (39)
+ +- ^ ProjectExecTransformer (37)
+ +- ^ RegularHashAggregateExecTransformer (36)
+ +- ^ ProjectExecTransformer (35)
+ +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (34)
+ :- ^ ProjectExecTransformer (20)
+ : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19)
+ : :- ^ InputIteratorTransformer (9)
+ : : +- ShuffleQueryStage (7), Statistics(X)
+ : : +- ColumnarExchange (6)
+ : : +- VeloxResizeBatches (5)
+ : : +- ^ ProjectExecTransformer (3)
+ : : +- ^ FilterExecTransformer (2)
+ : : +- ^ ScanTransformer parquet (1)
+ : +- ^ InputIteratorTransformer (18)
+ : +- ShuffleQueryStage (16), Statistics(X)
+ : +- ColumnarExchange (15)
+ : +- VeloxResizeBatches (14)
+ : +- ^ ProjectExecTransformer (12)
+ : +- ^ FilterExecTransformer (11)
+ : +- ^ ScanTransformer parquet (10)
+ +- ^ FilterExecTransformer (33)
+ +- ^ ProjectExecTransformer (32)
+ +- ^ RegularHashAggregateExecTransformer (31)
+ +- ^ InputIteratorTransformer (30)
+ +- ShuffleQueryStage (28), Statistics(X)
+ +- ColumnarExchange (27)
+ +- VeloxResizeBatches (26)
+ +- ^ ProjectExecTransformer (24)
+ +- ^ FlushableHashAggregateExecTransformer (23)
+ +- ^ FilterExecTransformer (22)
+ +- ^ ScanTransformer parquet (21)
+- == Initial Plan ==
- HashAggregate (62)
- +- HashAggregate (61)
- +- Project (60)
- +- SortMergeJoin Inner (59)
- :- Project (51)
- : +- SortMergeJoin Inner (50)
- : :- Sort (44)
- : : +- Exchange (43)
- : : +- Filter (42)
- : : +- Scan parquet (41)
- : +- Sort (49)
- : +- Exchange (48)
- : +- Project (47)
- : +- Filter (46)
- : +- Scan parquet (45)
- +- Sort (58)
- +- Filter (57)
- +- HashAggregate (56)
- +- Exchange (55)
- +- HashAggregate (54)
- +- Filter (53)
- +- Scan parquet (52)
-
-
-(1) Scan parquet
+ HashAggregate (61)
+ +- HashAggregate (60)
+ +- Project (59)
+ +- SortMergeJoin Inner (58)
+ :- Project (50)
+ : +- SortMergeJoin Inner (49)
+ : :- Sort (43)
+ : : +- Exchange (42)
+ : : +- Filter (41)
+ : : +- Scan parquet (40)
+ : +- Sort (48)
+ : +- Exchange (47)
+ : +- Project (46)
+ : +- Filter (45)
+ : +- Scan parquet (44)
+ +- Sort (57)
+ +- Filter (56)
+ +- HashAggregate (55)
+ +- Exchange (54)
+ +- HashAggregate (53)
+ +- Filter (52)
+ +- Scan parquet (51)
+
+
+(1) ScanTransformer parquet
Output [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -96,7 +95,7 @@ Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X]
(9) InputIteratorTransformer
Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [3]: [p_partkey#X, p_brand#X, p_container#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -143,7 +142,7 @@ Join condition: None
Output [3]: [l_quantity#X, l_extendedprice#X, p_partkey#X]
Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, p_partkey#X]
-(21) Scan parquet
+(21) ScanTransformer parquet
Output [2]: [l_partkey#X, l_quantity#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -215,141 +214,134 @@ Input [5]: [l_quantity#X, l_extendedprice#X, p_partkey#X, (0.2 * avg(l_quantity)
(36) RegularHashAggregateExecTransformer
Input [1]: [l_extendedprice#X]
Keys: []
-Functions [1]: [partial_sum(l_extendedprice#X)]
-Aggregate Attributes [2]: [sum#X, isEmpty#X]
-Results [2]: [sum#X, isEmpty#X]
-
-(37) RegularHashAggregateExecTransformer
-Input [2]: [sum#X, isEmpty#X]
-Keys: []
Functions [1]: [sum(l_extendedprice#X)]
Aggregate Attributes [1]: [sum(l_extendedprice#X)#X]
Results [1]: [sum(l_extendedprice#X)#X]
-(38) ProjectExecTransformer
+(37) ProjectExecTransformer
Output [1]: [(sum(l_extendedprice#X)#X / 7.0) AS avg_yearly#X]
Input [1]: [sum(l_extendedprice#X)#X]
-(39) WholeStageCodegenTransformer (X)
+(38) WholeStageCodegenTransformer (X)
Input [1]: [avg_yearly#X]
Arguments: false
-(40) VeloxColumnarToRow
+(39) VeloxColumnarToRow
Input [1]: [avg_yearly#X]
-(41) Scan parquet
+(40) Scan parquet
Output [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(l_partkey), IsNotNull(l_quantity)]
ReadSchema: struct
-(42) Filter
+(41) Filter
Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X]
Condition : (isnotnull(l_partkey#X) AND isnotnull(l_quantity#X))
-(43) Exchange
+(42) Exchange
Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X]
Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(44) Sort
+(43) Sort
Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X]
Arguments: [l_partkey#X ASC NULLS FIRST], false, 0
-(45) Scan parquet
+(44) Scan parquet
Output [3]: [p_partkey#X, p_brand#X, p_container#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(p_brand), IsNotNull(p_container), EqualTo(p_brand,Brand#X), EqualTo(p_container,MED BOX), IsNotNull(p_partkey)]
ReadSchema: struct
-(46) Filter
+(45) Filter
Input [3]: [p_partkey#X, p_brand#X, p_container#X]
Condition : ((((isnotnull(p_brand#X) AND isnotnull(p_container#X)) AND (p_brand#X = Brand#X)) AND (p_container#X = MED BOX)) AND isnotnull(p_partkey#X))
-(47) Project
+(46) Project
Output [1]: [p_partkey#X]
Input [3]: [p_partkey#X, p_brand#X, p_container#X]
-(48) Exchange
+(47) Exchange
Input [1]: [p_partkey#X]
Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(49) Sort
+(48) Sort
Input [1]: [p_partkey#X]
Arguments: [p_partkey#X ASC NULLS FIRST], false, 0
-(50) SortMergeJoin
+(49) SortMergeJoin
Left keys [1]: [l_partkey#X]
Right keys [1]: [p_partkey#X]
Join type: Inner
Join condition: None
-(51) Project
+(50) Project
Output [3]: [l_quantity#X, l_extendedprice#X, p_partkey#X]
Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, p_partkey#X]
-(52) Scan parquet
+(51) Scan parquet
Output [2]: [l_partkey#X, l_quantity#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(l_partkey)]
ReadSchema: struct
-(53) Filter
+(52) Filter
Input [2]: [l_partkey#X, l_quantity#X]
Condition : isnotnull(l_partkey#X)
-(54) HashAggregate
+(53) HashAggregate
Input [2]: [l_partkey#X, l_quantity#X]
Keys [1]: [l_partkey#X]
Functions [1]: [partial_avg(l_quantity#X)]
Aggregate Attributes [2]: [sum#X, count#X]
Results [3]: [l_partkey#X, sum#X, count#X]
-(55) Exchange
+(54) Exchange
Input [3]: [l_partkey#X, sum#X, count#X]
Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(56) HashAggregate
+(55) HashAggregate
Input [3]: [l_partkey#X, sum#X, count#X]
Keys [1]: [l_partkey#X]
Functions [1]: [avg(l_quantity#X)]
Aggregate Attributes [1]: [avg(l_quantity#X)#X]
Results [2]: [(0.2 * avg(l_quantity#X)#X) AS (0.2 * avg(l_quantity))#X, l_partkey#X]
-(57) Filter
+(56) Filter
Input [2]: [(0.2 * avg(l_quantity))#X, l_partkey#X]
Condition : isnotnull((0.2 * avg(l_quantity))#X)
-(58) Sort
+(57) Sort
Input [2]: [(0.2 * avg(l_quantity))#X, l_partkey#X]
Arguments: [l_partkey#X ASC NULLS FIRST], false, 0
-(59) SortMergeJoin
+(58) SortMergeJoin
Left keys [1]: [p_partkey#X]
Right keys [1]: [l_partkey#X]
Join type: Inner
Join condition: (cast(l_quantity#X as decimal(18,7)) < (0.2 * avg(l_quantity))#X)
-(60) Project
+(59) Project
Output [1]: [l_extendedprice#X]
Input [5]: [l_quantity#X, l_extendedprice#X, p_partkey#X, (0.2 * avg(l_quantity))#X, l_partkey#X]
-(61) HashAggregate
+(60) HashAggregate
Input [1]: [l_extendedprice#X]
Keys: []
Functions [1]: [partial_sum(l_extendedprice#X)]
Aggregate Attributes [2]: [sum#X, isEmpty#X]
Results [2]: [sum#X, isEmpty#X]
-(62) HashAggregate
+(61) HashAggregate
Input [2]: [sum#X, isEmpty#X]
Keys: []
Functions [1]: [sum(l_extendedprice#X)]
Aggregate Attributes [1]: [sum(l_extendedprice#X)#X]
Results [1]: [(sum(l_extendedprice#X)#X / 7.0) AS avg_yearly#X]
-(63) AdaptiveSparkPlan
+(62) AdaptiveSparkPlan
Output [1]: [avg_yearly#X]
Arguments: isFinalPlan=true
\ No newline at end of file
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 85b11fa8ca18..c1a6b181bb34 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
@@ -1,105 +1,104 @@
== Physical Plan ==
-AdaptiveSparkPlan (110)
+AdaptiveSparkPlan (109)
+- == Final Plan ==
- VeloxColumnarToRow (70)
- +- TakeOrderedAndProjectExecTransformer (69)
- +- ^ RegularHashAggregateExecTransformer (67)
- +- ^ RegularHashAggregateExecTransformer (66)
- +- ^ ProjectExecTransformer (65)
- +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (64)
- :- ^ InputIteratorTransformer (46)
- : +- ShuffleQueryStage (44), Statistics(X)
- : +- ColumnarExchange (43)
- : +- VeloxResizeBatches (42)
- : +- ^ ProjectExecTransformer (40)
- : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (39)
- : :- ^ InputIteratorTransformer (9)
- : : +- ShuffleQueryStage (7), Statistics(X)
- : : +- ColumnarExchange (6)
- : : +- VeloxResizeBatches (5)
- : : +- ^ ProjectExecTransformer (3)
- : : +- ^ FilterExecTransformer (2)
- : : +- ^ Scan parquet (1)
- : +- ^ InputIteratorTransformer (38)
- : +- ShuffleQueryStage (36), Statistics(X)
- : +- ColumnarExchange (35)
- : +- VeloxResizeBatches (34)
- : +- ^ ProjectExecTransformer (32)
- : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (31)
- : :- ^ InputIteratorTransformer (18)
- : : +- ShuffleQueryStage (16), Statistics(X)
- : : +- ColumnarExchange (15)
- : : +- VeloxResizeBatches (14)
- : : +- ^ ProjectExecTransformer (12)
- : : +- ^ FilterExecTransformer (11)
- : : +- ^ Scan parquet (10)
- : +- ^ ProjectExecTransformer (30)
- : +- ^ FilterExecTransformer (29)
- : +- ^ RegularHashAggregateExecTransformer (28)
- : +- ^ InputIteratorTransformer (27)
- : +- ShuffleQueryStage (25), Statistics(X)
- : +- ColumnarExchange (24)
- : +- VeloxResizeBatches (23)
- : +- ^ ProjectExecTransformer (21)
- : +- ^ FlushableHashAggregateExecTransformer (20)
- : +- ^ Scan parquet (19)
- +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (63)
- :- ^ InputIteratorTransformer (55)
- : +- ShuffleQueryStage (53), Statistics(X)
- : +- ColumnarExchange (52)
- : +- VeloxResizeBatches (51)
- : +- ^ ProjectExecTransformer (49)
- : +- ^ FilterExecTransformer (48)
- : +- ^ Scan parquet (47)
- +- ^ ProjectExecTransformer (62)
- +- ^ FilterExecTransformer (61)
- +- ^ RegularHashAggregateExecTransformer (60)
- +- ^ InputIteratorTransformer (59)
- +- ShuffleQueryStage (57), Statistics(X)
- +- ReusedExchange (56)
+ VeloxColumnarToRow (69)
+ +- TakeOrderedAndProjectExecTransformer (68)
+ +- ^ RegularHashAggregateExecTransformer (66)
+ +- ^ ProjectExecTransformer (65)
+ +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (64)
+ :- ^ InputIteratorTransformer (46)
+ : +- ShuffleQueryStage (44), Statistics(X)
+ : +- ColumnarExchange (43)
+ : +- VeloxResizeBatches (42)
+ : +- ^ ProjectExecTransformer (40)
+ : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (39)
+ : :- ^ InputIteratorTransformer (9)
+ : : +- ShuffleQueryStage (7), Statistics(X)
+ : : +- ColumnarExchange (6)
+ : : +- VeloxResizeBatches (5)
+ : : +- ^ ProjectExecTransformer (3)
+ : : +- ^ FilterExecTransformer (2)
+ : : +- ^ ScanTransformer parquet (1)
+ : +- ^ InputIteratorTransformer (38)
+ : +- ShuffleQueryStage (36), Statistics(X)
+ : +- ColumnarExchange (35)
+ : +- VeloxResizeBatches (34)
+ : +- ^ ProjectExecTransformer (32)
+ : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (31)
+ : :- ^ InputIteratorTransformer (18)
+ : : +- ShuffleQueryStage (16), Statistics(X)
+ : : +- ColumnarExchange (15)
+ : : +- VeloxResizeBatches (14)
+ : : +- ^ ProjectExecTransformer (12)
+ : : +- ^ FilterExecTransformer (11)
+ : : +- ^ ScanTransformer parquet (10)
+ : +- ^ ProjectExecTransformer (30)
+ : +- ^ FilterExecTransformer (29)
+ : +- ^ RegularHashAggregateExecTransformer (28)
+ : +- ^ InputIteratorTransformer (27)
+ : +- ShuffleQueryStage (25), Statistics(X)
+ : +- ColumnarExchange (24)
+ : +- VeloxResizeBatches (23)
+ : +- ^ ProjectExecTransformer (21)
+ : +- ^ FlushableHashAggregateExecTransformer (20)
+ : +- ^ ScanTransformer parquet (19)
+ +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (63)
+ :- ^ InputIteratorTransformer (55)
+ : +- ShuffleQueryStage (53), Statistics(X)
+ : +- ColumnarExchange (52)
+ : +- VeloxResizeBatches (51)
+ : +- ^ ProjectExecTransformer (49)
+ : +- ^ FilterExecTransformer (48)
+ : +- ^ ScanTransformer parquet (47)
+ +- ^ ProjectExecTransformer (62)
+ +- ^ FilterExecTransformer (61)
+ +- ^ RegularHashAggregateExecTransformer (60)
+ +- ^ InputIteratorTransformer (59)
+ +- ShuffleQueryStage (57), Statistics(X)
+ +- ReusedExchange (56)
+- == Initial Plan ==
- TakeOrderedAndProject (109)
- +- HashAggregate (108)
- +- HashAggregate (107)
- +- Project (106)
- +- SortMergeJoin Inner (105)
- :- Sort (92)
- : +- Exchange (91)
- : +- Project (90)
- : +- SortMergeJoin Inner (89)
- : :- Sort (74)
- : : +- Exchange (73)
- : : +- Filter (72)
- : : +- Scan parquet (71)
- : +- Sort (88)
- : +- Exchange (87)
- : +- SortMergeJoin LeftSemi (86)
- : :- Sort (78)
- : : +- Exchange (77)
- : : +- Filter (76)
- : : +- Scan parquet (75)
- : +- Sort (85)
- : +- Project (84)
- : +- Filter (83)
- : +- HashAggregate (82)
- : +- Exchange (81)
- : +- HashAggregate (80)
- : +- Scan parquet (79)
- +- SortMergeJoin LeftSemi (104)
- :- Sort (96)
- : +- Exchange (95)
- : +- Filter (94)
- : +- Scan parquet (93)
- +- Sort (103)
- +- Project (102)
- +- Filter (101)
- +- HashAggregate (100)
- +- Exchange (99)
- +- HashAggregate (98)
- +- Scan parquet (97)
-
-
-(1) Scan parquet
+ TakeOrderedAndProject (108)
+ +- HashAggregate (107)
+ +- HashAggregate (106)
+ +- Project (105)
+ +- SortMergeJoin Inner (104)
+ :- Sort (91)
+ : +- Exchange (90)
+ : +- Project (89)
+ : +- SortMergeJoin Inner (88)
+ : :- Sort (73)
+ : : +- Exchange (72)
+ : : +- Filter (71)
+ : : +- Scan parquet (70)
+ : +- Sort (87)
+ : +- Exchange (86)
+ : +- SortMergeJoin LeftSemi (85)
+ : :- Sort (77)
+ : : +- Exchange (76)
+ : : +- Filter (75)
+ : : +- Scan parquet (74)
+ : +- Sort (84)
+ : +- Project (83)
+ : +- Filter (82)
+ : +- HashAggregate (81)
+ : +- Exchange (80)
+ : +- HashAggregate (79)
+ : +- Scan parquet (78)
+ +- SortMergeJoin LeftSemi (103)
+ :- Sort (95)
+ : +- Exchange (94)
+ : +- Filter (93)
+ : +- Scan parquet (92)
+ +- Sort (102)
+ +- Project (101)
+ +- Filter (100)
+ +- HashAggregate (99)
+ +- Exchange (98)
+ +- HashAggregate (97)
+ +- Scan parquet (96)
+
+
+(1) ScanTransformer parquet
Output [2]: [c_custkey#X, c_name#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -136,7 +135,7 @@ Input [2]: [c_custkey#X, c_name#X]
(9) InputIteratorTransformer
Input [2]: [c_custkey#X, c_name#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -173,7 +172,7 @@ Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X]
(18) InputIteratorTransformer
Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X]
-(19) Scan parquet
+(19) ScanTransformer parquet
Output [2]: [l_orderkey#X, l_quantity#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -291,7 +290,7 @@ Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X]
(46) InputIteratorTransformer
Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X]
-(47) Scan parquet
+(47) ScanTransformer parquet
Output [2]: [l_orderkey#X, l_quantity#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -375,223 +374,216 @@ Input [7]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X,
(66) RegularHashAggregateExecTransformer
Input [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_quantity#X]
Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X]
-Functions [1]: [partial_sum(l_quantity#X)]
-Aggregate Attributes [2]: [sum#X, isEmpty#X]
-Results [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X]
-
-(67) RegularHashAggregateExecTransformer
-Input [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X]
-Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X]
Functions [1]: [sum(l_quantity#X)]
Aggregate Attributes [1]: [sum(l_quantity#X)#X]
Results [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity#X)#X AS sum(l_quantity)#X]
-(68) WholeStageCodegenTransformer (X)
+(67) WholeStageCodegenTransformer (X)
Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X]
Arguments: false
-(69) TakeOrderedAndProjectExecTransformer
+(68) TakeOrderedAndProjectExecTransformer
Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X]
Arguments: X, [o_totalprice#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X], 0
-(70) VeloxColumnarToRow
+(69) VeloxColumnarToRow
Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X]
-(71) Scan parquet
+(70) Scan parquet
Output [2]: [c_custkey#X, c_name#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(c_custkey)]
ReadSchema: struct
-(72) Filter
+(71) Filter
Input [2]: [c_custkey#X, c_name#X]
Condition : isnotnull(c_custkey#X)
-(73) Exchange
+(72) Exchange
Input [2]: [c_custkey#X, c_name#X]
Arguments: hashpartitioning(c_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(74) Sort
+(73) Sort
Input [2]: [c_custkey#X, c_name#X]
Arguments: [c_custkey#X ASC NULLS FIRST], false, 0
-(75) Scan parquet
+(74) Scan parquet
Output [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(o_custkey), IsNotNull(o_orderkey)]
ReadSchema: struct
-(76) Filter
+(75) Filter
Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X]
Condition : (isnotnull(o_custkey#X) AND isnotnull(o_orderkey#X))
-(77) Exchange
+(76) Exchange
Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X]
Arguments: hashpartitioning(o_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(78) Sort
+(77) Sort
Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X]
Arguments: [o_orderkey#X ASC NULLS FIRST], false, 0
-(79) Scan parquet
+(78) Scan parquet
Output [2]: [l_orderkey#X, l_quantity#X]
Batched: true
Location: InMemoryFileIndex [*]
ReadSchema: struct
-(80) HashAggregate
+(79) HashAggregate
Input [2]: [l_orderkey#X, l_quantity#X]
Keys [1]: [l_orderkey#X]
Functions [1]: [partial_sum(l_quantity#X)]
Aggregate Attributes [2]: [sum#X, isEmpty#X]
Results [3]: [l_orderkey#X, sum#X, isEmpty#X]
-(81) Exchange
+(80) Exchange
Input [3]: [l_orderkey#X, sum#X, isEmpty#X]
Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(82) HashAggregate
+(81) HashAggregate
Input [3]: [l_orderkey#X, sum#X, isEmpty#X]
Keys [1]: [l_orderkey#X]
Functions [1]: [sum(l_quantity#X)]
Aggregate Attributes [1]: [sum(l_quantity#X)#X]
Results [2]: [l_orderkey#X, sum(l_quantity#X)#X AS sum(l_quantity#X)#X]
-(83) Filter
+(82) Filter
Input [2]: [l_orderkey#X, sum(l_quantity#X)#X]
Condition : (isnotnull(sum(l_quantity#X)#X) AND (sum(l_quantity#X)#X > 300.00))
-(84) Project
+(83) Project
Output [1]: [l_orderkey#X]
Input [2]: [l_orderkey#X, sum(l_quantity#X)#X]
-(85) Sort
+(84) Sort
Input [1]: [l_orderkey#X]
Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0
-(86) SortMergeJoin
+(85) SortMergeJoin
Left keys [1]: [o_orderkey#X]
Right keys [1]: [l_orderkey#X]
Join type: LeftSemi
Join condition: None
-(87) Exchange
+(86) Exchange
Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X]
Arguments: hashpartitioning(o_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(88) Sort
+(87) Sort
Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X]
Arguments: [o_custkey#X ASC NULLS FIRST], false, 0
-(89) SortMergeJoin
+(88) SortMergeJoin
Left keys [1]: [c_custkey#X]
Right keys [1]: [o_custkey#X]
Join type: Inner
Join condition: None
-(90) Project
+(89) Project
Output [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X]
Input [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X]
-(91) Exchange
+(90) Exchange
Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X]
Arguments: hashpartitioning(o_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(92) Sort
+(91) Sort
Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X]
Arguments: [o_orderkey#X ASC NULLS FIRST], false, 0
-(93) Scan parquet
+(92) Scan parquet
Output [2]: [l_orderkey#X, l_quantity#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(l_orderkey)]
ReadSchema: struct
-(94) Filter
+(93) Filter
Input [2]: [l_orderkey#X, l_quantity#X]
Condition : isnotnull(l_orderkey#X)
-(95) Exchange
+(94) Exchange
Input [2]: [l_orderkey#X, l_quantity#X]
Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(96) Sort
+(95) Sort
Input [2]: [l_orderkey#X, l_quantity#X]
Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0
-(97) Scan parquet
+(96) Scan parquet
Output [2]: [l_orderkey#X, l_quantity#X]
Batched: true
Location: InMemoryFileIndex [*]
ReadSchema: struct
-(98) HashAggregate
+(97) HashAggregate
Input [2]: [l_orderkey#X, l_quantity#X]
Keys [1]: [l_orderkey#X]
Functions [1]: [partial_sum(l_quantity#X)]
Aggregate Attributes [2]: [sum#X, isEmpty#X]
Results [3]: [l_orderkey#X, sum#X, isEmpty#X]
-(99) Exchange
+(98) Exchange
Input [3]: [l_orderkey#X, sum#X, isEmpty#X]
Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(100) HashAggregate
+(99) HashAggregate
Input [3]: [l_orderkey#X, sum#X, isEmpty#X]
Keys [1]: [l_orderkey#X]
Functions [1]: [sum(l_quantity#X)]
Aggregate Attributes [1]: [sum(l_quantity#X)#X]
Results [2]: [l_orderkey#X, sum(l_quantity#X)#X AS sum(l_quantity#X)#X]
-(101) Filter
+(100) Filter
Input [2]: [l_orderkey#X, sum(l_quantity#X)#X]
Condition : (isnotnull(sum(l_quantity#X)#X) AND (sum(l_quantity#X)#X > 300.00))
-(102) Project
+(101) Project
Output [1]: [l_orderkey#X]
Input [2]: [l_orderkey#X, sum(l_quantity#X)#X]
-(103) Sort
+(102) Sort
Input [1]: [l_orderkey#X]
Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0
-(104) SortMergeJoin
+(103) SortMergeJoin
Left keys [1]: [l_orderkey#X]
Right keys [1]: [l_orderkey#X]
Join type: LeftSemi
Join condition: None
-(105) SortMergeJoin
+(104) SortMergeJoin
Left keys [1]: [o_orderkey#X]
Right keys [1]: [l_orderkey#X]
Join type: Inner
Join condition: None
-(106) Project
+(105) Project
Output [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_quantity#X]
Input [7]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_orderkey#X, l_quantity#X]
-(107) HashAggregate
+(106) HashAggregate
Input [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_quantity#X]
Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X]
Functions [1]: [partial_sum(l_quantity#X)]
Aggregate Attributes [2]: [sum#X, isEmpty#X]
Results [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X]
-(108) HashAggregate
+(107) HashAggregate
Input [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X]
Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X]
Functions [1]: [sum(l_quantity#X)]
Aggregate Attributes [1]: [sum(l_quantity#X)#X]
Results [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity#X)#X AS sum(l_quantity)#X]
-(109) TakeOrderedAndProject
+(108) TakeOrderedAndProject
Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X]
Arguments: X, [o_totalprice#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X]
-(110) AdaptiveSparkPlan
+(109) AdaptiveSparkPlan
Output [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X]
Arguments: isFinalPlan=true
\ No newline at end of file
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 c72ca859ce37..4db9ca0c8393 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
@@ -1,42 +1,41 @@
== Physical Plan ==
-AdaptiveSparkPlan (38)
+AdaptiveSparkPlan (37)
+- == Final Plan ==
- VeloxColumnarToRow (24)
- +- ^ RegularHashAggregateExecTransformer (22)
- +- ^ RegularHashAggregateExecTransformer (21)
- +- ^ ProjectExecTransformer (20)
- +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19)
- :- ^ InputIteratorTransformer (9)
- : +- ShuffleQueryStage (7), Statistics(X)
- : +- ColumnarExchange (6)
- : +- VeloxResizeBatches (5)
- : +- ^ ProjectExecTransformer (3)
- : +- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
- +- ^ InputIteratorTransformer (18)
- +- ShuffleQueryStage (16), Statistics(X)
- +- ColumnarExchange (15)
- +- VeloxResizeBatches (14)
- +- ^ ProjectExecTransformer (12)
- +- ^ FilterExecTransformer (11)
- +- ^ Scan parquet (10)
+ VeloxColumnarToRow (23)
+ +- ^ RegularHashAggregateExecTransformer (21)
+ +- ^ ProjectExecTransformer (20)
+ +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19)
+ :- ^ InputIteratorTransformer (9)
+ : +- ShuffleQueryStage (7), Statistics(X)
+ : +- ColumnarExchange (6)
+ : +- VeloxResizeBatches (5)
+ : +- ^ ProjectExecTransformer (3)
+ : +- ^ FilterExecTransformer (2)
+ : +- ^ ScanTransformer parquet (1)
+ +- ^ InputIteratorTransformer (18)
+ +- ShuffleQueryStage (16), Statistics(X)
+ +- ColumnarExchange (15)
+ +- VeloxResizeBatches (14)
+ +- ^ ProjectExecTransformer (12)
+ +- ^ FilterExecTransformer (11)
+ +- ^ ScanTransformer parquet (10)
+- == Initial Plan ==
- HashAggregate (37)
- +- HashAggregate (36)
- +- Project (35)
- +- SortMergeJoin Inner (34)
- :- Sort (29)
- : +- Exchange (28)
- : +- Project (27)
- : +- Filter (26)
- : +- Scan parquet (25)
- +- Sort (33)
- +- Exchange (32)
- +- Filter (31)
- +- Scan parquet (30)
-
-
-(1) Scan parquet
+ HashAggregate (36)
+ +- HashAggregate (35)
+ +- Project (34)
+ +- SortMergeJoin Inner (33)
+ :- Sort (28)
+ : +- Exchange (27)
+ : +- Project (26)
+ : +- Filter (25)
+ : +- Scan parquet (24)
+ +- Sort (32)
+ +- Exchange (31)
+ +- Filter (30)
+ +- Scan parquet (29)
+
+
+(1) ScanTransformer parquet
Output [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -73,7 +72,7 @@ Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X]
(9) InputIteratorTransformer
Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -123,90 +122,83 @@ Input [8]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, p_partke
(21) RegularHashAggregateExecTransformer
Input [3]: [l_extendedprice#X, l_discount#X, _pre_X#X]
Keys: []
-Functions [1]: [partial_sum(_pre_X#X)]
-Aggregate Attributes [2]: [sum#X, isEmpty#X]
-Results [2]: [sum#X, isEmpty#X]
-
-(22) RegularHashAggregateExecTransformer
-Input [2]: [sum#X, isEmpty#X]
-Keys: []
-Functions [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))]
+Functions [1]: [sum(_pre_X#X)]
Aggregate Attributes [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))#X]
Results [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))#X AS revenue#X]
-(23) WholeStageCodegenTransformer (X)
+(22) WholeStageCodegenTransformer (X)
Input [1]: [revenue#X]
Arguments: false
-(24) VeloxColumnarToRow
+(23) VeloxColumnarToRow
Input [1]: [revenue#X]
-(25) Scan parquet
+(24) Scan parquet
Output [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(l_shipinstruct), In(l_shipmode, [AIR,AIR REG]), EqualTo(l_shipinstruct,DELIVER IN PERSON), IsNotNull(l_partkey), Or(Or(And(GreaterThanOrEqual(l_quantity,1.00),LessThanOrEqual(l_quantity,11.00)),And(GreaterThanOrEqual(l_quantity,10.00),LessThanOrEqual(l_quantity,20.00))),And(GreaterThanOrEqual(l_quantity,20.00),LessThanOrEqual(l_quantity,30.00)))]
ReadSchema: struct
-(26) Filter
+(25) Filter
Input [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X]
Condition : ((((isnotnull(l_shipinstruct#X) AND l_shipmode#X IN (AIR,AIR REG)) AND (l_shipinstruct#X = DELIVER IN PERSON)) AND isnotnull(l_partkey#X)) AND ((((l_quantity#X >= 1.00) AND (l_quantity#X <= 11.00)) OR ((l_quantity#X >= 10.00) AND (l_quantity#X <= 20.00))) OR ((l_quantity#X >= 20.00) AND (l_quantity#X <= 30.00))))
-(27) Project
+(26) Project
Output [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X]
Input [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X]
-(28) Exchange
+(27) Exchange
Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X]
Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(29) Sort
+(28) Sort
Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X]
Arguments: [l_partkey#X ASC NULLS FIRST], false, 0
-(30) Scan parquet
+(29) Scan parquet
Output [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(p_size), GreaterThanOrEqual(p_size,1), IsNotNull(p_partkey), Or(Or(And(And(EqualTo(p_brand,Brand#X),In(p_container, [SM BOX,SM CASE,SM PACK,SM PKG])),LessThanOrEqual(p_size,5)),And(And(EqualTo(p_brand,Brand#X),In(p_container, [MED BAG,MED BOX,MED PACK,MED PKG])),LessThanOrEqual(p_size,10))),And(And(EqualTo(p_brand,Brand#X),In(p_container, [LG BOX,LG CASE,LG PACK,LG PKG])),LessThanOrEqual(p_size,15)))]
ReadSchema: struct
-(31) Filter
+(30) Filter
Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X]
Condition : (((isnotnull(p_size#X) AND (p_size#X >= 1)) AND isnotnull(p_partkey#X)) AND (((((p_brand#X = Brand#X) AND p_container#X IN (SM CASE,SM BOX,SM PACK,SM PKG)) AND (p_size#X <= 5)) OR (((p_brand#X = Brand#X) AND p_container#X IN (MED BAG,MED BOX,MED PKG,MED PACK)) AND (p_size#X <= 10))) OR (((p_brand#X = Brand#X) AND p_container#X IN (LG CASE,LG BOX,LG PACK,LG PKG)) AND (p_size#X <= 15))))
-(32) Exchange
+(31) Exchange
Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X]
Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(33) Sort
+(32) Sort
Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X]
Arguments: [p_partkey#X ASC NULLS FIRST], false, 0
-(34) SortMergeJoin
+(33) SortMergeJoin
Left keys [1]: [l_partkey#X]
Right keys [1]: [p_partkey#X]
Join type: Inner
Join condition: (((((((p_brand#X = Brand#X) AND p_container#X IN (SM CASE,SM BOX,SM PACK,SM PKG)) AND (l_quantity#X >= 1.00)) AND (l_quantity#X <= 11.00)) AND (p_size#X <= 5)) OR (((((p_brand#X = Brand#X) AND p_container#X IN (MED BAG,MED BOX,MED PKG,MED PACK)) AND (l_quantity#X >= 10.00)) AND (l_quantity#X <= 20.00)) AND (p_size#X <= 10))) OR (((((p_brand#X = Brand#X) AND p_container#X IN (LG CASE,LG BOX,LG PACK,LG PKG)) AND (l_quantity#X >= 20.00)) AND (l_quantity#X <= 30.00)) AND (p_size#X <= 15)))
-(35) Project
+(34) Project
Output [2]: [l_extendedprice#X, l_discount#X]
Input [8]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, p_partkey#X, p_brand#X, p_size#X, p_container#X]
-(36) HashAggregate
+(35) HashAggregate
Input [2]: [l_extendedprice#X, l_discount#X]
Keys: []
Functions [1]: [partial_sum((l_extendedprice#X * (1 - l_discount#X)))]
Aggregate Attributes [2]: [sum#X, isEmpty#X]
Results [2]: [sum#X, isEmpty#X]
-(37) HashAggregate
+(36) HashAggregate
Input [2]: [sum#X, isEmpty#X]
Keys: []
Functions [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))]
Aggregate Attributes [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))#X]
Results [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))#X AS revenue#X]
-(38) AdaptiveSparkPlan
+(37) AdaptiveSparkPlan
Output [1]: [revenue#X]
Arguments: isFinalPlan=true
\ No newline at end of file
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 ed09215178a4..4d8eee1f1fb7 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
@@ -1,32 +1,32 @@
== Physical Plan ==
-AdaptiveSparkPlan (143)
+AdaptiveSparkPlan (142)
+- == Final Plan ==
- VeloxColumnarToRow (93)
- +- AQEShuffleRead (92)
- +- ShuffleQueryStage (91), Statistics(X)
- +- ColumnarExchange (90)
- +- VeloxResizeBatches (89)
- +- ^ ProjectExecTransformer (87)
- +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (86)
- :- ^ InputIteratorTransformer (76)
- : +- ShuffleQueryStage (74), Statistics(X)
- : +- ColumnarExchange (73)
- : +- VeloxResizeBatches (72)
- : +- ^ ProjectExecTransformer (70)
- : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (69)
+ VeloxColumnarToRow (92)
+ +- AQEShuffleRead (91)
+ +- ShuffleQueryStage (90), Statistics(X)
+ +- ColumnarExchange (89)
+ +- VeloxResizeBatches (88)
+ +- ^ ProjectExecTransformer (86)
+ +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (85)
+ :- ^ InputIteratorTransformer (75)
+ : +- ShuffleQueryStage (73), Statistics(X)
+ : +- ColumnarExchange (72)
+ : +- VeloxResizeBatches (71)
+ : +- ^ ProjectExecTransformer (69)
+ : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (68)
: :- ^ InputIteratorTransformer (9)
: : +- ShuffleQueryStage (7), Statistics(X)
: : +- ColumnarExchange (6)
: : +- VeloxResizeBatches (5)
: : +- ^ ProjectExecTransformer (3)
: : +- ^ FilterExecTransformer (2)
- : : +- ^ Scan parquet (1)
- : +- ^ InputIteratorTransformer (68)
- : +- ShuffleQueryStage (66), Statistics(X)
- : +- ColumnarExchange (65)
- : +- VeloxResizeBatches (64)
- : +- ^ ProjectExecTransformer (62)
- : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (61)
+ : : +- ^ ScanTransformer parquet (1)
+ : +- ^ InputIteratorTransformer (67)
+ : +- ShuffleQueryStage (65), Statistics(X)
+ : +- ColumnarExchange (64)
+ : +- VeloxResizeBatches (63)
+ : +- ^ ProjectExecTransformer (61)
+ : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (60)
: :- ^ InputIteratorTransformer (35)
: : +- ShuffleQueryStage (33), Statistics(X)
: : +- ColumnarExchange (32)
@@ -39,94 +39,93 @@ AdaptiveSparkPlan (143)
: : : +- VeloxResizeBatches (14)
: : : +- ^ ProjectExecTransformer (12)
: : : +- ^ FilterExecTransformer (11)
- : : : +- ^ Scan parquet (10)
+ : : : +- ^ ScanTransformer parquet (10)
: : +- ^ InputIteratorTransformer (27)
: : +- ShuffleQueryStage (25), Statistics(X)
: : +- ColumnarExchange (24)
: : +- VeloxResizeBatches (23)
: : +- ^ ProjectExecTransformer (21)
: : +- ^ FilterExecTransformer (20)
- : : +- ^ Scan parquet (19)
- : +- ^ InputIteratorTransformer (60)
- : +- ShuffleQueryStage (58), Statistics(X)
- : +- ColumnarExchange (57)
- : +- VeloxResizeBatches (56)
- : +- ^ ProjectExecTransformer (54)
- : +- ^ FilterExecTransformer (53)
- : +- ^ ProjectExecTransformer (52)
- : +- ^ RegularHashAggregateExecTransformer (51)
- : +- ^ RegularHashAggregateExecTransformer (50)
- : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (49)
- : :- ^ InputIteratorTransformer (44)
- : : +- ShuffleQueryStage (42), Statistics(X)
- : : +- ColumnarExchange (41)
- : : +- VeloxResizeBatches (40)
- : : +- ^ ProjectExecTransformer (38)
- : : +- ^ FilterExecTransformer (37)
- : : +- ^ Scan parquet (36)
- : +- ^ InputIteratorTransformer (48)
- : +- ShuffleQueryStage (46), Statistics(X)
- : +- ReusedExchange (45)
- +- ^ InputIteratorTransformer (85)
- +- ShuffleQueryStage (83), Statistics(X)
- +- ColumnarExchange (82)
- +- VeloxResizeBatches (81)
- +- ^ ProjectExecTransformer (79)
- +- ^ FilterExecTransformer (78)
- +- ^ Scan parquet (77)
+ : : +- ^ ScanTransformer parquet (19)
+ : +- ^ InputIteratorTransformer (59)
+ : +- ShuffleQueryStage (57), Statistics(X)
+ : +- ColumnarExchange (56)
+ : +- VeloxResizeBatches (55)
+ : +- ^ ProjectExecTransformer (53)
+ : +- ^ FilterExecTransformer (52)
+ : +- ^ ProjectExecTransformer (51)
+ : +- ^ RegularHashAggregateExecTransformer (50)
+ : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (49)
+ : :- ^ InputIteratorTransformer (44)
+ : : +- ShuffleQueryStage (42), Statistics(X)
+ : : +- ColumnarExchange (41)
+ : : +- VeloxResizeBatches (40)
+ : : +- ^ ProjectExecTransformer (38)
+ : : +- ^ FilterExecTransformer (37)
+ : : +- ^ ScanTransformer parquet (36)
+ : +- ^ InputIteratorTransformer (48)
+ : +- ShuffleQueryStage (46), Statistics(X)
+ : +- ReusedExchange (45)
+ +- ^ InputIteratorTransformer (84)
+ +- ShuffleQueryStage (82), Statistics(X)
+ +- ColumnarExchange (81)
+ +- VeloxResizeBatches (80)
+ +- ^ ProjectExecTransformer (78)
+ +- ^ FilterExecTransformer (77)
+ +- ^ ScanTransformer parquet (76)
+- == Initial Plan ==
- Sort (142)
- +- Exchange (141)
- +- Project (140)
- +- SortMergeJoin Inner (139)
- :- Sort (133)
- : +- Exchange (132)
- : +- Project (131)
- : +- SortMergeJoin LeftSemi (130)
- : :- Sort (97)
- : : +- Exchange (96)
- : : +- Filter (95)
- : : +- Scan parquet (94)
- : +- Sort (129)
- : +- Exchange (128)
- : +- Project (127)
- : +- SortMergeJoin Inner (126)
- : :- Sort (109)
- : : +- Exchange (108)
- : : +- SortMergeJoin LeftSemi (107)
- : : :- Sort (101)
- : : : +- Exchange (100)
- : : : +- Filter (99)
- : : : +- Scan parquet (98)
- : : +- Sort (106)
- : : +- Exchange (105)
- : : +- Project (104)
- : : +- Filter (103)
- : : +- Scan parquet (102)
- : +- Sort (125)
- : +- Exchange (124)
- : +- Filter (123)
- : +- HashAggregate (122)
- : +- HashAggregate (121)
- : +- SortMergeJoin LeftSemi (120)
- : :- Sort (114)
- : : +- Exchange (113)
- : : +- Project (112)
- : : +- Filter (111)
- : : +- Scan parquet (110)
- : +- Sort (119)
- : +- Exchange (118)
- : +- Project (117)
- : +- Filter (116)
- : +- Scan parquet (115)
- +- Sort (138)
- +- Exchange (137)
- +- Project (136)
- +- Filter (135)
- +- Scan parquet (134)
-
-
-(1) Scan parquet
+ Sort (141)
+ +- Exchange (140)
+ +- Project (139)
+ +- SortMergeJoin Inner (138)
+ :- Sort (132)
+ : +- Exchange (131)
+ : +- Project (130)
+ : +- SortMergeJoin LeftSemi (129)
+ : :- Sort (96)
+ : : +- Exchange (95)
+ : : +- Filter (94)
+ : : +- Scan parquet (93)
+ : +- Sort (128)
+ : +- Exchange (127)
+ : +- Project (126)
+ : +- SortMergeJoin Inner (125)
+ : :- Sort (108)
+ : : +- Exchange (107)
+ : : +- SortMergeJoin LeftSemi (106)
+ : : :- Sort (100)
+ : : : +- Exchange (99)
+ : : : +- Filter (98)
+ : : : +- Scan parquet (97)
+ : : +- Sort (105)
+ : : +- Exchange (104)
+ : : +- Project (103)
+ : : +- Filter (102)
+ : : +- Scan parquet (101)
+ : +- Sort (124)
+ : +- Exchange (123)
+ : +- Filter (122)
+ : +- HashAggregate (121)
+ : +- HashAggregate (120)
+ : +- SortMergeJoin LeftSemi (119)
+ : :- Sort (113)
+ : : +- Exchange (112)
+ : : +- Project (111)
+ : : +- Filter (110)
+ : : +- Scan parquet (109)
+ : +- Sort (118)
+ : +- Exchange (117)
+ : +- Project (116)
+ : +- Filter (115)
+ : +- Scan parquet (114)
+ +- Sort (137)
+ +- Exchange (136)
+ +- Project (135)
+ +- Filter (134)
+ +- Scan parquet (133)
+
+
+(1) ScanTransformer parquet
Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -163,7 +162,7 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X]
(9) InputIteratorTransformer
Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -200,7 +199,7 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
(18) InputIteratorTransformer
Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
-(19) Scan parquet
+(19) ScanTransformer parquet
Output [2]: [p_partkey#X, p_name#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -269,7 +268,7 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
(35) InputIteratorTransformer
Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
-(36) Scan parquet
+(36) ScanTransformer parquet
Output [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -328,415 +327,408 @@ Join condition: None
(50) RegularHashAggregateExecTransformer
Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X]
Keys [2]: [l_partkey#X, l_suppkey#X]
-Functions [1]: [partial_sum(l_quantity#X)]
-Aggregate Attributes [2]: [sum#X, isEmpty#X]
-Results [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X]
-
-(51) RegularHashAggregateExecTransformer
-Input [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X]
-Keys [2]: [l_partkey#X, l_suppkey#X]
Functions [1]: [sum(l_quantity#X)]
Aggregate Attributes [1]: [sum(l_quantity#X)#X]
Results [3]: [l_partkey#X, l_suppkey#X, sum(l_quantity#X)#X]
-(52) ProjectExecTransformer
+(51) ProjectExecTransformer
Output [3]: [(0.5 * sum(l_quantity#X)#X) AS (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
Input [3]: [l_partkey#X, l_suppkey#X, sum(l_quantity#X)#X]
-(53) FilterExecTransformer
+(52) FilterExecTransformer
Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
Arguments: isnotnull((0.5 * sum(l_quantity))#X)
-(54) ProjectExecTransformer
+(53) ProjectExecTransformer
Output [4]: [hash(l_partkey#X, l_suppkey#X, 42) AS hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
-(55) WholeStageCodegenTransformer (X)
+(54) WholeStageCodegenTransformer (X)
Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
Arguments: false
-(56) VeloxResizeBatches
+(55) VeloxResizeBatches
Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
Arguments: X, X
-(57) ColumnarExchange
+(56) ColumnarExchange
Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
Arguments: hashpartitioning(l_partkey#X, l_suppkey#X, 1), ENSURE_REQUIREMENTS, [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X], [plan_id=X], [shuffle_writer_type=hash]
-(58) ShuffleQueryStage
+(57) ShuffleQueryStage
Output [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
Arguments: X
-(59) InputAdapter
+(58) InputAdapter
Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
-(60) InputIteratorTransformer
+(59) InputIteratorTransformer
Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
-(61) ShuffledHashJoinExecTransformer
+(60) ShuffledHashJoinExecTransformer
Left keys [2]: [ps_partkey#X, ps_suppkey#X]
Right keys [2]: [l_partkey#X, l_suppkey#X]
Join type: Inner
Join condition: (cast(ps_availqty#X as decimal(24,3)) > (0.5 * sum(l_quantity))#X)
-(62) ProjectExecTransformer
+(61) ProjectExecTransformer
Output [2]: [hash(ps_suppkey#X, 42) AS hash_partition_key#X, ps_suppkey#X]
Input [6]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
-(63) WholeStageCodegenTransformer (X)
+(62) WholeStageCodegenTransformer (X)
Input [2]: [hash_partition_key#X, ps_suppkey#X]
Arguments: false
-(64) VeloxResizeBatches
+(63) VeloxResizeBatches
Input [2]: [hash_partition_key#X, ps_suppkey#X]
Arguments: X, X
-(65) ColumnarExchange
+(64) ColumnarExchange
Input [2]: [hash_partition_key#X, ps_suppkey#X]
Arguments: hashpartitioning(ps_suppkey#X, 1), ENSURE_REQUIREMENTS, [ps_suppkey#X], [plan_id=X], [shuffle_writer_type=hash]
-(66) ShuffleQueryStage
+(65) ShuffleQueryStage
Output [1]: [ps_suppkey#X]
Arguments: X
-(67) InputAdapter
+(66) InputAdapter
Input [1]: [ps_suppkey#X]
-(68) InputIteratorTransformer
+(67) InputIteratorTransformer
Input [1]: [ps_suppkey#X]
-(69) ShuffledHashJoinExecTransformer
+(68) ShuffledHashJoinExecTransformer
Left keys [1]: [s_suppkey#X]
Right keys [1]: [ps_suppkey#X]
Join type: LeftSemi
Join condition: None
-(70) ProjectExecTransformer
+(69) ProjectExecTransformer
Output [4]: [hash(s_nationkey#X, 42) AS hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X]
Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X]
-(71) WholeStageCodegenTransformer (X)
+(70) WholeStageCodegenTransformer (X)
Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X]
Arguments: false
-(72) VeloxResizeBatches
+(71) VeloxResizeBatches
Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X]
Arguments: X, X
-(73) ColumnarExchange
+(72) ColumnarExchange
Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X]
Arguments: hashpartitioning(s_nationkey#X, 1), ENSURE_REQUIREMENTS, [s_name#X, s_address#X, s_nationkey#X], [plan_id=X], [shuffle_writer_type=hash]
-(74) ShuffleQueryStage
+(73) ShuffleQueryStage
Output [3]: [s_name#X, s_address#X, s_nationkey#X]
Arguments: X
-(75) InputAdapter
+(74) InputAdapter
Input [3]: [s_name#X, s_address#X, s_nationkey#X]
-(76) InputIteratorTransformer
+(75) InputIteratorTransformer
Input [3]: [s_name#X, s_address#X, s_nationkey#X]
-(77) Scan parquet
+(76) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(n_name), EqualTo(n_name,CANADA), IsNotNull(n_nationkey)]
ReadSchema: struct
-(78) FilterExecTransformer
+(77) FilterExecTransformer
Input [2]: [n_nationkey#X, n_name#X]
Arguments: ((isnotnull(n_name#X) AND (n_name#X = CANADA)) AND isnotnull(n_nationkey#X))
-(79) ProjectExecTransformer
+(78) ProjectExecTransformer
Output [2]: [hash(n_nationkey#X, 42) AS hash_partition_key#X, n_nationkey#X]
Input [2]: [n_nationkey#X, n_name#X]
-(80) WholeStageCodegenTransformer (X)
+(79) WholeStageCodegenTransformer (X)
Input [2]: [hash_partition_key#X, n_nationkey#X]
Arguments: false
-(81) VeloxResizeBatches
+(80) VeloxResizeBatches
Input [2]: [hash_partition_key#X, n_nationkey#X]
Arguments: X, X
-(82) ColumnarExchange
+(81) ColumnarExchange
Input [2]: [hash_partition_key#X, n_nationkey#X]
Arguments: hashpartitioning(n_nationkey#X, 1), ENSURE_REQUIREMENTS, [n_nationkey#X], [plan_id=X], [shuffle_writer_type=hash]
-(83) ShuffleQueryStage
+(82) ShuffleQueryStage
Output [1]: [n_nationkey#X]
Arguments: X
-(84) InputAdapter
+(83) InputAdapter
Input [1]: [n_nationkey#X]
-(85) InputIteratorTransformer
+(84) InputIteratorTransformer
Input [1]: [n_nationkey#X]
-(86) ShuffledHashJoinExecTransformer
+(85) ShuffledHashJoinExecTransformer
Left keys [1]: [s_nationkey#X]
Right keys [1]: [n_nationkey#X]
Join type: Inner
Join condition: None
-(87) ProjectExecTransformer
+(86) ProjectExecTransformer
Output [2]: [s_name#X, s_address#X]
Input [4]: [s_name#X, s_address#X, s_nationkey#X, n_nationkey#X]
-(88) WholeStageCodegenTransformer (X)
+(87) WholeStageCodegenTransformer (X)
Input [2]: [s_name#X, s_address#X]
Arguments: false
-(89) VeloxResizeBatches
+(88) VeloxResizeBatches
Input [2]: [s_name#X, s_address#X]
Arguments: X, X
-(90) ColumnarExchange
+(89) ColumnarExchange
Input [2]: [s_name#X, s_address#X]
Arguments: rangepartitioning(s_name#X ASC NULLS FIRST, 1), ENSURE_REQUIREMENTS, [plan_id=X], [shuffle_writer_type=hash]
-(91) ShuffleQueryStage
+(90) ShuffleQueryStage
Output [2]: [s_name#X, s_address#X]
Arguments: X
-(92) AQEShuffleRead
+(91) AQEShuffleRead
Input [2]: [s_name#X, s_address#X]
Arguments: local
-(93) VeloxColumnarToRow
+(92) VeloxColumnarToRow
Input [2]: [s_name#X, s_address#X]
-(94) Scan parquet
+(93) Scan parquet
Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(s_nationkey)]
ReadSchema: struct
-(95) Filter
+(94) Filter
Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X]
Condition : isnotnull(s_nationkey#X)
-(96) Exchange
+(95) Exchange
Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X]
Arguments: hashpartitioning(s_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(97) Sort
+(96) Sort
Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X]
Arguments: [s_suppkey#X ASC NULLS FIRST], false, 0
-(98) Scan parquet
+(97) Scan parquet
Output [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(ps_availqty), IsNotNull(ps_partkey), IsNotNull(ps_suppkey)]
ReadSchema: struct
-(99) Filter
+(98) Filter
Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
Condition : ((isnotnull(ps_availqty#X) AND isnotnull(ps_partkey#X)) AND isnotnull(ps_suppkey#X))
-(100) Exchange
+(99) Exchange
Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
Arguments: hashpartitioning(ps_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(101) Sort
+(100) Sort
Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
Arguments: [ps_partkey#X ASC NULLS FIRST], false, 0
-(102) Scan parquet
+(101) Scan parquet
Output [2]: [p_partkey#X, p_name#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(p_name), StringStartsWith(p_name,forest)]
ReadSchema: struct
-(103) Filter
+(102) Filter
Input [2]: [p_partkey#X, p_name#X]
Condition : (isnotnull(p_name#X) AND StartsWith(p_name#X, forest))
-(104) Project
+(103) Project
Output [1]: [p_partkey#X]
Input [2]: [p_partkey#X, p_name#X]
-(105) Exchange
+(104) Exchange
Input [1]: [p_partkey#X]
Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(106) Sort
+(105) Sort
Input [1]: [p_partkey#X]
Arguments: [p_partkey#X ASC NULLS FIRST], false, 0
-(107) SortMergeJoin
+(106) SortMergeJoin
Left keys [1]: [ps_partkey#X]
Right keys [1]: [p_partkey#X]
Join type: LeftSemi
Join condition: None
-(108) Exchange
+(107) Exchange
Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
Arguments: hashpartitioning(ps_partkey#X, ps_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(109) Sort
+(108) Sort
Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X]
Arguments: [ps_partkey#X ASC NULLS FIRST, ps_suppkey#X ASC NULLS FIRST], false, 0
-(110) Scan parquet
+(109) Scan parquet
Output [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1994-01-01), LessThan(l_shipdate,1995-01-01), IsNotNull(l_partkey), IsNotNull(l_suppkey)]
ReadSchema: struct
-(111) Filter
+(110) Filter
Input [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X]
Condition : ((((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1994-01-01)) AND (l_shipdate#X < 1995-01-01)) AND isnotnull(l_partkey#X)) AND isnotnull(l_suppkey#X))
-(112) Project
+(111) Project
Output [3]: [l_partkey#X, l_suppkey#X, l_quantity#X]
Input [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X]
-(113) Exchange
+(112) Exchange
Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X]
Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(114) Sort
+(113) Sort
Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X]
Arguments: [l_partkey#X ASC NULLS FIRST], false, 0
-(115) Scan parquet
+(114) Scan parquet
Output [2]: [p_partkey#X, p_name#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(p_name), StringStartsWith(p_name,forest)]
ReadSchema: struct
-(116) Filter
+(115) Filter
Input [2]: [p_partkey#X, p_name#X]
Condition : (isnotnull(p_name#X) AND StartsWith(p_name#X, forest))
-(117) Project
+(116) Project
Output [1]: [p_partkey#X]
Input [2]: [p_partkey#X, p_name#X]
-(118) Exchange
+(117) Exchange
Input [1]: [p_partkey#X]
Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(119) Sort
+(118) Sort
Input [1]: [p_partkey#X]
Arguments: [p_partkey#X ASC NULLS FIRST], false, 0
-(120) SortMergeJoin
+(119) SortMergeJoin
Left keys [1]: [l_partkey#X]
Right keys [1]: [p_partkey#X]
Join type: LeftSemi
Join condition: None
-(121) HashAggregate
+(120) HashAggregate
Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X]
Keys [2]: [l_partkey#X, l_suppkey#X]
Functions [1]: [partial_sum(l_quantity#X)]
Aggregate Attributes [2]: [sum#X, isEmpty#X]
Results [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X]
-(122) HashAggregate
+(121) HashAggregate
Input [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X]
Keys [2]: [l_partkey#X, l_suppkey#X]
Functions [1]: [sum(l_quantity#X)]
Aggregate Attributes [1]: [sum(l_quantity#X)#X]
Results [3]: [(0.5 * sum(l_quantity#X)#X) AS (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
-(123) Filter
+(122) Filter
Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
Condition : isnotnull((0.5 * sum(l_quantity))#X)
-(124) Exchange
+(123) Exchange
Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
Arguments: hashpartitioning(l_partkey#X, l_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(125) Sort
+(124) Sort
Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
Arguments: [l_partkey#X ASC NULLS FIRST, l_suppkey#X ASC NULLS FIRST], false, 0
-(126) SortMergeJoin
+(125) SortMergeJoin
Left keys [2]: [ps_partkey#X, ps_suppkey#X]
Right keys [2]: [l_partkey#X, l_suppkey#X]
Join type: Inner
Join condition: (cast(ps_availqty#X as decimal(24,3)) > (0.5 * sum(l_quantity))#X)
-(127) Project
+(126) Project
Output [1]: [ps_suppkey#X]
Input [6]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X]
-(128) Exchange
+(127) Exchange
Input [1]: [ps_suppkey#X]
Arguments: hashpartitioning(ps_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(129) Sort
+(128) Sort
Input [1]: [ps_suppkey#X]
Arguments: [ps_suppkey#X ASC NULLS FIRST], false, 0
-(130) SortMergeJoin
+(129) SortMergeJoin
Left keys [1]: [s_suppkey#X]
Right keys [1]: [ps_suppkey#X]
Join type: LeftSemi
Join condition: None
-(131) Project
+(130) Project
Output [3]: [s_name#X, s_address#X, s_nationkey#X]
Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X]
-(132) Exchange
+(131) Exchange
Input [3]: [s_name#X, s_address#X, s_nationkey#X]
Arguments: hashpartitioning(s_nationkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(133) Sort
+(132) Sort
Input [3]: [s_name#X, s_address#X, s_nationkey#X]
Arguments: [s_nationkey#X ASC NULLS FIRST], false, 0
-(134) Scan parquet
+(133) Scan parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(n_name), EqualTo(n_name,CANADA), IsNotNull(n_nationkey)]
ReadSchema: struct
-(135) Filter
+(134) Filter
Input [2]: [n_nationkey#X, n_name#X]
Condition : ((isnotnull(n_name#X) AND (n_name#X = CANADA)) AND isnotnull(n_nationkey#X))
-(136) Project
+(135) Project
Output [1]: [n_nationkey#X]
Input [2]: [n_nationkey#X, n_name#X]
-(137) Exchange
+(136) Exchange
Input [1]: [n_nationkey#X]
Arguments: hashpartitioning(n_nationkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(138) Sort
+(137) Sort
Input [1]: [n_nationkey#X]
Arguments: [n_nationkey#X ASC NULLS FIRST], false, 0
-(139) SortMergeJoin
+(138) SortMergeJoin
Left keys [1]: [s_nationkey#X]
Right keys [1]: [n_nationkey#X]
Join type: Inner
Join condition: None
-(140) Project
+(139) Project
Output [2]: [s_name#X, s_address#X]
Input [4]: [s_name#X, s_address#X, s_nationkey#X, n_nationkey#X]
-(141) Exchange
+(140) Exchange
Input [2]: [s_name#X, s_address#X]
Arguments: rangepartitioning(s_name#X ASC NULLS FIRST, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(142) Sort
+(141) Sort
Input [2]: [s_name#X, s_address#X]
Arguments: [s_name#X ASC NULLS FIRST], true, 0
-(143) AdaptiveSparkPlan
+(142) AdaptiveSparkPlan
Output [2]: [s_name#X, s_address#X]
Arguments: isFinalPlan=true
\ No newline at end of file
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 271d0c6d5098..96750b2b3ecc 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
@@ -29,7 +29,7 @@ AdaptiveSparkPlan (137)
: : : +- VeloxResizeBatches (5)
: : : +- ^ ProjectExecTransformer (3)
: : : +- ^ FilterExecTransformer (2)
- : : : +- ^ Scan parquet (1)
+ : : : +- ^ ScanTransformer parquet (1)
: : +- ^ InputIteratorTransformer (44)
: : +- ShuffleQueryStage (42), Statistics(X)
: : +- ColumnarExchange (41)
@@ -43,34 +43,34 @@ AdaptiveSparkPlan (137)
: : : : +- VeloxResizeBatches (14)
: : : : +- ^ ProjectExecTransformer (12)
: : : : +- ^ FilterExecTransformer (11)
- : : : : +- ^ Scan parquet (10)
+ : : : : +- ^ ScanTransformer parquet (10)
: : : +- ^ InputIteratorTransformer (26)
: : : +- ShuffleQueryStage (24), Statistics(X)
: : : +- ColumnarExchange (23)
: : : +- VeloxResizeBatches (22)
: : : +- ^ ProjectExecTransformer (20)
- : : : +- ^ Scan parquet (19)
+ : : : +- ^ ScanTransformer parquet (19)
: : +- ^ InputIteratorTransformer (36)
: : +- ShuffleQueryStage (34), Statistics(X)
: : +- ColumnarExchange (33)
: : +- VeloxResizeBatches (32)
: : +- ^ ProjectExecTransformer (30)
: : +- ^ FilterExecTransformer (29)
- : : +- ^ Scan parquet (28)
+ : : +- ^ ScanTransformer parquet (28)
: +- ^ InputIteratorTransformer (61)
: +- ShuffleQueryStage (59), Statistics(X)
: +- ColumnarExchange (58)
: +- VeloxResizeBatches (57)
: +- ^ ProjectExecTransformer (55)
: +- ^ FilterExecTransformer (54)
- : +- ^ Scan parquet (53)
+ : +- ^ ScanTransformer parquet (53)
+- ^ InputIteratorTransformer (78)
+- ShuffleQueryStage (76), Statistics(X)
+- ColumnarExchange (75)
+- VeloxResizeBatches (74)
+- ^ ProjectExecTransformer (72)
+- ^ FilterExecTransformer (71)
- +- ^ Scan parquet (70)
+ +- ^ ScanTransformer parquet (70)
+- == Initial Plan ==
TakeOrderedAndProject (136)
+- HashAggregate (135)
@@ -119,7 +119,7 @@ AdaptiveSparkPlan (137)
+- Scan parquet (126)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [3]: [s_suppkey#X, s_name#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -156,7 +156,7 @@ Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X]
(9) InputIteratorTransformer
Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -193,7 +193,7 @@ Input [2]: [l_orderkey#X, l_suppkey#X]
(18) InputIteratorTransformer
Input [2]: [l_orderkey#X, l_suppkey#X]
-(19) Scan parquet
+(19) ScanTransformer parquet
Output [2]: [l_orderkey#X, l_suppkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -231,7 +231,7 @@ Right keys [1]: [l_orderkey#X]
Join type: LeftSemi
Join condition: NOT (l_suppkey#X = l_suppkey#X)
-(28) Scan parquet
+(28) ScanTransformer parquet
Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -332,7 +332,7 @@ Input [3]: [s_name#X, s_nationkey#X, l_orderkey#X]
(52) InputIteratorTransformer
Input [3]: [s_name#X, s_nationkey#X, l_orderkey#X]
-(53) Scan parquet
+(53) ScanTransformer parquet
Output [2]: [o_orderkey#X, o_orderstatus#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -401,7 +401,7 @@ Input [2]: [s_name#X, s_nationkey#X]
(69) InputIteratorTransformer
Input [2]: [s_name#X, s_nationkey#X]
-(70) Scan parquet
+(70) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 b8123fb6dc0e..c6e147da04d8 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
@@ -22,13 +22,13 @@ AdaptiveSparkPlan (52)
: +- VeloxResizeBatches (5)
: +- ^ ProjectExecTransformer (3)
: +- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
+ : +- ^ ScanTransformer parquet (1)
+- ^ InputIteratorTransformer (17)
+- ShuffleQueryStage (15), Statistics(X)
+- ColumnarExchange (14)
+- VeloxResizeBatches (13)
+- ^ ProjectExecTransformer (11)
- +- ^ Scan parquet (10)
+ +- ^ ScanTransformer parquet (10)
+- == Initial Plan ==
Sort (51)
+- Exchange (50)
@@ -46,7 +46,7 @@ AdaptiveSparkPlan (52)
+- Scan parquet (42)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [3]: [c_custkey#X, c_phone#X, c_acctbal#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -83,7 +83,7 @@ Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X]
(9) InputIteratorTransformer
Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [1]: [o_custkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -285,7 +285,7 @@ AdaptiveSparkPlan (72)
+- ^ FlushableHashAggregateExecTransformer (56)
+- ^ ProjectExecTransformer (55)
+- ^ FilterExecTransformer (54)
- +- ^ Scan parquet (53)
+ +- ^ ScanTransformer parquet (53)
+- == Initial Plan ==
HashAggregate (71)
+- Exchange (70)
@@ -295,7 +295,7 @@ AdaptiveSparkPlan (72)
+- Scan parquet (66)
-(53) Scan parquet
+(53) ScanTransformer parquet
Output [2]: [c_phone#X, c_acctbal#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -402,7 +402,7 @@ AdaptiveSparkPlan (72)
+- ^ FlushableHashAggregateExecTransformer (56)
+- ^ ProjectExecTransformer (55)
+- ^ FilterExecTransformer (54)
- +- ^ Scan parquet (53)
+ +- ^ ScanTransformer parquet (53)
+- == Initial Plan ==
HashAggregate (71)
+- Exchange (70)
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 8eea7581a417..1f86369a62af 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
@@ -1,67 +1,66 @@
== Physical Plan ==
-AdaptiveSparkPlan (67)
+AdaptiveSparkPlan (66)
+- == Final Plan ==
- VeloxColumnarToRow (43)
- +- TakeOrderedAndProjectExecTransformer (42)
- +- ^ ProjectExecTransformer (40)
- +- ^ RegularHashAggregateExecTransformer (39)
- +- ^ RegularHashAggregateExecTransformer (38)
- +- ^ ProjectExecTransformer (37)
- +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (36)
- :- ^ InputIteratorTransformer (26)
- : +- ShuffleQueryStage (24), Statistics(X)
- : +- ColumnarExchange (23)
- : +- VeloxResizeBatches (22)
- : +- ^ ProjectExecTransformer (20)
- : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19)
- : :- ^ InputIteratorTransformer (9)
- : : +- ShuffleQueryStage (7), Statistics(X)
- : : +- ColumnarExchange (6)
- : : +- VeloxResizeBatches (5)
- : : +- ^ ProjectExecTransformer (3)
- : : +- ^ FilterExecTransformer (2)
- : : +- ^ Scan parquet (1)
- : +- ^ InputIteratorTransformer (18)
- : +- ShuffleQueryStage (16), Statistics(X)
- : +- ColumnarExchange (15)
- : +- VeloxResizeBatches (14)
- : +- ^ ProjectExecTransformer (12)
- : +- ^ FilterExecTransformer (11)
- : +- ^ Scan parquet (10)
- +- ^ InputIteratorTransformer (35)
- +- ShuffleQueryStage (33), Statistics(X)
- +- ColumnarExchange (32)
- +- VeloxResizeBatches (31)
- +- ^ ProjectExecTransformer (29)
- +- ^ FilterExecTransformer (28)
- +- ^ Scan parquet (27)
+ VeloxColumnarToRow (42)
+ +- TakeOrderedAndProjectExecTransformer (41)
+ +- ^ ProjectExecTransformer (39)
+ +- ^ RegularHashAggregateExecTransformer (38)
+ +- ^ ProjectExecTransformer (37)
+ +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (36)
+ :- ^ InputIteratorTransformer (26)
+ : +- ShuffleQueryStage (24), Statistics(X)
+ : +- ColumnarExchange (23)
+ : +- VeloxResizeBatches (22)
+ : +- ^ ProjectExecTransformer (20)
+ : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19)
+ : :- ^ InputIteratorTransformer (9)
+ : : +- ShuffleQueryStage (7), Statistics(X)
+ : : +- ColumnarExchange (6)
+ : : +- VeloxResizeBatches (5)
+ : : +- ^ ProjectExecTransformer (3)
+ : : +- ^ FilterExecTransformer (2)
+ : : +- ^ ScanTransformer parquet (1)
+ : +- ^ InputIteratorTransformer (18)
+ : +- ShuffleQueryStage (16), Statistics(X)
+ : +- ColumnarExchange (15)
+ : +- VeloxResizeBatches (14)
+ : +- ^ ProjectExecTransformer (12)
+ : +- ^ FilterExecTransformer (11)
+ : +- ^ ScanTransformer parquet (10)
+ +- ^ InputIteratorTransformer (35)
+ +- ShuffleQueryStage (33), Statistics(X)
+ +- ColumnarExchange (32)
+ +- VeloxResizeBatches (31)
+ +- ^ ProjectExecTransformer (29)
+ +- ^ FilterExecTransformer (28)
+ +- ^ ScanTransformer parquet (27)
+- == Initial Plan ==
- TakeOrderedAndProject (66)
- +- HashAggregate (65)
- +- HashAggregate (64)
- +- Project (63)
- +- SortMergeJoin Inner (62)
- :- Sort (56)
- : +- Exchange (55)
- : +- Project (54)
- : +- SortMergeJoin Inner (53)
- : :- Sort (48)
- : : +- Exchange (47)
- : : +- Project (46)
- : : +- Filter (45)
- : : +- Scan parquet (44)
- : +- Sort (52)
- : +- Exchange (51)
- : +- Filter (50)
- : +- Scan parquet (49)
- +- Sort (61)
- +- Exchange (60)
- +- Project (59)
- +- Filter (58)
- +- Scan parquet (57)
-
-
-(1) Scan parquet
+ TakeOrderedAndProject (65)
+ +- HashAggregate (64)
+ +- HashAggregate (63)
+ +- Project (62)
+ +- SortMergeJoin Inner (61)
+ :- Sort (55)
+ : +- Exchange (54)
+ : +- Project (53)
+ : +- SortMergeJoin Inner (52)
+ : :- Sort (47)
+ : : +- Exchange (46)
+ : : +- Project (45)
+ : : +- Filter (44)
+ : : +- Scan parquet (43)
+ : +- Sort (51)
+ : +- Exchange (50)
+ : +- Filter (49)
+ : +- Scan parquet (48)
+ +- Sort (60)
+ +- Exchange (59)
+ +- Project (58)
+ +- Filter (57)
+ +- Scan parquet (56)
+
+
+(1) ScanTransformer parquet
Output [2]: [c_custkey#X, c_mktsegment#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -98,7 +97,7 @@ Input [1]: [c_custkey#X]
(9) InputIteratorTransformer
Input [1]: [c_custkey#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -167,7 +166,7 @@ Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X]
(26) InputIteratorTransformer
Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X]
-(27) Scan parquet
+(27) ScanTransformer parquet
Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -217,143 +216,136 @@ Input [6]: [o_orderkey#X, o_orderdate#X, o_shippriority#X, l_orderkey#X, l_exten
(38) RegularHashAggregateExecTransformer
Input [6]: [o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X, _pre_X#X]
Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X]
-Functions [1]: [partial_sum(_pre_X#X)]
-Aggregate Attributes [2]: [sum#X, isEmpty#X]
-Results [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X]
-
-(39) RegularHashAggregateExecTransformer
-Input [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X]
-Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X]
-Functions [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))]
+Functions [1]: [sum(_pre_X#X)]
Aggregate Attributes [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))#X]
Results [4]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum((l_extendedprice#X * (1 - l_discount#X)))#X]
-(40) ProjectExecTransformer
+(39) ProjectExecTransformer
Output [4]: [l_orderkey#X, sum((l_extendedprice#X * (1 - l_discount#X)))#X AS revenue#X, o_orderdate#X, o_shippriority#X]
Input [4]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum((l_extendedprice#X * (1 - l_discount#X)))#X]
-(41) WholeStageCodegenTransformer (X)
+(40) WholeStageCodegenTransformer (X)
Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X]
Arguments: false
-(42) TakeOrderedAndProjectExecTransformer
+(41) TakeOrderedAndProjectExecTransformer
Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X]
Arguments: X, [revenue#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X], 0
-(43) VeloxColumnarToRow
+(42) VeloxColumnarToRow
Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X]
-(44) Scan parquet
+(43) Scan parquet
Output [2]: [c_custkey#X, c_mktsegment#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(c_mktsegment), EqualTo(c_mktsegment,BUILDING), IsNotNull(c_custkey)]
ReadSchema: struct
-(45) Filter
+(44) Filter
Input [2]: [c_custkey#X, c_mktsegment#X]
Condition : ((isnotnull(c_mktsegment#X) AND (c_mktsegment#X = BUILDING)) AND isnotnull(c_custkey#X))
-(46) Project
+(45) Project
Output [1]: [c_custkey#X]
Input [2]: [c_custkey#X, c_mktsegment#X]
-(47) Exchange
+(46) Exchange
Input [1]: [c_custkey#X]
Arguments: hashpartitioning(c_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(48) Sort
+(47) Sort
Input [1]: [c_custkey#X]
Arguments: [c_custkey#X ASC NULLS FIRST], false, 0
-(49) Scan parquet
+(48) Scan parquet
Output [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(o_orderdate), LessThan(o_orderdate,1995-03-15), IsNotNull(o_custkey), IsNotNull(o_orderkey)]
ReadSchema: struct
-(50) Filter
+(49) Filter
Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X]
Condition : (((isnotnull(o_orderdate#X) AND (o_orderdate#X < 1995-03-15)) AND isnotnull(o_custkey#X)) AND isnotnull(o_orderkey#X))
-(51) Exchange
+(50) Exchange
Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X]
Arguments: hashpartitioning(o_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(52) Sort
+(51) Sort
Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X]
Arguments: [o_custkey#X ASC NULLS FIRST], false, 0
-(53) SortMergeJoin
+(52) SortMergeJoin
Left keys [1]: [c_custkey#X]
Right keys [1]: [o_custkey#X]
Join type: Inner
Join condition: None
-(54) Project
+(53) Project
Output [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X]
Input [5]: [c_custkey#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X]
-(55) Exchange
+(54) Exchange
Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X]
Arguments: hashpartitioning(o_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(56) Sort
+(55) Sort
Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X]
Arguments: [o_orderkey#X ASC NULLS FIRST], false, 0
-(57) Scan parquet
+(56) Scan parquet
Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
PushedFilters: [IsNotNull(l_shipdate), GreaterThan(l_shipdate,1995-03-15), IsNotNull(l_orderkey)]
ReadSchema: struct
-(58) Filter
+(57) Filter
Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Condition : ((isnotnull(l_shipdate#X) AND (l_shipdate#X > 1995-03-15)) AND isnotnull(l_orderkey#X))
-(59) Project
+(58) Project
Output [3]: [l_orderkey#X, l_extendedprice#X, l_discount#X]
Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
-(60) Exchange
+(59) Exchange
Input [3]: [l_orderkey#X, l_extendedprice#X, l_discount#X]
Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X]
-(61) Sort
+(60) Sort
Input [3]: [l_orderkey#X, l_extendedprice#X, l_discount#X]
Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0
-(62) SortMergeJoin
+(61) SortMergeJoin
Left keys [1]: [o_orderkey#X]
Right keys [1]: [l_orderkey#X]
Join type: Inner
Join condition: None
-(63) Project
+(62) Project
Output [5]: [o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X]
Input [6]: [o_orderkey#X, o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X]
-(64) HashAggregate
+(63) HashAggregate
Input [5]: [o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X]
Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X]
Functions [1]: [partial_sum((l_extendedprice#X * (1 - l_discount#X)))]
Aggregate Attributes [2]: [sum#X, isEmpty#X]
Results [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X]
-(65) HashAggregate
+(64) HashAggregate
Input [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X]
Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X]
Functions [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))]
Aggregate Attributes [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))#X]
Results [4]: [l_orderkey#X, sum((l_extendedprice#X * (1 - l_discount#X)))#X AS revenue#X, o_orderdate#X, o_shippriority#X]
-(66) TakeOrderedAndProject
+(65) TakeOrderedAndProject
Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X]
Arguments: X, [revenue#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X]
-(67) AdaptiveSparkPlan
+(66) AdaptiveSparkPlan
Output [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X]
Arguments: isFinalPlan=true
\ No newline at end of file
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 286881fb99b1..71a8f6974ed1 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
@@ -22,14 +22,14 @@ AdaptiveSparkPlan (56)
: +- VeloxResizeBatches (5)
: +- ^ ProjectExecTransformer (3)
: +- ^ FilterExecTransformer (2)
- : +- ^ Scan parquet (1)
+ : +- ^ ScanTransformer parquet (1)
+- ^ InputIteratorTransformer (18)
+- ShuffleQueryStage (16), Statistics(X)
+- ColumnarExchange (15)
+- VeloxResizeBatches (14)
+- ^ ProjectExecTransformer (12)
+- ^ FilterExecTransformer (11)
- +- ^ Scan parquet (10)
+ +- ^ ScanTransformer parquet (10)
+- == Initial Plan ==
Sort (55)
+- Exchange (54)
@@ -50,7 +50,7 @@ AdaptiveSparkPlan (56)
+- Scan parquet (44)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -87,7 +87,7 @@ Input [2]: [o_orderkey#X, o_orderpriority#X]
(9) InputIteratorTransformer
Input [2]: [o_orderkey#X, o_orderpriority#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [3]: [l_orderkey#X, l_commitdate#X, l_receiptdate#X]
Batched: true
Location: InMemoryFileIndex [*]
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 0a9af460526c..da9339abd628 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
@@ -46,42 +46,42 @@ AdaptiveSparkPlan (156)
: : : : : +- VeloxResizeBatches (5)
: : : : : +- ^ ProjectExecTransformer (3)
: : : : : +- ^ FilterExecTransformer (2)
- : : : : : +- ^ Scan parquet (1)
+ : : : : : +- ^ ScanTransformer parquet (1)
: : : : +- ^ InputIteratorTransformer (18)
: : : : +- ShuffleQueryStage (16), Statistics(X)
: : : : +- ColumnarExchange (15)
: : : : +- VeloxResizeBatches (14)
: : : : +- ^ ProjectExecTransformer (12)
: : : : +- ^ FilterExecTransformer (11)
- : : : : +- ^ Scan parquet (10)
+ : : : : +- ^ ScanTransformer parquet (10)
: : : +- ^ InputIteratorTransformer (35)
: : : +- ShuffleQueryStage (33), Statistics(X)
: : : +- ColumnarExchange (32)
: : : +- VeloxResizeBatches (31)
: : : +- ^ ProjectExecTransformer (29)
: : : +- ^ FilterExecTransformer (28)
- : : : +- ^ Scan parquet (27)
+ : : : +- ^ ScanTransformer parquet (27)
: : +- ^ InputIteratorTransformer (52)
: : +- ShuffleQueryStage (50), Statistics(X)
: : +- ColumnarExchange (49)
: : +- VeloxResizeBatches (48)
: : +- ^ ProjectExecTransformer (46)
: : +- ^ FilterExecTransformer (45)
- : : +- ^ Scan parquet (44)
+ : : +- ^ ScanTransformer parquet (44)
: +- ^ InputIteratorTransformer (69)
: +- ShuffleQueryStage (67), Statistics(X)
: +- ColumnarExchange (66)
: +- VeloxResizeBatches (65)
: +- ^ ProjectExecTransformer (63)
: +- ^ FilterExecTransformer (62)
- : +- ^ Scan parquet (61)
+ : +- ^ ScanTransformer parquet (61)
+- ^ InputIteratorTransformer (86)
+- ShuffleQueryStage (84), Statistics(X)
+- ColumnarExchange (83)
+- VeloxResizeBatches (82)
+- ^ ProjectExecTransformer (80)
+- ^ FilterExecTransformer (79)
- +- ^ Scan parquet (78)
+ +- ^ ScanTransformer parquet (78)
+- == Initial Plan ==
Sort (155)
+- Exchange (154)
@@ -134,7 +134,7 @@ AdaptiveSparkPlan (156)
+- Scan parquet (144)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [c_custkey#X, c_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -171,7 +171,7 @@ Input [2]: [c_custkey#X, c_nationkey#X]
(9) InputIteratorTransformer
Input [2]: [c_custkey#X, c_nationkey#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -240,7 +240,7 @@ Input [2]: [c_nationkey#X, o_orderkey#X]
(26) InputIteratorTransformer
Input [2]: [c_nationkey#X, o_orderkey#X]
-(27) Scan parquet
+(27) ScanTransformer parquet
Output [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -309,7 +309,7 @@ Input [4]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
(43) InputIteratorTransformer
Input [4]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
-(44) Scan parquet
+(44) ScanTransformer parquet
Output [2]: [s_suppkey#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -378,7 +378,7 @@ Input [3]: [l_extendedprice#X, l_discount#X, s_nationkey#X]
(60) InputIteratorTransformer
Input [3]: [l_extendedprice#X, l_discount#X, s_nationkey#X]
-(61) Scan parquet
+(61) ScanTransformer parquet
Output [3]: [n_nationkey#X, n_name#X, n_regionkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -447,7 +447,7 @@ Input [4]: [l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X]
(77) InputIteratorTransformer
Input [4]: [l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X]
-(78) Scan parquet
+(78) ScanTransformer parquet
Output [2]: [r_regionkey#X, r_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 e9dc68456e55..319c457c1b4f 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
@@ -10,7 +10,7 @@ AdaptiveSparkPlan (20)
+- ^ FlushableHashAggregateExecTransformer (4)
+- ^ ProjectExecTransformer (3)
+- ^ FilterExecTransformer (2)
- +- ^ Scan parquet (1)
+ +- ^ ScanTransformer parquet (1)
+- == Initial Plan ==
HashAggregate (19)
+- Exchange (18)
@@ -20,7 +20,7 @@ AdaptiveSparkPlan (20)
+- Scan parquet (14)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [4]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
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 aa69dbcd11f7..a0894489f0a5 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
@@ -46,35 +46,35 @@ AdaptiveSparkPlan (149)
: : : : : +- VeloxResizeBatches (5)
: : : : : +- ^ ProjectExecTransformer (3)
: : : : : +- ^ FilterExecTransformer (2)
- : : : : : +- ^ Scan parquet (1)
+ : : : : : +- ^ ScanTransformer parquet (1)
: : : : +- ^ InputIteratorTransformer (18)
: : : : +- ShuffleQueryStage (16), Statistics(X)
: : : : +- ColumnarExchange (15)
: : : : +- VeloxResizeBatches (14)
: : : : +- ^ ProjectExecTransformer (12)
: : : : +- ^ FilterExecTransformer (11)
- : : : : +- ^ Scan parquet (10)
+ : : : : +- ^ ScanTransformer parquet (10)
: : : +- ^ InputIteratorTransformer (35)
: : : +- ShuffleQueryStage (33), Statistics(X)
: : : +- ColumnarExchange (32)
: : : +- VeloxResizeBatches (31)
: : : +- ^ ProjectExecTransformer (29)
: : : +- ^ FilterExecTransformer (28)
- : : : +- ^ Scan parquet (27)
+ : : : +- ^ ScanTransformer parquet (27)
: : +- ^ InputIteratorTransformer (52)
: : +- ShuffleQueryStage (50), Statistics(X)
: : +- ColumnarExchange (49)
: : +- VeloxResizeBatches (48)
: : +- ^ ProjectExecTransformer (46)
: : +- ^ FilterExecTransformer (45)
- : : +- ^ Scan parquet (44)
+ : : +- ^ ScanTransformer parquet (44)
: +- ^ InputIteratorTransformer (69)
: +- ShuffleQueryStage (67), Statistics(X)
: +- ColumnarExchange (66)
: +- VeloxResizeBatches (65)
: +- ^ ProjectExecTransformer (63)
: +- ^ FilterExecTransformer (62)
- : +- ^ Scan parquet (61)
+ : +- ^ ScanTransformer parquet (61)
+- ^ InputIteratorTransformer (81)
+- ShuffleQueryStage (79), Statistics(X)
+- ReusedExchange (78)
@@ -128,7 +128,7 @@ AdaptiveSparkPlan (149)
+- Scan parquet (138)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [s_suppkey#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -165,7 +165,7 @@ Input [2]: [s_suppkey#X, s_nationkey#X]
(9) InputIteratorTransformer
Input [2]: [s_suppkey#X, s_nationkey#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [5]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -234,7 +234,7 @@ Input [5]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_ship
(26) InputIteratorTransformer
Input [5]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X]
-(27) Scan parquet
+(27) ScanTransformer parquet
Output [2]: [o_orderkey#X, o_custkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -303,7 +303,7 @@ Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_cust
(43) InputIteratorTransformer
Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X]
-(44) Scan parquet
+(44) ScanTransformer parquet
Output [2]: [c_custkey#X, c_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -372,7 +372,7 @@ Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nati
(60) InputIteratorTransformer
Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X]
-(61) Scan parquet
+(61) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 05f9c2c90e5d..8ce3620f30d5 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
@@ -59,56 +59,56 @@ AdaptiveSparkPlan (207)
: : : : : : : +- VeloxResizeBatches (5)
: : : : : : : +- ^ ProjectExecTransformer (3)
: : : : : : : +- ^ FilterExecTransformer (2)
- : : : : : : : +- ^ Scan parquet (1)
+ : : : : : : : +- ^ ScanTransformer parquet (1)
: : : : : : +- ^ InputIteratorTransformer (18)
: : : : : : +- ShuffleQueryStage (16), Statistics(X)
: : : : : : +- ColumnarExchange (15)
: : : : : : +- VeloxResizeBatches (14)
: : : : : : +- ^ ProjectExecTransformer (12)
: : : : : : +- ^ FilterExecTransformer (11)
- : : : : : : +- ^ Scan parquet (10)
+ : : : : : : +- ^ ScanTransformer parquet (10)
: : : : : +- ^ InputIteratorTransformer (35)
: : : : : +- ShuffleQueryStage (33), Statistics(X)
: : : : : +- ColumnarExchange (32)
: : : : : +- VeloxResizeBatches (31)
: : : : : +- ^ ProjectExecTransformer (29)
: : : : : +- ^ FilterExecTransformer (28)
- : : : : : +- ^ Scan parquet (27)
+ : : : : : +- ^ ScanTransformer parquet (27)
: : : : +- ^ InputIteratorTransformer (52)
: : : : +- ShuffleQueryStage (50), Statistics(X)
: : : : +- ColumnarExchange (49)
: : : : +- VeloxResizeBatches (48)
: : : : +- ^ ProjectExecTransformer (46)
: : : : +- ^ FilterExecTransformer (45)
- : : : : +- ^ Scan parquet (44)
+ : : : : +- ^ ScanTransformer parquet (44)
: : : +- ^ InputIteratorTransformer (69)
: : : +- ShuffleQueryStage (67), Statistics(X)
: : : +- ColumnarExchange (66)
: : : +- VeloxResizeBatches (65)
: : : +- ^ ProjectExecTransformer (63)
: : : +- ^ FilterExecTransformer (62)
- : : : +- ^ Scan parquet (61)
+ : : : +- ^ ScanTransformer parquet (61)
: : +- ^ InputIteratorTransformer (86)
: : +- ShuffleQueryStage (84), Statistics(X)
: : +- ColumnarExchange (83)
: : +- VeloxResizeBatches (82)
: : +- ^ ProjectExecTransformer (80)
: : +- ^ FilterExecTransformer (79)
- : : +- ^ Scan parquet (78)
+ : : +- ^ ScanTransformer parquet (78)
: +- ^ InputIteratorTransformer (103)
: +- ShuffleQueryStage (101), Statistics(X)
: +- ColumnarExchange (100)
: +- VeloxResizeBatches (99)
: +- ^ ProjectExecTransformer (97)
: +- ^ FilterExecTransformer (96)
- : +- ^ Scan parquet (95)
+ : +- ^ ScanTransformer parquet (95)
+- ^ InputIteratorTransformer (120)
+- ShuffleQueryStage (118), Statistics(X)
+- ColumnarExchange (117)
+- VeloxResizeBatches (116)
+- ^ ProjectExecTransformer (114)
+- ^ FilterExecTransformer (113)
- +- ^ Scan parquet (112)
+ +- ^ ScanTransformer parquet (112)
+- == Initial Plan ==
Sort (206)
+- Exchange (205)
@@ -177,7 +177,7 @@ AdaptiveSparkPlan (207)
+- Scan parquet (195)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [p_partkey#X, p_type#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -214,7 +214,7 @@ Input [1]: [p_partkey#X]
(9) InputIteratorTransformer
Input [1]: [p_partkey#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [5]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -283,7 +283,7 @@ Input [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
(26) InputIteratorTransformer
Input [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X]
-(27) Scan parquet
+(27) ScanTransformer parquet
Output [2]: [s_suppkey#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -352,7 +352,7 @@ Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X]
(43) InputIteratorTransformer
Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X]
-(44) Scan parquet
+(44) ScanTransformer parquet
Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -421,7 +421,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_order
(60) InputIteratorTransformer
Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X]
-(61) Scan parquet
+(61) ScanTransformer parquet
Output [2]: [c_custkey#X, c_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -490,7 +490,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nat
(77) InputIteratorTransformer
Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X]
-(78) Scan parquet
+(78) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_regionkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -559,7 +559,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_reg
(94) InputIteratorTransformer
Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X]
-(95) Scan parquet
+(95) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -628,7 +628,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_nam
(111) InputIteratorTransformer
Input [5]: [l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X]
-(112) Scan parquet
+(112) ScanTransformer parquet
Output [2]: [r_regionkey#X, r_name#X]
Batched: true
Location: InMemoryFileIndex [*]
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 e298d2a75196..4f537e91e132 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
@@ -46,42 +46,42 @@ AdaptiveSparkPlan (155)
: : : : : +- VeloxResizeBatches (5)
: : : : : +- ^ ProjectExecTransformer (3)
: : : : : +- ^ FilterExecTransformer (2)
- : : : : : +- ^ Scan parquet (1)
+ : : : : : +- ^ ScanTransformer parquet (1)
: : : : +- ^ InputIteratorTransformer (18)
: : : : +- ShuffleQueryStage (16), Statistics(X)
: : : : +- ColumnarExchange (15)
: : : : +- VeloxResizeBatches (14)
: : : : +- ^ ProjectExecTransformer (12)
: : : : +- ^ FilterExecTransformer (11)
- : : : : +- ^ Scan parquet (10)
+ : : : : +- ^ ScanTransformer parquet (10)
: : : +- ^ InputIteratorTransformer (35)
: : : +- ShuffleQueryStage (33), Statistics(X)
: : : +- ColumnarExchange (32)
: : : +- VeloxResizeBatches (31)
: : : +- ^ ProjectExecTransformer (29)
: : : +- ^ FilterExecTransformer (28)
- : : : +- ^ Scan parquet (27)
+ : : : +- ^ ScanTransformer parquet (27)
: : +- ^ InputIteratorTransformer (52)
: : +- ShuffleQueryStage (50), Statistics(X)
: : +- ColumnarExchange (49)
: : +- VeloxResizeBatches (48)
: : +- ^ ProjectExecTransformer (46)
: : +- ^ FilterExecTransformer (45)
- : : +- ^ Scan parquet (44)
+ : : +- ^ ScanTransformer parquet (44)
: +- ^ InputIteratorTransformer (69)
: +- ShuffleQueryStage (67), Statistics(X)
: +- ColumnarExchange (66)
: +- VeloxResizeBatches (65)
: +- ^ ProjectExecTransformer (63)
: +- ^ FilterExecTransformer (62)
- : +- ^ Scan parquet (61)
+ : +- ^ ScanTransformer parquet (61)
+- ^ InputIteratorTransformer (86)
+- ShuffleQueryStage (84), Statistics(X)
+- ColumnarExchange (83)
+- VeloxResizeBatches (82)
+- ^ ProjectExecTransformer (80)
+- ^ FilterExecTransformer (79)
- +- ^ Scan parquet (78)
+ +- ^ ScanTransformer parquet (78)
+- == Initial Plan ==
Sort (154)
+- Exchange (153)
@@ -133,7 +133,7 @@ AdaptiveSparkPlan (155)
+- Scan parquet (144)
-(1) Scan parquet
+(1) ScanTransformer parquet
Output [2]: [p_partkey#X, p_name#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -170,7 +170,7 @@ Input [1]: [p_partkey#X]
(9) InputIteratorTransformer
Input [1]: [p_partkey#X]
-(10) Scan parquet
+(10) ScanTransformer parquet
Output [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -239,7 +239,7 @@ Input [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpric
(26) InputIteratorTransformer
Input [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X]
-(27) Scan parquet
+(27) ScanTransformer parquet
Output [2]: [s_suppkey#X, s_nationkey#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -308,7 +308,7 @@ Input [7]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpric
(43) InputIteratorTransformer
Input [7]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X]
-(44) Scan parquet
+(44) ScanTransformer parquet
Output [3]: [ps_partkey#X, ps_suppkey#X, ps_supplycost#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -377,7 +377,7 @@ Input [6]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_natio
(60) InputIteratorTransformer
Input [6]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X]
-(61) Scan parquet
+(61) ScanTransformer parquet
Output [2]: [o_orderkey#X, o_orderdate#X]
Batched: true
Location: InMemoryFileIndex [*]
@@ -446,7 +446,7 @@ Input [6]: [l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_sup
(77) InputIteratorTransformer
Input [6]: [l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X]
-(78) Scan parquet
+(78) ScanTransformer parquet
Output [2]: [n_nationkey#X, n_name#X]
Batched: true
Location: InMemoryFileIndex [*]
diff --git a/backends-velox/src/test/scala/org/apache/gluten/benchmarks/NativeBenchmarkPlanGenerator.scala b/backends-velox/src/test/scala/org/apache/gluten/benchmarks/NativeBenchmarkPlanGenerator.scala
index 57fbca17447a..1e378d16f14b 100644
--- a/backends-velox/src/test/scala/org/apache/gluten/benchmarks/NativeBenchmarkPlanGenerator.scala
+++ b/backends-velox/src/test/scala/org/apache/gluten/benchmarks/NativeBenchmarkPlanGenerator.scala
@@ -19,18 +19,14 @@ package org.apache.gluten.benchmarks
import org.apache.gluten.GlutenConfig
import org.apache.gluten.execution.{VeloxWholeStageTransformerSuite, WholeStageTransformer}
-import org.apache.spark.sql.DataFrame
-import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, ShuffleQueryStageExec}
+import org.apache.spark.SparkConf
+import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec
import org.apache.spark.sql.internal.SQLConf
import org.apache.commons.io.FileUtils
import org.scalatest.Tag
import java.io.File
-import java.nio.charset.StandardCharsets
-import java.nio.file.{Files, Paths}
-
-import scala.collection.JavaConverters._
object GenerateExample extends Tag("org.apache.gluten.tags.GenerateExample")
@@ -50,6 +46,11 @@ class NativeBenchmarkPlanGenerator extends VeloxWholeStageTransformerSuite {
createTPCHNotNullTables()
}
+ override protected def sparkConf: SparkConf = {
+ super.sparkConf
+ .set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager")
+ }
+
test("Test plan json non-empty - AQE off") {
withSQLConf(
SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false",
@@ -67,7 +68,6 @@ class NativeBenchmarkPlanGenerator extends VeloxWholeStageTransformerSuite {
planJson = lastStageTransformer.get.asInstanceOf[WholeStageTransformer].substraitPlanJson
assert(planJson.nonEmpty)
}
- spark.sparkContext.setLogLevel(logLevel)
}
test("Test plan json non-empty - AQE on") {
@@ -88,70 +88,42 @@ class NativeBenchmarkPlanGenerator extends VeloxWholeStageTransformerSuite {
val planJson = lastStageTransformer.get.asInstanceOf[WholeStageTransformer].substraitPlanJson
assert(planJson.nonEmpty)
}
- spark.sparkContext.setLogLevel(logLevel)
}
test("generate example", GenerateExample) {
- import testImplicits._
withSQLConf(
SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1",
SQLConf.SHUFFLE_PARTITIONS.key -> "2",
- GlutenConfig.CACHE_WHOLE_STAGE_TRANSFORMER_CONTEXT.key -> "true"
+ GlutenConfig.BENCHMARK_SAVE_DIR.key -> generatedPlanDir,
+ GlutenConfig.BENCHMARK_TASK_STAGEID.key -> "12",
+ GlutenConfig.BENCHMARK_TASK_PARTITIONID.key -> "0"
) {
logWarning(s"Generating inputs for micro benchmark to $generatedPlanDir")
- val q4_lineitem = spark
- .sql(s"""
- |select l_orderkey from lineitem where l_commitdate < l_receiptdate
- |""".stripMargin)
- val q4_orders = spark
- .sql(s"""
- |select o_orderkey, o_orderpriority
- | from orders
- | where o_orderdate >= '1993-07-01' and o_orderdate < '1993-10-01'
- |""".stripMargin)
- q4_lineitem
- .createOrReplaceTempView("q4_lineitem")
- q4_orders
- .createOrReplaceTempView("q4_orders")
-
- q4_lineitem
- .repartition(1, 'l_orderkey)
- .write
- .format(outputFileFormat)
- .save(generatedPlanDir + "/example_lineitem")
- q4_orders
- .repartition(1, 'o_orderkey)
- .write
- .format(outputFileFormat)
- .save(generatedPlanDir + "/example_orders")
-
- val df =
- spark.sql("""
- |select * from q4_orders left semi join q4_lineitem on l_orderkey = o_orderkey
- |""".stripMargin)
- generateSubstraitJson(df, "example.json")
+ spark
+ .sql("""
+ |select /*+ REPARTITION(1) */
+ | o_orderpriority,
+ | count(*) as order_count
+ |from
+ | orders
+ |where
+ | o_orderdate >= date '1993-07-01'
+ | and o_orderdate < date '1993-07-01' + interval '3' month
+ | and exists (
+ | select /*+ REPARTITION(1) */
+ | *
+ | from
+ | lineitem
+ | where
+ | l_orderkey = o_orderkey
+ | and l_commitdate < l_receiptdate
+ | )
+ |group by
+ | o_orderpriority
+ |order by
+ | o_orderpriority
+ |""".stripMargin)
+ .foreach(_ => ())
}
- spark.sparkContext.setLogLevel(logLevel)
- }
-
- def generateSubstraitJson(df: DataFrame, file: String): Unit = {
- val executedPlan = df.queryExecution.executedPlan
- executedPlan.execute()
- val finalPlan =
- executedPlan match {
- case aqe: AdaptiveSparkPlanExec =>
- aqe.executedPlan match {
- case s: ShuffleQueryStageExec => s.shuffle.child
- case other => other
- }
- case plan => plan
- }
- val lastStageTransformer = finalPlan.find(_.isInstanceOf[WholeStageTransformer])
- assert(lastStageTransformer.nonEmpty)
- val plan =
- lastStageTransformer.get.asInstanceOf[WholeStageTransformer].substraitPlanJson.split('\n')
-
- val exampleJsonFile = Paths.get(generatedPlanDir, file)
- Files.write(exampleJsonFile, plan.toList.asJava, StandardCharsets.UTF_8)
}
}
diff --git a/backends-velox/src/test/scala/org/apache/gluten/execution/FallbackSuite.scala b/backends-velox/src/test/scala/org/apache/gluten/execution/FallbackSuite.scala
index d8b0ee8981e2..0f94b8648e71 100644
--- a/backends-velox/src/test/scala/org/apache/gluten/execution/FallbackSuite.scala
+++ b/backends-velox/src/test/scala/org/apache/gluten/execution/FallbackSuite.scala
@@ -17,7 +17,6 @@
package org.apache.gluten.execution
import org.apache.gluten.GlutenConfig
-import org.apache.gluten.extension.GlutenPlan
import org.apache.spark.SparkConf
import org.apache.spark.sql.execution.{ColumnarBroadcastExchangeExec, ColumnarShuffleExchangeExec, SparkPlan}
@@ -253,7 +252,7 @@ class FallbackSuite extends VeloxWholeStageTransformerSuite with AdaptiveSparkPl
test("fallback with smj") {
val sql = "SELECT /*+ SHUFFLE_MERGE(tmp1) */ * FROM tmp1 join tmp2 on tmp1.c1 = tmp2.c1"
withSQLConf(
- GlutenConfig.COLUMNAR_FPRCE_SHUFFLED_HASH_JOIN_ENABLED.key -> "true",
+ GlutenConfig.COLUMNAR_FORCE_SHUFFLED_HASH_JOIN_ENABLED.key -> "true",
GlutenConfig.COLUMNAR_SHUFFLED_HASH_JOIN_ENABLED.key -> "false") {
runQueryAndCompare(sql) {
df =>
@@ -262,7 +261,7 @@ class FallbackSuite extends VeloxWholeStageTransformerSuite with AdaptiveSparkPl
}
}
withSQLConf(
- GlutenConfig.COLUMNAR_FPRCE_SHUFFLED_HASH_JOIN_ENABLED.key -> "false",
+ GlutenConfig.COLUMNAR_FORCE_SHUFFLED_HASH_JOIN_ENABLED.key -> "false",
GlutenConfig.COLUMNAR_SORTMERGEJOIN_ENABLED.key -> "false") {
runQueryAndCompare(sql) {
df =>
diff --git a/backends-velox/src/test/scala/org/apache/gluten/execution/MiscOperatorSuite.scala b/backends-velox/src/test/scala/org/apache/gluten/execution/MiscOperatorSuite.scala
index 7e9b48e96ca9..8063a5d12207 100644
--- a/backends-velox/src/test/scala/org/apache/gluten/execution/MiscOperatorSuite.scala
+++ b/backends-velox/src/test/scala/org/apache/gluten/execution/MiscOperatorSuite.scala
@@ -208,6 +208,13 @@ class MiscOperatorSuite extends VeloxWholeStageTransformerSuite with AdaptiveSpa
"select l_orderkey from lineitem " +
"where l_partkey in (1552, 674) or l_partkey in (1552) and l_orderkey > 1") { _ => }
checkLengthAndPlan(df, 73)
+
+ runQueryAndCompare(
+ "select count(1) from lineitem " +
+ "where (l_shipmode in ('TRUCK', 'MAIL') or l_shipmode in ('AIR', 'FOB')) " +
+ "and l_shipmode in ('RAIL','SHIP')") {
+ checkGlutenOperatorMatch[FileSourceScanExecTransformer]
+ }
}
test("in_not") {
@@ -505,6 +512,13 @@ class MiscOperatorSuite extends VeloxWholeStageTransformerSuite with AdaptiveSpa
checkGlutenOperatorMatch[WindowExecTransformer]
}
}
+
+ // Foldable input of nth_value is not supported.
+ runQueryAndCompare(
+ "select l_suppkey, l_orderkey, nth_value(1, 2) over" +
+ " (partition by l_suppkey order by l_orderkey) from lineitem ") {
+ checkSparkOperatorMatch[WindowExec]
+ }
}
}
@@ -523,11 +537,37 @@ class MiscOperatorSuite extends VeloxWholeStageTransformerSuite with AdaptiveSpa
|""".stripMargin) {
df =>
{
- getExecutedPlan(df).exists(plan => plan.find(_.isInstanceOf[ColumnarUnionExec]).isDefined)
+ assert(
+ getExecutedPlan(df).exists(
+ plan => plan.find(_.isInstanceOf[ColumnarUnionExec]).isDefined))
}
}
}
+ test("union_all two tables with known partitioning") {
+ withSQLConf(GlutenConfig.NATIVE_UNION_ENABLED.key -> "true") {
+ compareDfResultsAgainstVanillaSpark(
+ () => {
+ val df1 = spark.sql("select l_orderkey as orderkey from lineitem")
+ val df2 = spark.sql("select o_orderkey as orderkey from orders")
+ df1.repartition(5).union(df2.repartition(5))
+ },
+ compareResult = true,
+ checkGlutenOperatorMatch[UnionExecTransformer]
+ )
+
+ compareDfResultsAgainstVanillaSpark(
+ () => {
+ val df1 = spark.sql("select l_orderkey as orderkey from lineitem")
+ val df2 = spark.sql("select o_orderkey as orderkey from orders")
+ df1.repartition(5).union(df2.repartition(6))
+ },
+ compareResult = true,
+ checkGlutenOperatorMatch[ColumnarUnionExec]
+ )
+ }
+ }
+
test("union_all three tables") {
runQueryAndCompare("""
|select count(orderkey) from (
@@ -566,7 +606,7 @@ class MiscOperatorSuite extends VeloxWholeStageTransformerSuite with AdaptiveSpa
| select * from lineitem limit 10
|) where l_suppkey != 0 limit 100;
|""".stripMargin) {
- checkGlutenOperatorMatch[LimitTransformer]
+ checkGlutenOperatorMatch[LimitExecTransformer]
}
}
@@ -845,7 +885,7 @@ class MiscOperatorSuite extends VeloxWholeStageTransformerSuite with AdaptiveSpa
withSQLConf(
GlutenConfig.COLUMNAR_PREFER_STREAMING_AGGREGATE.key -> "true",
- GlutenConfig.COLUMNAR_FPRCE_SHUFFLED_HASH_JOIN_ENABLED.key -> "false",
+ GlutenConfig.COLUMNAR_FORCE_SHUFFLED_HASH_JOIN_ENABLED.key -> "false",
SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1"
) {
val query =
diff --git a/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxAggregateFunctionsSuite.scala b/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxAggregateFunctionsSuite.scala
index 84d1fe8db919..cf0bf272bc5b 100644
--- a/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxAggregateFunctionsSuite.scala
+++ b/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxAggregateFunctionsSuite.scala
@@ -49,6 +49,7 @@ abstract class VeloxAggregateFunctionsSuite extends VeloxWholeStageTransformerSu
.set("spark.unsafe.exceptionOnMemoryLeak", "true")
.set("spark.sql.autoBroadcastJoinThreshold", "-1")
.set("spark.sql.sources.useV1SourceList", "avro")
+ .set("spark.gluten.sql.mergeTwoPhasesAggregate.enabled", "false")
}
test("count") {
diff --git a/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxMetricsSuite.scala b/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxMetricsSuite.scala
index 4a251449845f..e41f00821c27 100644
--- a/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxMetricsSuite.scala
+++ b/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxMetricsSuite.scala
@@ -22,8 +22,9 @@ import org.apache.gluten.sql.shims.SparkShimLoader
import org.apache.spark.SparkConf
import org.apache.spark.scheduler.{SparkListener, SparkListenerStageCompleted}
import org.apache.spark.sql.TestUtils
-import org.apache.spark.sql.execution.CommandResultExec
-import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper
+import org.apache.spark.sql.execution.{ColumnarInputAdapter, CommandResultExec, InputIteratorTransformer}
+import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanHelper, BroadcastQueryStageExec}
+import org.apache.spark.sql.execution.exchange.BroadcastExchangeLike
import org.apache.spark.sql.internal.SQLConf
class VeloxMetricsSuite extends VeloxWholeStageTransformerSuite with AdaptiveSparkPlanHelper {
@@ -62,7 +63,7 @@ class VeloxMetricsSuite extends VeloxWholeStageTransformerSuite with AdaptiveSpa
test("test sort merge join metrics") {
withSQLConf(
- GlutenConfig.COLUMNAR_FPRCE_SHUFFLED_HASH_JOIN_ENABLED.key -> "false",
+ GlutenConfig.COLUMNAR_FORCE_SHUFFLED_HASH_JOIN_ENABLED.key -> "false",
SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") {
// without preproject
runQueryAndCompare(
@@ -227,4 +228,39 @@ class VeloxMetricsSuite extends VeloxWholeStageTransformerSuite with AdaptiveSpa
assert(inputRecords == (partTableRecords + itemTableRecords))
}
+
+ test("Metrics for input iterator of broadcast exchange") {
+ createTPCHNotNullTables()
+ val partTableRecords = spark.sql("select * from part").count()
+
+ // Repartition to make sure we have multiple tasks executing the join.
+ spark
+ .sql("select * from lineitem")
+ .repartition(2)
+ .createOrReplaceTempView("lineitem")
+
+ Seq("true", "false").foreach {
+ adaptiveEnabled =>
+ withSQLConf("spark.sql.adaptive.enabled" -> adaptiveEnabled) {
+ val sqlStr =
+ """
+ |select /*+ BROADCAST(part) */ * from part join lineitem
+ |on l_partkey = p_partkey
+ |""".stripMargin
+
+ runQueryAndCompare(sqlStr) {
+ df =>
+ val inputIterator = find(df.queryExecution.executedPlan) {
+ case InputIteratorTransformer(ColumnarInputAdapter(child)) =>
+ child.isInstanceOf[BroadcastQueryStageExec] || child
+ .isInstanceOf[BroadcastExchangeLike]
+ case _ => false
+ }
+ assert(inputIterator.isDefined)
+ val metrics = inputIterator.get.metrics
+ assert(metrics("numOutputRows").value == partTableRecords)
+ }
+ }
+ }
+ }
}
diff --git a/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxOrcDataTypeValidationSuite.scala b/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxOrcDataTypeValidationSuite.scala
index 3bf49d33c511..6ac59ba4fa6b 100644
--- a/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxOrcDataTypeValidationSuite.scala
+++ b/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxOrcDataTypeValidationSuite.scala
@@ -255,7 +255,10 @@ class VeloxOrcDataTypeValidationSuite extends VeloxWholeStageTransformerSuite {
|""".stripMargin) {
df =>
{
- assert(getExecutedPlan(df).exists(plan => plan.isInstanceOf[ColumnarUnionExec]))
+ assert(
+ getExecutedPlan(df).exists(
+ plan =>
+ plan.isInstanceOf[ColumnarUnionExec] || plan.isInstanceOf[UnionExecTransformer]))
}
}
@@ -265,7 +268,7 @@ class VeloxOrcDataTypeValidationSuite extends VeloxWholeStageTransformerSuite {
| select date, int from type1 limit 100
|) where int != 0 limit 10;
|""".stripMargin) {
- checkGlutenOperatorMatch[LimitTransformer]
+ checkGlutenOperatorMatch[LimitExecTransformer]
}
// Validation: Window.
diff --git a/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxParquetDataTypeValidationSuite.scala b/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxParquetDataTypeValidationSuite.scala
index 8b6cc63c954d..cb5614f39669 100644
--- a/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxParquetDataTypeValidationSuite.scala
+++ b/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxParquetDataTypeValidationSuite.scala
@@ -254,7 +254,10 @@ class VeloxParquetDataTypeValidationSuite extends VeloxWholeStageTransformerSuit
|""".stripMargin) {
df =>
{
- assert(getExecutedPlan(df).exists(plan => plan.isInstanceOf[ColumnarUnionExec]))
+ assert(
+ getExecutedPlan(df).exists(
+ plan =>
+ plan.isInstanceOf[ColumnarUnionExec] || plan.isInstanceOf[UnionExecTransformer]))
}
}
@@ -264,7 +267,7 @@ class VeloxParquetDataTypeValidationSuite extends VeloxWholeStageTransformerSuit
| select date, int from type1 limit 100
|) where int != 0 limit 10;
|""".stripMargin) {
- checkGlutenOperatorMatch[LimitTransformer]
+ checkGlutenOperatorMatch[LimitExecTransformer]
}
// Validation: Window.
diff --git a/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxScanSuite.scala b/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxScanSuite.scala
index a50bffa3edcd..a4f16ecc3c8f 100644
--- a/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxScanSuite.scala
+++ b/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxScanSuite.scala
@@ -18,11 +18,13 @@ package org.apache.gluten.execution
import org.apache.gluten.GlutenConfig
import org.apache.gluten.backendsapi.velox.VeloxBackendSettings
+import org.apache.gluten.benchmarks.RandomParquetDataGenerator
import org.apache.gluten.utils.VeloxFileSystemValidationJniWrapper
import org.apache.spark.SparkConf
import org.apache.spark.sql.catalyst.expressions.GreaterThan
import org.apache.spark.sql.execution.ScalarSubquery
+import org.apache.spark.sql.types._
class VeloxScanSuite extends VeloxWholeStageTransformerSuite {
protected val rootPath: String = getClass.getResource("/").getPath
@@ -114,4 +116,38 @@ class VeloxScanSuite extends VeloxWholeStageTransformerSuite {
!VeloxFileSystemValidationJniWrapper.allSupportedByRegisteredFileSystems(
Array("file:/test_path/", "unsupported://test_path")))
}
+
+ test("scan with filter on decimal/timestamp/binary field") {
+ withTempView("t") {
+ withTempDir {
+ dir =>
+ val path = dir.getAbsolutePath
+ val schema = StructType(
+ Array(
+ StructField("short_decimal_field", DecimalType(5, 2), nullable = true),
+ StructField("long_decimal_field", DecimalType(32, 8), nullable = true),
+ StructField("binary_field", BinaryType, nullable = true),
+ StructField("timestamp_field", TimestampType, nullable = true)
+ ))
+ RandomParquetDataGenerator(0).generateRandomData(spark, schema, 10, Some(path))
+ spark.catalog.createTable("t", path, "parquet")
+
+ runQueryAndCompare(
+ """select * from t where long_decimal_field = 3.14""".stripMargin
+ )(checkGlutenOperatorMatch[FileSourceScanExecTransformer])
+
+ runQueryAndCompare(
+ """select * from t where short_decimal_field = 3.14""".stripMargin
+ )(checkGlutenOperatorMatch[FileSourceScanExecTransformer])
+
+ runQueryAndCompare(
+ """select * from t where binary_field = '3.14'""".stripMargin
+ )(checkGlutenOperatorMatch[FileSourceScanExecTransformer])
+
+ runQueryAndCompare(
+ """select * from t where timestamp_field = current_timestamp()""".stripMargin
+ )(checkGlutenOperatorMatch[FileSourceScanExecTransformer])
+ }
+ }
+ }
}
diff --git a/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxStringFunctionsSuite.scala b/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxStringFunctionsSuite.scala
index 1bef5f991fac..f1e5a42bc0a3 100644
--- a/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxStringFunctionsSuite.scala
+++ b/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxStringFunctionsSuite.scala
@@ -268,7 +268,7 @@ class VeloxStringFunctionsSuite extends VeloxWholeStageTransformerSuite {
s"from $LINEITEM_TABLE limit $LENGTH")(checkGlutenOperatorMatch[ProjectExecTransformer])
}
- ignore("locate") {
+ test("locate") {
runQueryAndCompare(
s"select l_orderkey, locate(l_comment, 'a', 1) " +
s"from $LINEITEM_TABLE limit $LENGTH")(checkGlutenOperatorMatch[ProjectExecTransformer])
diff --git a/backends-velox/src/test/scala/org/apache/gluten/expression/VeloxUdfSuite.scala b/backends-velox/src/test/scala/org/apache/gluten/expression/VeloxUdfSuite.scala
index f85103deb847..61ba927cd457 100644
--- a/backends-velox/src/test/scala/org/apache/gluten/expression/VeloxUdfSuite.scala
+++ b/backends-velox/src/test/scala/org/apache/gluten/expression/VeloxUdfSuite.scala
@@ -16,11 +16,13 @@
*/
package org.apache.gluten.expression
+import org.apache.gluten.execution.ProjectExecTransformer
import org.apache.gluten.tags.{SkipTestTags, UDFTest}
import org.apache.spark.SparkConf
import org.apache.spark.sql.{GlutenQueryTest, Row, SparkSession}
import org.apache.spark.sql.catalyst.plans.SQLHelper
+import org.apache.spark.sql.execution.ProjectExec
import org.apache.spark.sql.expression.UDFResolver
import java.nio.file.Paths
@@ -158,16 +160,24 @@ abstract class VeloxUdfSuite extends GlutenQueryTest with SQLHelper {
|AS 'org.apache.spark.sql.hive.execution.UDFStringString'
|""".stripMargin)
- val nativeResultWithImplicitConversion =
- spark.sql(s"""SELECT hive_string_string(col1, 'a') FROM $tbl""").collect()
- val nativeResult =
- spark.sql(s"""SELECT hive_string_string(col2, 'a') FROM $tbl""").collect()
+ val offloadWithImplicitConversionDF =
+ spark.sql(s"""SELECT hive_string_string(col1, 'a') FROM $tbl""")
+ checkGlutenOperatorMatch[ProjectExecTransformer](offloadWithImplicitConversionDF)
+ val offloadWithImplicitConversionResult = offloadWithImplicitConversionDF.collect()
+
+ val offloadDF =
+ spark.sql(s"""SELECT hive_string_string(col2, 'a') FROM $tbl""")
+ checkGlutenOperatorMatch[ProjectExecTransformer](offloadDF)
+ val offloadResult = offloadWithImplicitConversionDF.collect()
+
// Unregister native hive udf to fallback.
UDFResolver.UDFNames.remove("org.apache.spark.sql.hive.execution.UDFStringString")
- val fallbackResult =
- spark.sql(s"""SELECT hive_string_string(col2, 'a') FROM $tbl""").collect()
- assert(nativeResultWithImplicitConversion.sameElements(fallbackResult))
- assert(nativeResult.sameElements(fallbackResult))
+ val fallbackDF =
+ spark.sql(s"""SELECT hive_string_string(col2, 'a') FROM $tbl""")
+ checkSparkOperatorMatch[ProjectExec](fallbackDF)
+ val fallbackResult = fallbackDF.collect()
+ assert(offloadWithImplicitConversionResult.sameElements(fallbackResult))
+ assert(offloadResult.sameElements(fallbackResult))
// Add an unimplemented udf to the map to test fallback of registered native hive udf.
UDFResolver.UDFNames.add("org.apache.spark.sql.hive.execution.UDFIntegerToString")
@@ -176,6 +186,7 @@ abstract class VeloxUdfSuite extends GlutenQueryTest with SQLHelper {
|AS 'org.apache.spark.sql.hive.execution.UDFIntegerToString'
|""".stripMargin)
val df = spark.sql(s"""select hive_int_to_string(col1) from $tbl""")
+ checkSparkOperatorMatch[ProjectExec](df)
checkAnswer(df, Seq(Row("1"), Row("2"), Row("3")))
} finally {
spark.sql(s"DROP TABLE IF EXISTS $tbl")
diff --git a/backends-velox/src/test/scala/org/apache/gluten/extension/columnar/transition/VeloxTransitionSuite.scala b/backends-velox/src/test/scala/org/apache/gluten/extension/columnar/transition/VeloxTransitionSuite.scala
index d12faae0f73d..e14ffd43d82d 100644
--- a/backends-velox/src/test/scala/org/apache/gluten/extension/columnar/transition/VeloxTransitionSuite.scala
+++ b/backends-velox/src/test/scala/org/apache/gluten/extension/columnar/transition/VeloxTransitionSuite.scala
@@ -33,37 +33,37 @@ class VeloxTransitionSuite extends SharedSparkSession {
test("Vanilla C2R - outputs row") {
val in = BatchLeaf(VanillaBatch)
- val out = Transitions.insertTransitions(in, outputsColumnar = false)
+ val out = BackendTransitions.insert(in, outputsColumnar = false)
assert(out == ColumnarToRowExec(BatchLeaf(VanillaBatch)))
}
test("Vanilla C2R - requires row input") {
val in = RowUnary(BatchLeaf(VanillaBatch))
- val out = Transitions.insertTransitions(in, outputsColumnar = false)
+ val out = BackendTransitions.insert(in, outputsColumnar = false)
assert(out == RowUnary(ColumnarToRowExec(BatchLeaf(VanillaBatch))))
}
test("Vanilla R2C - requires vanilla input") {
val in = BatchUnary(VanillaBatch, RowLeaf())
- val out = Transitions.insertTransitions(in, outputsColumnar = false)
+ val out = BackendTransitions.insert(in, outputsColumnar = false)
assert(out == ColumnarToRowExec(BatchUnary(VanillaBatch, RowToColumnarExec(RowLeaf()))))
}
test("ArrowNative C2R - outputs row") {
val in = BatchLeaf(ArrowNativeBatch)
- val out = Transitions.insertTransitions(in, outputsColumnar = false)
+ val out = BackendTransitions.insert(in, outputsColumnar = false)
assert(out == ColumnarToRowExec(LoadArrowDataExec(BatchLeaf(ArrowNativeBatch))))
}
test("ArrowNative C2R - requires row input") {
val in = RowUnary(BatchLeaf(ArrowNativeBatch))
- val out = Transitions.insertTransitions(in, outputsColumnar = false)
+ val out = BackendTransitions.insert(in, outputsColumnar = false)
assert(out == RowUnary(ColumnarToRowExec(LoadArrowDataExec(BatchLeaf(ArrowNativeBatch)))))
}
test("ArrowNative R2C - requires Arrow input") {
val in = BatchUnary(ArrowNativeBatch, RowLeaf())
- val out = Transitions.insertTransitions(in, outputsColumnar = false)
+ val out = BackendTransitions.insert(in, outputsColumnar = false)
assert(
out == ColumnarToRowExec(
LoadArrowDataExec(BatchUnary(ArrowNativeBatch, RowToVeloxColumnarExec(RowLeaf())))))
@@ -71,7 +71,7 @@ class VeloxTransitionSuite extends SharedSparkSession {
test("ArrowNative-to-Velox C2C") {
val in = BatchUnary(VeloxBatch, BatchLeaf(ArrowNativeBatch))
- val out = Transitions.insertTransitions(in, outputsColumnar = false)
+ val out = BackendTransitions.insert(in, outputsColumnar = false)
// No explicit transition needed for ArrowNative-to-Velox.
// FIXME: Add explicit transitions.
// See https://github.com/apache/incubator-gluten/issues/7313.
@@ -82,7 +82,7 @@ class VeloxTransitionSuite extends SharedSparkSession {
test("Velox-to-ArrowNative C2C") {
val in = BatchUnary(ArrowNativeBatch, BatchLeaf(VeloxBatch))
- val out = Transitions.insertTransitions(in, outputsColumnar = false)
+ val out = BackendTransitions.insert(in, outputsColumnar = false)
assert(
out == ColumnarToRowExec(
LoadArrowDataExec(BatchUnary(ArrowNativeBatch, BatchLeaf(VeloxBatch)))))
@@ -90,7 +90,7 @@ class VeloxTransitionSuite extends SharedSparkSession {
test("Vanilla-to-ArrowNative C2C") {
val in = BatchUnary(ArrowNativeBatch, BatchLeaf(VanillaBatch))
- val out = Transitions.insertTransitions(in, outputsColumnar = false)
+ val out = BackendTransitions.insert(in, outputsColumnar = false)
assert(
out == ColumnarToRowExec(
LoadArrowDataExec(BatchUnary(
@@ -100,7 +100,7 @@ class VeloxTransitionSuite extends SharedSparkSession {
test("ArrowNative-to-Vanilla C2C") {
val in = BatchUnary(VanillaBatch, BatchLeaf(ArrowNativeBatch))
- val out = Transitions.insertTransitions(in, outputsColumnar = false)
+ val out = BackendTransitions.insert(in, outputsColumnar = false)
assert(
out == ColumnarToRowExec(
BatchUnary(VanillaBatch, LoadArrowDataExec(BatchLeaf(ArrowNativeBatch)))))
@@ -108,19 +108,19 @@ class VeloxTransitionSuite extends SharedSparkSession {
test("ArrowJava C2R - outputs row") {
val in = BatchLeaf(ArrowJavaBatch)
- val out = Transitions.insertTransitions(in, outputsColumnar = false)
+ val out = BackendTransitions.insert(in, outputsColumnar = false)
assert(out == ColumnarToRowExec(BatchLeaf(ArrowJavaBatch)))
}
test("ArrowJava C2R - requires row input") {
val in = RowUnary(BatchLeaf(ArrowJavaBatch))
- val out = Transitions.insertTransitions(in, outputsColumnar = false)
+ val out = BackendTransitions.insert(in, outputsColumnar = false)
assert(out == RowUnary(ColumnarToRowExec(BatchLeaf(ArrowJavaBatch))))
}
test("ArrowJava R2C - requires Arrow input") {
val in = BatchUnary(ArrowJavaBatch, RowLeaf())
- val out = Transitions.insertTransitions(in, outputsColumnar = false)
+ val out = BackendTransitions.insert(in, outputsColumnar = false)
assert(
out == ColumnarToRowExec(
BatchUnary(ArrowJavaBatch, LoadArrowDataExec(RowToVeloxColumnarExec(RowLeaf())))))
@@ -128,7 +128,7 @@ class VeloxTransitionSuite extends SharedSparkSession {
test("ArrowJava-to-Velox C2C") {
val in = BatchUnary(VeloxBatch, BatchLeaf(ArrowJavaBatch))
- val out = Transitions.insertTransitions(in, outputsColumnar = false)
+ val out = BackendTransitions.insert(in, outputsColumnar = false)
assert(
out == VeloxColumnarToRowExec(
BatchUnary(
@@ -138,7 +138,7 @@ class VeloxTransitionSuite extends SharedSparkSession {
test("Velox-to-ArrowJava C2C") {
val in = BatchUnary(ArrowJavaBatch, BatchLeaf(VeloxBatch))
- val out = Transitions.insertTransitions(in, outputsColumnar = false)
+ val out = BackendTransitions.insert(in, outputsColumnar = false)
assert(
out == ColumnarToRowExec(
BatchUnary(ArrowJavaBatch, LoadArrowDataExec(BatchLeaf(VeloxBatch)))))
@@ -146,7 +146,7 @@ class VeloxTransitionSuite extends SharedSparkSession {
test("Vanilla-to-ArrowJava C2C") {
val in = BatchUnary(ArrowJavaBatch, BatchLeaf(VanillaBatch))
- val out = Transitions.insertTransitions(in, outputsColumnar = false)
+ val out = BackendTransitions.insert(in, outputsColumnar = false)
assert(
out == ColumnarToRowExec(
BatchUnary(
@@ -156,37 +156,37 @@ class VeloxTransitionSuite extends SharedSparkSession {
test("ArrowJava-to-Vanilla C2C") {
val in = BatchUnary(VanillaBatch, BatchLeaf(ArrowJavaBatch))
- val out = Transitions.insertTransitions(in, outputsColumnar = false)
+ val out = BackendTransitions.insert(in, outputsColumnar = false)
assert(out == ColumnarToRowExec(BatchUnary(VanillaBatch, BatchLeaf(ArrowJavaBatch))))
}
test("Velox C2R - outputs row") {
val in = BatchLeaf(VeloxBatch)
- val out = Transitions.insertTransitions(in, outputsColumnar = false)
+ val out = BackendTransitions.insert(in, outputsColumnar = false)
assert(out == VeloxColumnarToRowExec(BatchLeaf(VeloxBatch)))
}
test("Velox C2R - requires row input") {
val in = RowUnary(BatchLeaf(VeloxBatch))
- val out = Transitions.insertTransitions(in, outputsColumnar = false)
+ val out = BackendTransitions.insert(in, outputsColumnar = false)
assert(out == RowUnary(VeloxColumnarToRowExec(BatchLeaf(VeloxBatch))))
}
test("Velox R2C - outputs Velox") {
val in = RowLeaf()
- val out = Transitions.insertTransitions(in, outputsColumnar = true)
+ val out = BackendTransitions.insert(in, outputsColumnar = true)
assert(out == RowToVeloxColumnarExec(RowLeaf()))
}
test("Velox R2C - requires Velox input") {
val in = BatchUnary(VeloxBatch, RowLeaf())
- val out = Transitions.insertTransitions(in, outputsColumnar = false)
+ val out = BackendTransitions.insert(in, outputsColumnar = false)
assert(out == VeloxColumnarToRowExec(BatchUnary(VeloxBatch, RowToVeloxColumnarExec(RowLeaf()))))
}
test("Vanilla-to-Velox C2C") {
val in = BatchUnary(VeloxBatch, BatchLeaf(VanillaBatch))
- val out = Transitions.insertTransitions(in, outputsColumnar = false)
+ val out = BackendTransitions.insert(in, outputsColumnar = false)
assert(
out == VeloxColumnarToRowExec(
BatchUnary(VeloxBatch, RowToVeloxColumnarExec(ColumnarToRowExec(BatchLeaf(VanillaBatch))))))
@@ -194,7 +194,7 @@ class VeloxTransitionSuite extends SharedSparkSession {
test("Velox-to-Vanilla C2C") {
val in = BatchUnary(VanillaBatch, BatchLeaf(VeloxBatch))
- val out = Transitions.insertTransitions(in, outputsColumnar = false)
+ val out = BackendTransitions.insert(in, outputsColumnar = false)
assert(
out == ColumnarToRowExec(BatchUnary(VanillaBatch, LoadArrowDataExec(BatchLeaf(VeloxBatch)))))
}
diff --git a/backends-velox/src/test/scala/org/apache/gluten/planner/VeloxRasSuite.scala b/backends-velox/src/test/scala/org/apache/gluten/planner/VeloxRasSuite.scala
index 38a6832c4e3f..885b549d0def 100644
--- a/backends-velox/src/test/scala/org/apache/gluten/planner/VeloxRasSuite.scala
+++ b/backends-velox/src/test/scala/org/apache/gluten/planner/VeloxRasSuite.scala
@@ -17,9 +17,11 @@
package org.apache.gluten.planner
import org.apache.gluten.GlutenConfig
+import org.apache.gluten.extension.columnar.enumerated.EnumeratedTransform
+import org.apache.gluten.extension.columnar.enumerated.planner.GlutenOptimization
+import org.apache.gluten.extension.columnar.enumerated.planner.cost.{LegacyCoster, LongCostModel}
+import org.apache.gluten.extension.columnar.enumerated.planner.property.Conv
import org.apache.gluten.extension.columnar.transition.ConventionReq
-import org.apache.gluten.planner.cost.GlutenCostModel
-import org.apache.gluten.planner.property.Conv
import org.apache.gluten.ras.{Cost, CostModel, Ras}
import org.apache.gluten.ras.RasSuiteBase._
import org.apache.gluten.ras.path.RasPath
@@ -140,23 +142,30 @@ class VeloxRasSuite extends SharedSparkSession {
object VeloxRasSuite {
def newRas(): Ras[SparkPlan] = {
- GlutenOptimization
- .builder()
- .costModel(GlutenCostModel.find())
- .addRules(List())
- .create()
- .asInstanceOf[Ras[SparkPlan]]
+ newRas(Nil)
}
- def newRas(RasRules: Seq[RasRule[SparkPlan]]): Ras[SparkPlan] = {
+ def newRas(rasRules: Seq[RasRule[SparkPlan]]): Ras[SparkPlan] = {
GlutenOptimization
.builder()
- .costModel(GlutenCostModel.find())
- .addRules(RasRules)
+ .costModel(sessionCostModel())
+ .addRules(rasRules)
.create()
.asInstanceOf[Ras[SparkPlan]]
}
+ private def legacyCostModel(): CostModel[SparkPlan] = {
+ val registry = LongCostModel.registry()
+ val coster = LegacyCoster
+ registry.register(coster)
+ registry.get(coster.kind())
+ }
+
+ private def sessionCostModel(): CostModel[SparkPlan] = {
+ val transform = EnumeratedTransform.static()
+ transform.costModel
+ }
+
val TRIVIAL_SCHEMA: Seq[AttributeReference] = List(AttributeReference("value", StringType)())
val EMPTY_SCHEMA: Seq[AttributeReference] = List.empty
@@ -191,7 +200,7 @@ object VeloxRasSuite {
}
class UserCostModel1 extends CostModel[SparkPlan] {
- private val base = GlutenCostModel.legacy()
+ private val base = legacyCostModel()
override def costOf(node: SparkPlan): Cost = node match {
case _: RowUnary => base.makeInfCost()
case other => base.costOf(other)
@@ -201,7 +210,7 @@ object VeloxRasSuite {
}
class UserCostModel2 extends CostModel[SparkPlan] {
- private val base = GlutenCostModel.legacy()
+ private val base = legacyCostModel()
override def costOf(node: SparkPlan): Cost = node match {
case _: ColumnarUnary => base.makeInfCost()
case other => base.costOf(other)
diff --git a/backends-velox/src/test/scala/org/apache/spark/sql/execution/VeloxParquetWriteForHiveSuite.scala b/backends-velox/src/test/scala/org/apache/spark/sql/execution/VeloxParquetWriteForHiveSuite.scala
index 412548de9c44..5932f4e5a741 100644
--- a/backends-velox/src/test/scala/org/apache/spark/sql/execution/VeloxParquetWriteForHiveSuite.scala
+++ b/backends-velox/src/test/scala/org/apache/spark/sql/execution/VeloxParquetWriteForHiveSuite.scala
@@ -27,6 +27,11 @@ import org.apache.spark.sql.hive.HiveUtils
import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf}
import org.apache.spark.sql.test.SQLTestUtils
import org.apache.spark.sql.util.QueryExecutionListener
+import org.apache.spark.util.Utils
+
+import org.apache.hadoop.fs.Path
+import org.apache.parquet.hadoop.ParquetFileReader
+import org.apache.parquet.hadoop.util.HadoopInputFile
class VeloxParquetWriteForHiveSuite extends GlutenQueryTest with SQLTestUtils {
private var _spark: SparkSession = _
@@ -176,4 +181,45 @@ class VeloxParquetWriteForHiveSuite extends GlutenQueryTest with SQLTestUtils {
checkAnswer(sql("SELECT * FROM t"), Row(1))
}
}
+
+ test("native writer support CreateHiveTableAsSelectCommand") {
+ withTable("t") {
+ withSQLConf("spark.sql.hive.convertMetastoreParquet" -> "false") {
+ checkNativeWrite("CREATE TABLE t STORED AS PARQUET AS SELECT 1 as c", checkNative = true)
+ }
+ checkAnswer(spark.table("t"), Row(1))
+ }
+ }
+
+ test("native writer should respect table properties") {
+ Seq(true, false).foreach {
+ enableNativeWrite =>
+ withSQLConf("spark.gluten.sql.native.writer.enabled" -> enableNativeWrite.toString) {
+ withTable("t") {
+ withSQLConf(
+ "spark.sql.hive.convertMetastoreParquet" -> "false",
+ "spark.sql.parquet.compression.codec" -> "gzip") {
+ checkNativeWrite(
+ "CREATE TABLE t STORED AS PARQUET TBLPROPERTIES ('parquet.compression'='zstd') " +
+ "AS SELECT 1 as c",
+ checkNative = enableNativeWrite)
+ val tableDir = new Path(s"${conf.getConf(StaticSQLConf.WAREHOUSE_PATH)}/t")
+ val configuration = spark.sessionState.newHadoopConf()
+ val files = tableDir
+ .getFileSystem(configuration)
+ .listStatus(tableDir)
+ .filterNot(_.getPath.getName.startsWith("\\."))
+ assert(files.nonEmpty)
+ val in = HadoopInputFile.fromStatus(files.head, spark.sessionState.newHadoopConf())
+ Utils.tryWithResource(ParquetFileReader.open(in)) {
+ reader =>
+ val column = reader.getFooter.getBlocks.get(0).getColumns.get(0)
+ // native writer and vanilla spark hive writer should be consistent
+ "zstd".equalsIgnoreCase(column.getCodec.toString)
+ }
+ }
+ }
+ }
+ }
+ }
}
diff --git a/backends-velox/src/test/scala/org/apache/spark/sql/execution/VeloxParquetWriteSuite.scala b/backends-velox/src/test/scala/org/apache/spark/sql/execution/VeloxParquetWriteSuite.scala
index 4c76c753b90e..d19d279fbb2c 100644
--- a/backends-velox/src/test/scala/org/apache/spark/sql/execution/VeloxParquetWriteSuite.scala
+++ b/backends-velox/src/test/scala/org/apache/spark/sql/execution/VeloxParquetWriteSuite.scala
@@ -31,6 +31,22 @@ class VeloxParquetWriteSuite extends VeloxWholeStageTransformerSuite {
override protected val resourcePath: String = "/tpch-data-parquet"
override protected val fileFormat: String = "parquet"
+ // The parquet compression codec extensions
+ private val parquetCompressionCodecExtensions = Map(
+ "none" -> "",
+ "uncompressed" -> "",
+ "snappy" -> ".snappy",
+ "gzip" -> ".gz",
+ "lzo" -> ".lzo",
+ "lz4" -> ".lz4",
+ "brotli" -> ".br",
+ "zstd" -> ".zstd"
+ )
+
+ private def getParquetFileExtension(codec: String): String = {
+ s"${parquetCompressionCodecExtensions(codec)}.parquet"
+ }
+
override def beforeAll(): Unit = {
super.beforeAll()
createTPCHNotNullTables()
@@ -49,8 +65,8 @@ class VeloxParquetWriteSuite extends VeloxWholeStageTransformerSuite {
spark.sql("select array(struct(1), null) as var1").write.mode("overwrite").save(path)
}
assert(
- testAppender.loggingEvents.exists(
- _.getMessage.toString.contains("Use Gluten parquet write for hive")) == false)
+ !testAppender.loggingEvents.exists(
+ _.getMessage.toString.contains("Use Gluten parquet write for hive")))
}
}
@@ -77,6 +93,7 @@ class VeloxParquetWriteSuite extends VeloxWholeStageTransformerSuite {
parquetFiles.forall {
file =>
val path = new Path(f.getCanonicalPath, file)
+ assert(file.endsWith(getParquetFileExtension(codec)))
val in = HadoopInputFile.fromPath(path, spark.sessionState.newHadoopConf())
Utils.tryWithResource(ParquetFileReader.open(in)) {
reader =>
diff --git a/backends-velox/src/test/scala/org/apache/spark/sql/execution/benchmark/StreamingAggregateBenchmark.scala b/backends-velox/src/test/scala/org/apache/spark/sql/execution/benchmark/StreamingAggregateBenchmark.scala
index 7f11fd043db4..77370d059caa 100644
--- a/backends-velox/src/test/scala/org/apache/spark/sql/execution/benchmark/StreamingAggregateBenchmark.scala
+++ b/backends-velox/src/test/scala/org/apache/spark/sql/execution/benchmark/StreamingAggregateBenchmark.scala
@@ -51,7 +51,7 @@ object StreamingAggregateBenchmark extends SqlBasedBenchmark {
withSQLConf(
GlutenConfig.COLUMNAR_PREFER_STREAMING_AGGREGATE.key -> "true",
SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1",
- GlutenConfig.COLUMNAR_FPRCE_SHUFFLED_HASH_JOIN_ENABLED.key -> "false"
+ GlutenConfig.COLUMNAR_FORCE_SHUFFLED_HASH_JOIN_ENABLED.key -> "false"
) {
spark.sql(query).noop()
}
@@ -62,7 +62,7 @@ object StreamingAggregateBenchmark extends SqlBasedBenchmark {
withSQLConf(
GlutenConfig.COLUMNAR_PREFER_STREAMING_AGGREGATE.key -> "false",
SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1",
- GlutenConfig.COLUMNAR_FPRCE_SHUFFLED_HASH_JOIN_ENABLED.key -> "false"
+ GlutenConfig.COLUMNAR_FORCE_SHUFFLED_HASH_JOIN_ENABLED.key -> "false"
) {
spark.sql(query).noop()
}
diff --git a/cpp-ch/clickhouse.version b/cpp-ch/clickhouse.version
index 445cd99068a1..7c94e253c7b7 100644
--- a/cpp-ch/clickhouse.version
+++ b/cpp-ch/clickhouse.version
@@ -1,3 +1,3 @@
CH_ORG=Kyligence
-CH_BRANCH=rebase_ch/20241101
-CH_COMMIT=7cd7bb8ece2
\ No newline at end of file
+CH_BRANCH=rebase_ch/20241205
+CH_COMMIT=7b8b1c0b554
diff --git a/cpp-ch/local-engine/AggregateFunctions/GroupLimitFunctions.cpp b/cpp-ch/local-engine/AggregateFunctions/GroupLimitFunctions.cpp
new file mode 100644
index 000000000000..67c9e8380aa3
--- /dev/null
+++ b/cpp-ch/local-engine/AggregateFunctions/GroupLimitFunctions.cpp
@@ -0,0 +1,296 @@
+/*
+ * 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
+#include
+#include
+#include
+#include
+#include
+#include
+#include
+#include
+#include
+#include
+#include
+#include
+#include
+#include
+#include
+#include
+#include
+#include
+#include
+#include
+
+namespace DB::ErrorCodes
+{
+extern const int LOGICAL_ERROR;
+extern const int BAD_ARGUMENTS;
+}
+
+namespace local_engine
+{
+
+struct SortOrderField
+{
+ size_t pos = 0;
+ Int8 direction = 0;
+ Int8 nulls_direction = 0;
+};
+using SortOrderFields = std::vector;
+
+struct RowNumGroupArraySortedData
+{
+public:
+ using Data = DB::Tuple;
+ std::vector values;
+
+ static bool compare(const Data & lhs, const Data & rhs, const SortOrderFields & sort_orders)
+ {
+ for (const auto & sort_order : sort_orders)
+ {
+ const auto & pos = sort_order.pos;
+ const auto & asc = sort_order.direction;
+ const auto & nulls_first = sort_order.nulls_direction;
+ bool l_is_null = lhs[pos].isNull();
+ bool r_is_null = rhs[pos].isNull();
+ if (l_is_null && r_is_null)
+ continue;
+ else if (l_is_null)
+ return nulls_first;
+ else if (r_is_null)
+ return !nulls_first;
+ else if (lhs[pos] < rhs[pos])
+ return asc;
+ else if (lhs[pos] > rhs[pos])
+ return !asc;
+ }
+ return false;
+ }
+
+ ALWAYS_INLINE void heapReplaceTop(const SortOrderFields & sort_orders)
+ {
+ size_t size = values.size();
+ if (size < 2)
+ return;
+ size_t child_index = 1;
+ if (size > 2 && compare(values[1], values[2], sort_orders))
+ ++child_index;
+
+ if (compare(values[child_index], values[0], sort_orders))
+ return;
+
+ size_t current_index = 0;
+ auto current = values[current_index];
+ do
+ {
+ values[current_index] = values[child_index];
+ current_index = child_index;
+
+ child_index = 2 * child_index + 1;
+
+ if (child_index >= size)
+ break;
+
+ if ((child_index + 1) < size && compare(values[child_index], values[child_index + 1], sort_orders))
+ ++child_index;
+ } while (!compare(values[child_index], current, sort_orders));
+
+ values[current_index] = current;
+ }
+
+ ALWAYS_INLINE void addElement(const Data && data, const SortOrderFields & sort_orders, size_t max_elements)
+ {
+ if (values.size() >= max_elements)
+ {
+ if (!compare(data, values[0], sort_orders))
+ return;
+ values[0] = data;
+ heapReplaceTop(sort_orders);
+ return;
+ }
+ values.emplace_back(std::move(data));
+ auto cmp = [&sort_orders](const Data & a, const Data & b) { return compare(a, b, sort_orders); };
+ std::push_heap(values.begin(), values.end(), cmp);
+ }
+
+ ALWAYS_INLINE void sortAndLimit(size_t max_elements, const SortOrderFields & sort_orders)
+ {
+ ::sort(values.begin(), values.end(), [&sort_orders](const Data & a, const Data & b) { return compare(a, b, sort_orders); });
+ if (values.size() > max_elements)
+ values.resize(max_elements);
+ }
+
+ ALWAYS_INLINE void insertResultInto(DB::IColumn & to, size_t max_elements, const SortOrderFields & sort_orders)
+ {
+ auto & result_array = assert_cast(to);
+ auto & result_array_offsets = result_array.getOffsets();
+
+ sortAndLimit(max_elements, sort_orders);
+
+ result_array_offsets.push_back(result_array_offsets.back() + values.size());
+
+ if (values.empty())
+ return;
+ auto & result_array_data = result_array.getData();
+ for (int i = 0, sz = static_cast(values.size()); i < sz; ++i)
+ {
+ auto & value = values[i];
+ value.push_back(i + 1);
+ result_array_data.insert(value);
+ }
+ }
+};
+
+static DB::DataTypePtr getRowNumReultDataType(DB::DataTypePtr data_type)
+{
+ const auto * tuple_type = typeid_cast(data_type.get());
+ if (!tuple_type)
+ throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Tuple type is expected, but got: {}", data_type->getName());
+ DB::DataTypes element_types = tuple_type->getElements();
+ std::vector element_names = tuple_type->getElementNames();
+ element_types.push_back(std::make_shared());
+ element_names.push_back("row_num");
+ auto nested_tuple_type = std::make_shared(element_types, element_names);
+ return std::make_shared(nested_tuple_type);
+}
+
+// usage: rowNumGroupArraySorted(1, "a asc nulls first, b desc nulls last")(tuple(a,b))
+class RowNumGroupArraySorted final : public DB::IAggregateFunctionDataHelper
+{
+public:
+ explicit RowNumGroupArraySorted(DB::DataTypePtr data_type, const DB::Array & parameters_)
+ : DB::IAggregateFunctionDataHelper(
+ {data_type}, parameters_, getRowNumReultDataType(data_type))
+ {
+ if (parameters_.size() != 2)
+ throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "{} needs two parameters: limit and order clause", getName());
+ const auto * tuple_type = typeid_cast(data_type.get());
+ if (!tuple_type)
+ throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Tuple type is expected, but got: {}", data_type->getName());
+
+ limit = parameters_[0].safeGet();
+
+ String order_by_clause = parameters_[1].safeGet();
+ sort_order_fields = parseSortOrderFields(order_by_clause);
+
+ serialization = data_type->getDefaultSerialization();
+ }
+
+ String getName() const override { return "rowNumGroupArraySorted"; }
+
+ void add(DB::AggregateDataPtr __restrict place, const DB::IColumn ** columns, size_t row_num, DB::Arena * /*arena*/) const override
+ {
+ auto & data = this->data(place);
+ DB::Tuple data_tuple = (*columns[0])[row_num].safeGet();
+ this->data(place).addElement(std::move(data_tuple), sort_order_fields, limit);
+ }
+
+ void merge(DB::AggregateDataPtr __restrict place, DB::ConstAggregateDataPtr rhs, DB::Arena * /*arena*/) const override
+ {
+ auto & rhs_values = this->data(rhs).values;
+ for (auto & rhs_element : rhs_values)
+ this->data(place).addElement(std::move(rhs_element), sort_order_fields, limit);
+ }
+
+ void serialize(DB::ConstAggregateDataPtr __restrict place, DB::WriteBuffer & buf, std::optional /* version */) const override
+ {
+ auto & values = this->data(place).values;
+ size_t size = values.size();
+ DB::writeVarUInt(size, buf);
+
+ for (const auto & value : values)
+ serialization->serializeBinary(value, buf, {});
+ }
+
+ void deserialize(
+ DB::AggregateDataPtr __restrict place, DB::ReadBuffer & buf, std::optional /* version */, DB::Arena *) const override
+ {
+ size_t size = 0;
+ DB::readVarUInt(size, buf);
+
+ auto & values = this->data(place).values;
+ values.reserve(size);
+ for (size_t i = 0; i < size; ++i)
+ {
+ DB::Field data;
+ serialization->deserializeBinary(data, buf, {});
+ values.emplace_back(data.safeGet());
+ }
+ }
+
+ void insertResultInto(DB::AggregateDataPtr __restrict place, DB::IColumn & to, DB::Arena * /*arena*/) const override
+ {
+ this->data(place).insertResultInto(to, limit, sort_order_fields);
+ }
+
+ bool allocatesMemoryInArena() const override { return true; }
+
+private:
+ size_t limit = 0;
+ SortOrderFields sort_order_fields;
+ DB::SerializationPtr serialization;
+
+ SortOrderFields parseSortOrderFields(const String & order_by_clause) const
+ {
+ DB::ParserOrderByExpressionList order_by_parser;
+ auto order_by_ast = DB::parseQuery(order_by_parser, order_by_clause, 1000, 1000, 1000);
+ SortOrderFields fields;
+ const auto expression_list_ast = assert_cast(order_by_ast.get());
+ const auto & tuple_element_names = assert_cast(argument_types[0].get())->getElementNames();
+ for (const auto & child : expression_list_ast->children)
+ {
+ const auto * order_by_element_ast = assert_cast(child.get());
+ const auto * ident_ast = assert_cast(order_by_element_ast->children[0].get());
+ const auto & ident_name = ident_ast->shortName();
+
+
+ SortOrderField field;
+ field.direction = order_by_element_ast->direction == 1;
+ field.nulls_direction
+ = field.direction ? order_by_element_ast->nulls_direction == -1 : order_by_element_ast->nulls_direction == 1;
+
+ auto name_pos = std::find(tuple_element_names.begin(), tuple_element_names.end(), ident_name);
+ if (name_pos == tuple_element_names.end())
+ {
+ throw DB::Exception(
+ DB::ErrorCodes::BAD_ARGUMENTS, "Not found column {} in tuple {}", ident_name, argument_types[0]->getName());
+ }
+ field.pos = std::distance(tuple_element_names.begin(), name_pos);
+
+ fields.push_back(field);
+ }
+ return fields;
+ }
+};
+
+
+DB::AggregateFunctionPtr createAggregateFunctionRowNumGroupArray(
+ const std::string & name, const DB::DataTypes & argument_types, const DB::Array & parameters, const DB::Settings *)
+{
+ if (argument_types.size() != 1 || !typeid_cast(argument_types[0].get()))
+ throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, " {} Nees only one tuple argument", name);
+ return std::make_shared(argument_types[0], parameters);
+}
+
+void registerAggregateFunctionRowNumGroup(DB::AggregateFunctionFactory & factory)
+{
+ DB::AggregateFunctionProperties properties = {.returns_default_when_only_null = false, .is_order_dependent = false};
+
+ factory.registerFunction("rowNumGroupArraySorted", {createAggregateFunctionRowNumGroupArray, properties});
+}
+}
diff --git a/cpp-ch/local-engine/CMakeLists.txt b/cpp-ch/local-engine/CMakeLists.txt
index 4392b55008bf..4b1c64363646 100644
--- a/cpp-ch/local-engine/CMakeLists.txt
+++ b/cpp-ch/local-engine/CMakeLists.txt
@@ -174,6 +174,26 @@ else()
set(LOCALENGINE_SHARED_LIB_NAME "libch.so")
endif()
+option(ENABLE_SEPARATE_SYMBOLS "support separate debug symbols from so" OFF)
+if(ENABLE_SEPARATE_SYMBOLS)
+ set(SYMBOL_OUTPUT_DIRECTORY ${CMAKE_CURRENT_BINARY_DIR}/debug_symbols)
+ file(MAKE_DIRECTORY ${SYMBOL_OUTPUT_DIRECTORY})
+ function(separate_symbols target)
+ add_custom_command(
+ TARGET ${target}
+ POST_BUILD
+ COMMAND ${CMAKE_OBJCOPY} --only-keep-debug $
+ ${SYMBOL_OUTPUT_DIRECTORY}/$.debug
+ COMMAND ${CMAKE_OBJCOPY} --strip-debug $
+ COMMAND
+ ${CMAKE_OBJCOPY}
+ --add-gnu-debuglink=${SYMBOL_OUTPUT_DIRECTORY}/$.debug
+ $
+ COMMENT "Separating debug symbols for target: ${target}")
+ endfunction()
+ separate_symbols(${LOCALENGINE_SHARED_LIB})
+endif()
+
add_custom_command(
OUTPUT ${LOCALENGINE_SHARED_LIB_NAME}
COMMAND ${CMAKE_COMMAND} -E rename $
diff --git a/cpp-ch/local-engine/Common/AggregateUtil.cpp b/cpp-ch/local-engine/Common/AggregateUtil.cpp
index 851dd2e7fe3d..36b26e94bd17 100644
--- a/cpp-ch/local-engine/Common/AggregateUtil.cpp
+++ b/cpp-ch/local-engine/Common/AggregateUtil.cpp
@@ -15,8 +15,11 @@
* limitations under the License.
*/
+#include "AggregateUtil.h"
+#include
#include
#include
+#include
#include
#include
#include
@@ -26,8 +29,29 @@ namespace DB
{
namespace ErrorCodes
{
- extern const int LOGICAL_ERROR;
- extern const int UNKNOWN_AGGREGATED_DATA_VARIANT;
+extern const int LOGICAL_ERROR;
+extern const int UNKNOWN_AGGREGATED_DATA_VARIANT;
+}
+
+namespace Setting
+{
+extern const SettingsDouble max_bytes_ratio_before_external_group_by;
+extern const SettingsUInt64 max_bytes_before_external_group_by;
+extern const SettingsBool optimize_group_by_constant_keys;
+extern const SettingsUInt64 min_free_disk_space_for_temporary_data;
+extern const SettingsMaxThreads max_threads;
+extern const SettingsBool empty_result_for_aggregation_by_empty_set;
+extern const SettingsUInt64 group_by_two_level_threshold_bytes;
+extern const SettingsOverflowModeGroupBy group_by_overflow_mode;
+extern const SettingsUInt64 max_rows_to_group_by;
+extern const SettingsBool enable_memory_bound_merging_of_aggregation_results;
+extern const SettingsUInt64 aggregation_in_order_max_block_bytes;
+extern const SettingsUInt64 group_by_two_level_threshold;
+extern const SettingsFloat min_hit_rate_to_use_consecutive_keys_optimization;
+extern const SettingsUInt64 max_block_size;
+extern const SettingsBool compile_aggregate_expressions;
+extern const SettingsUInt64 min_count_to_compile_aggregate_expression;
+extern const SettingsBool enable_software_prefetch_in_aggregation;
}
template
@@ -39,24 +63,23 @@ static Int32 extractMethodBucketsNum(Method & /*method*/)
Int32 GlutenAggregatorUtil::getBucketsNum(AggregatedDataVariants & data_variants)
{
if (!data_variants.isTwoLevel())
- {
return 0;
- }
-
+
Int32 buckets_num = 0;
#define M(NAME) \
- else if (data_variants.type == AggregatedDataVariants::Type::NAME) \
- buckets_num = extractMethodBucketsNum(*data_variants.NAME);
+ else if (data_variants.type == AggregatedDataVariants::Type::NAME) buckets_num = extractMethodBucketsNum(*data_variants.NAME);
- if (false) {} // NOLINT
+ if (false)
+ {
+ } // NOLINT
APPLY_FOR_VARIANTS_TWO_LEVEL(M)
#undef M
- else
- throw Exception(ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT, "Unknown aggregated data variant");
+ else throw Exception(ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT, "Unknown aggregated data variant");
return buckets_num;
}
-std::optional GlutenAggregatorUtil::safeConvertOneBucketToBlock(Aggregator & aggregator, AggregatedDataVariants & variants, Arena * arena, bool final, Int32 bucket)
+std::optional GlutenAggregatorUtil::safeConvertOneBucketToBlock(
+ Aggregator & aggregator, AggregatedDataVariants & variants, Arena * arena, bool final, Int32 bucket)
{
if (!variants.isTwoLevel())
return {};
@@ -65,7 +88,7 @@ std::optional