aboutsummaryrefslogtreecommitdiff
path: root/sql/core/src/main/scala/org
diff options
context:
space:
mode:
authorDavies Liu <davies@databricks.com>2016-03-09 12:04:29 -0800
committerDavies Liu <davies.liu@gmail.com>2016-03-09 12:04:29 -0800
commit3dc9ae2e158e5b51df6f799767946fe1d190156b (patch)
tree06d65a71e1612051ec62498f5175d7e556a89acf /sql/core/src/main/scala/org
parent0dd06485c4222a896c0d1ee6a04d30043de3626c (diff)
downloadspark-3dc9ae2e158e5b51df6f799767946fe1d190156b.tar.gz
spark-3dc9ae2e158e5b51df6f799767946fe1d190156b.tar.bz2
spark-3dc9ae2e158e5b51df6f799767946fe1d190156b.zip
[SPARK-13523] [SQL] Reuse exchanges in a query
## What changes were proposed in this pull request? It’s possible to have common parts in a query, for example, self join, it will be good to avoid the duplicated part to same CPUs and memory (Broadcast or cache). Exchange will materialize the underlying RDD by shuffle or collect, it’s a great point to check duplicates and reuse them. Duplicated exchanges means they generate exactly the same result inside a query. In order to find out the duplicated exchanges, we should be able to compare SparkPlan to check that they have same results or not. We already have that for LogicalPlan, so we should move that into QueryPlan to make it available for SparkPlan. Once we can find the duplicated exchanges, we should replace all of them with same SparkPlan object (could be wrapped by ReusedExchage for explain), then the plan tree become a DAG. Since all the planner only work with tree, so this rule should be the last one for the entire planning. After the rule, the plan will looks like: ``` WholeStageCodegen : +- Project [id#0L] : +- BroadcastHashJoin [id#0L], [id#2L], Inner, BuildRight, None : :- Project [id#0L] : : +- BroadcastHashJoin [id#0L], [id#1L], Inner, BuildRight, None : : :- Range 0, 1, 4, 1024, [id#0L] : : +- INPUT : +- INPUT :- BroadcastExchange HashedRelationBroadcastMode(true,List(id#1L),List(id#1L)) : +- WholeStageCodegen : : +- Range 0, 1, 4, 1024, [id#1L] +- ReusedExchange [id#2L], BroadcastExchange HashedRelationBroadcastMode(true,List(id#1L),List(id#1L)) ``` ![bjoin](https://cloud.githubusercontent.com/assets/40902/13414787/209e8c5c-df0a-11e5-8a0f-edff69d89e83.png) For three ways SortMergeJoin, ``` == Physical Plan == WholeStageCodegen : +- Project [id#0L] : +- SortMergeJoin [id#0L], [id#4L], None : :- INPUT : +- INPUT :- WholeStageCodegen : : +- Project [id#0L] : : +- SortMergeJoin [id#0L], [id#3L], None : : :- INPUT : : +- INPUT : :- WholeStageCodegen : : : +- Sort [id#0L ASC], false, 0 : : : +- INPUT : : +- Exchange hashpartitioning(id#0L, 200), None : : +- WholeStageCodegen : : : +- Range 0, 1, 4, 33554432, [id#0L] : +- WholeStageCodegen : : +- Sort [id#3L ASC], false, 0 : : +- INPUT : +- ReusedExchange [id#3L], Exchange hashpartitioning(id#0L, 200), None +- WholeStageCodegen : +- Sort [id#4L ASC], false, 0 : +- INPUT +- ReusedExchange [id#4L], Exchange hashpartitioning(id#0L, 200), None ``` ![sjoin](https://cloud.githubusercontent.com/assets/40902/13414790/27aea61c-df0a-11e5-8cbf-fbc985c31d95.png) If the same ShuffleExchange or BroadcastExchange, execute()/executeBroadcast() will be called by different parents, they should cached the RDD/Broadcast, return the same one for all the parents. ## How was this patch tested? Added some unit tests for this. Had done some manual tests on TPCDS query Q59 and Q64, we can see some exchanges are re-used (this requires a change in PhysicalRDD to for sameResult, is be done in #11514 ). Author: Davies Liu <davies@databricks.com> Closes #11403 from davies/dedup.
Diffstat (limited to 'sql/core/src/main/scala/org')
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala22
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala4
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala3
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchange.scala10
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/Exchange.scala92
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchange.scala29
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala15
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SparkPlanGraph.scala20
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala6
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala6
10 files changed, 179 insertions, 28 deletions
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala
index 9019e5dfd6..247f55da1d 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala
@@ -18,6 +18,7 @@
package org.apache.spark.sql.execution
import org.apache.spark.annotation.DeveloperApi
+import org.apache.spark.sql.execution.exchange.ReusedExchange
import org.apache.spark.sql.execution.metric.SQLMetricInfo
import org.apache.spark.util.Utils
@@ -31,13 +32,28 @@ class SparkPlanInfo(
val simpleString: String,
val children: Seq[SparkPlanInfo],
val metadata: Map[String, String],
- val metrics: Seq[SQLMetricInfo])
+ val metrics: Seq[SQLMetricInfo]) {
+
+ override def hashCode(): Int = {
+ // hashCode of simpleString should be good enough to distinguish the plans from each other
+ // within a plan
+ simpleString.hashCode
+ }
+
+ override def equals(other: Any): Boolean = other match {
+ case o: SparkPlanInfo =>
+ nodeName == o.nodeName && simpleString == o.simpleString && children == o.children
+ case _ => false
+ }
+}
private[sql] object SparkPlanInfo {
def fromSparkPlan(plan: SparkPlan): SparkPlanInfo = {
-
- val children = plan.children ++ plan.subqueries
+ val children = plan match {
+ case ReusedExchange(_, child) => child :: Nil
+ case _ => plan.children ++ plan.subqueries
+ }
val metrics = plan.metrics.toSeq.map { case (key, metric) =>
new SQLMetricInfo(metric.name.getOrElse(key), metric.id,
Utils.getFormattedClassName(metric.param))
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala
index f07add83d5..f856634cf7 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala
@@ -46,6 +46,10 @@ case class TungstenAggregate(
require(TungstenAggregate.supportsAggregate(aggregateBufferAttributes))
+ override lazy val allAttributes: Seq[Attribute] =
+ child.output ++ aggregateBufferAttributes ++ aggregateAttributes ++
+ aggregateExpressions.flatMap(_.aggregateFunction.inputAggBufferAttributes)
+
override private[sql] lazy val metrics = Map(
"numOutputRows" -> SQLMetrics.createLongMetric(sparkContext, "number of output rows"),
"dataSize" -> SQLMetrics.createSizeMetric(sparkContext, "data size"),
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala
index 4a9e736f7a..4901298227 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala
@@ -166,6 +166,9 @@ case class Range(
private[sql] override lazy val metrics = Map(
"numOutputRows" -> SQLMetrics.createLongMetric(sparkContext, "number of output rows"))
+ // output attributes should not affect the results
+ override lazy val cleanArgs: Seq[Any] = Seq(start, step, numSlices, numElements)
+
override def upstreams(): Seq[RDD[InternalRow]] = {
sqlContext.sparkContext.parallelize(0 until numSlices, numSlices)
.map(i => InternalRow(i)) :: Nil
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchange.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchange.scala
index 40cad4b1a7..1a5c6a66c4 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchange.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchange.scala
@@ -34,12 +34,16 @@ import org.apache.spark.util.ThreadUtils
*/
case class BroadcastExchange(
mode: BroadcastMode,
- child: SparkPlan) extends UnaryNode {
-
- override def output: Seq[Attribute] = child.output
+ child: SparkPlan) extends Exchange {
override def outputPartitioning: Partitioning = BroadcastPartitioning(mode)
+ override def sameResult(plan: SparkPlan): Boolean = plan match {
+ case p: BroadcastExchange =>
+ mode.compatibleWith(p.mode) && child.sameResult(p.child)
+ case _ => false
+ }
+
@transient
private val timeout: Duration = {
val timeoutValue = sqlContext.conf.broadcastTimeout
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/Exchange.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/Exchange.scala
new file mode 100644
index 0000000000..12513e9106
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/Exchange.scala
@@ -0,0 +1,92 @@
+/*
+ * 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.exchange
+
+import scala.collection.mutable
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.broadcast
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.SQLContext
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.execution.{LeafNode, SparkPlan, UnaryNode}
+import org.apache.spark.sql.types.StructType
+
+/**
+ * An interface for exchanges.
+ */
+abstract class Exchange extends UnaryNode {
+ override def output: Seq[Attribute] = child.output
+}
+
+/**
+ * A wrapper for reused exchange to have different output, because two exchanges which produce
+ * logically identical output will have distinct sets of output attribute ids, so we need to
+ * preserve the original ids because they're what downstream operators are expecting.
+ */
+case class ReusedExchange(override val output: Seq[Attribute], child: Exchange) extends LeafNode {
+
+ override def sameResult(plan: SparkPlan): Boolean = {
+ // Ignore this wrapper. `plan` could also be a ReusedExchange, so we reverse the order here.
+ plan.sameResult(child)
+ }
+
+ def doExecute(): RDD[InternalRow] = {
+ child.execute()
+ }
+
+ override protected[sql] def doExecuteBroadcast[T](): broadcast.Broadcast[T] = {
+ child.executeBroadcast()
+ }
+
+ // Do not repeat the same tree in explain.
+ override def treeChildren: Seq[SparkPlan] = Nil
+}
+
+/**
+ * Find out duplicated exchanges in the spark plan, then use the same exchange for all the
+ * references.
+ */
+private[sql] case class ReuseExchange(sqlContext: SQLContext) extends Rule[SparkPlan] {
+
+ def apply(plan: SparkPlan): SparkPlan = {
+ if (!sqlContext.conf.exchangeReuseEnabled) {
+ return plan
+ }
+ // Build a hash map using schema of exchanges to avoid O(N*N) sameResult calls.
+ val exchanges = mutable.HashMap[StructType, ArrayBuffer[Exchange]]()
+ plan.transformUp {
+ case exchange: Exchange =>
+ // the exchanges that have same results usually also have same schemas (same column names).
+ val sameSchema = exchanges.getOrElseUpdate(exchange.schema, ArrayBuffer[Exchange]())
+ val samePlan = sameSchema.find { e =>
+ exchange.sameResult(e)
+ }
+ if (samePlan.isDefined) {
+ // Keep the output of this exchange, the following plans require that to resolve
+ // attributes.
+ ReusedExchange(exchange.output, samePlan.get)
+ } else {
+ sameSchema += exchange
+ exchange
+ }
+ }
+ }
+}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchange.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchange.scala
index de21d7705e..4eb4d9adbd 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchange.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchange.scala
@@ -38,7 +38,7 @@ import org.apache.spark.util.MutablePair
case class ShuffleExchange(
var newPartitioning: Partitioning,
child: SparkPlan,
- @transient coordinator: Option[ExchangeCoordinator]) extends UnaryNode {
+ @transient coordinator: Option[ExchangeCoordinator]) extends Exchange {
override def nodeName: String = {
val extraInfo = coordinator match {
@@ -55,8 +55,6 @@ case class ShuffleExchange(
override def outputPartitioning: Partitioning = newPartitioning
- override def output: Seq[Attribute] = child.output
-
private val serializer: Serializer = new UnsafeRowSerializer(child.output.size)
override protected def doPrepare(): Unit = {
@@ -103,16 +101,25 @@ case class ShuffleExchange(
new ShuffledRowRDD(shuffleDependency, specifiedPartitionStartIndices)
}
+ /**
+ * Caches the created ShuffleRowRDD so we can reuse that.
+ */
+ private var cachedShuffleRDD: ShuffledRowRDD = null
+
protected override def doExecute(): RDD[InternalRow] = attachTree(this, "execute") {
- coordinator match {
- case Some(exchangeCoordinator) =>
- val shuffleRDD = exchangeCoordinator.postShuffleRDD(this)
- assert(shuffleRDD.partitions.length == newPartitioning.numPartitions)
- shuffleRDD
- case None =>
- val shuffleDependency = prepareShuffleDependency()
- preparePostShuffleRDD(shuffleDependency)
+ // Returns the same ShuffleRowRDD if this plan is used by multiple plans.
+ if (cachedShuffleRDD == null) {
+ cachedShuffleRDD = coordinator match {
+ case Some(exchangeCoordinator) =>
+ val shuffleRDD = exchangeCoordinator.postShuffleRDD(this)
+ assert(shuffleRDD.partitions.length == newPartitioning.numPartitions)
+ shuffleRDD
+ case None =>
+ val shuffleDependency = prepareShuffleDependency()
+ preparePostShuffleRDD(shuffleDependency)
+ }
}
+ cachedShuffleRDD
}
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala
index 9a3cdaf697..99f8841c87 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala
@@ -681,7 +681,7 @@ private[execution] case class HashedRelationBroadcastMode(
keys: Seq[Expression],
attributes: Seq[Attribute]) extends BroadcastMode {
- def transform(rows: Array[InternalRow]): HashedRelation = {
+ override def transform(rows: Array[InternalRow]): HashedRelation = {
val generator = UnsafeProjection.create(keys, attributes)
if (canJoinKeyFitWithinLong) {
LongHashedRelation(rows.iterator, generator, rows.length)
@@ -689,5 +689,18 @@ private[execution] case class HashedRelationBroadcastMode(
HashedRelation(rows.iterator, generator, rows.length)
}
}
+
+ private lazy val canonicalizedKeys: Seq[Expression] = {
+ keys.map { e =>
+ BindReferences.bindReference(e.canonicalized, attributes)
+ }
+ }
+
+ override def compatibleWith(other: BroadcastMode): Boolean = other match {
+ case m: HashedRelationBroadcastMode =>
+ canJoinKeyFitWithinLong == m.canJoinKeyFitWithinLong &&
+ canonicalizedKeys == m.canonicalizedKeys
+ case _ => false
+ }
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SparkPlanGraph.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SparkPlanGraph.scala
index 83372aa2e9..94d318e702 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SparkPlanGraph.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SparkPlanGraph.scala
@@ -64,7 +64,8 @@ private[sql] object SparkPlanGraph {
val nodeIdGenerator = new AtomicLong(0)
val nodes = mutable.ArrayBuffer[SparkPlanGraphNode]()
val edges = mutable.ArrayBuffer[SparkPlanGraphEdge]()
- buildSparkPlanGraphNode(planInfo, nodeIdGenerator, nodes, edges, null, null)
+ val exchanges = mutable.HashMap[SparkPlanInfo, SparkPlanGraphNode]()
+ buildSparkPlanGraphNode(planInfo, nodeIdGenerator, nodes, edges, null, null, exchanges)
new SparkPlanGraph(nodes, edges)
}
@@ -74,7 +75,8 @@ private[sql] object SparkPlanGraph {
nodes: mutable.ArrayBuffer[SparkPlanGraphNode],
edges: mutable.ArrayBuffer[SparkPlanGraphEdge],
parent: SparkPlanGraphNode,
- subgraph: SparkPlanGraphCluster): Unit = {
+ subgraph: SparkPlanGraphCluster,
+ exchanges: mutable.HashMap[SparkPlanInfo, SparkPlanGraphNode]): Unit = {
planInfo.nodeName match {
case "WholeStageCodegen" =>
val cluster = new SparkPlanGraphCluster(
@@ -84,13 +86,14 @@ private[sql] object SparkPlanGraph {
mutable.ArrayBuffer[SparkPlanGraphNode]())
nodes += cluster
buildSparkPlanGraphNode(
- planInfo.children.head, nodeIdGenerator, nodes, edges, parent, cluster)
+ planInfo.children.head, nodeIdGenerator, nodes, edges, parent, cluster, exchanges)
case "InputAdapter" =>
- buildSparkPlanGraphNode(planInfo.children.head, nodeIdGenerator, nodes, edges, parent, null)
+ buildSparkPlanGraphNode(
+ planInfo.children.head, nodeIdGenerator, nodes, edges, parent, null, exchanges)
case "Subquery" if subgraph != null =>
// Subquery should not be included in WholeStageCodegen
- buildSparkPlanGraphNode(planInfo, nodeIdGenerator, nodes, edges, parent, null)
- case _ =>
+ buildSparkPlanGraphNode(planInfo, nodeIdGenerator, nodes, edges, parent, null, exchanges)
+ case name =>
val metrics = planInfo.metrics.map { metric =>
SQLPlanMetric(metric.name, metric.accumulatorId,
SQLMetrics.getMetricParam(metric.metricParam))
@@ -103,12 +106,15 @@ private[sql] object SparkPlanGraph {
} else {
subgraph.nodes += node
}
+ if (name == "ShuffleExchange" || name == "BroadcastExchange") {
+ exchanges += planInfo -> node
+ }
if (parent != null) {
edges += SparkPlanGraphEdge(node.id, parent.id)
}
planInfo.children.foreach(
- buildSparkPlanGraphNode(_, nodeIdGenerator, nodes, edges, node, subgraph))
+ buildSparkPlanGraphNode(_, nodeIdGenerator, nodes, edges, node, subgraph, exchanges))
}
}
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
index 1d1e288441..384102e5ea 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
@@ -504,6 +504,10 @@ object SQLConf {
" method",
isPublic = false)
+ val EXCHANGE_REUSE_ENABLED = booleanConf("spark.sql.exchange.reuse",
+ defaultValue = Some(true),
+ doc = "When true, the planner will try to find out duplicated exchanges and re-use them",
+ isPublic = false)
object Deprecated {
val MAPRED_REDUCE_TASKS = "mapred.reduce.tasks"
@@ -564,6 +568,8 @@ class SQLConf extends Serializable with CatalystConf with ParserConf with Loggin
def wholeStageEnabled: Boolean = getConf(WHOLESTAGE_CODEGEN_ENABLED)
+ def exchangeReuseEnabled: Boolean = getConf(EXCHANGE_REUSE_ENABLED)
+
def canonicalView: Boolean = getConf(CANONICAL_NATIVE_VIEW)
def caseSensitiveAnalysis: Boolean = getConf(SQLConf.CASE_SENSITIVE)
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala
index 6f81794b29..98ada4d58a 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala
@@ -24,10 +24,9 @@ import org.apache.spark.sql.catalyst.parser.ParserInterface
import org.apache.spark.sql.catalyst.rules.RuleExecutor
import org.apache.spark.sql.execution._
import org.apache.spark.sql.execution.datasources.{DataSourceAnalysis, PreInsertCastAndRename, ResolveDataSource}
-import org.apache.spark.sql.execution.exchange.EnsureRequirements
+import org.apache.spark.sql.execution.exchange.{EnsureRequirements, ReuseExchange}
import org.apache.spark.sql.util.ExecutionListenerManager
-
/**
* A class that holds all session-specific state in a given [[SQLContext]].
*/
@@ -94,7 +93,8 @@ private[sql] class SessionState(ctx: SQLContext) {
override val batches: Seq[Batch] = Seq(
Batch("Subquery", Once, PlanSubqueries(ctx)),
Batch("Add exchange", Once, EnsureRequirements(ctx)),
- Batch("Whole stage codegen", Once, CollapseCodegenStages(ctx))
+ Batch("Whole stage codegen", Once, CollapseCodegenStages(ctx)),
+ Batch("Reuse duplicated exchanges", Once, ReuseExchange(ctx))
)
}