-
Notifications
You must be signed in to change notification settings - Fork 593
[CORE] Refactor columnar noop write rule #8422
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -30,7 +30,6 @@ import org.apache.spark.sql.catalyst.rules.Rule | |
| import org.apache.spark.sql.execution._ | ||
| import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec | ||
| import org.apache.spark.sql.execution.command.{CreateDataSourceTableAsSelectCommand, DataWritingCommand, DataWritingCommandExec} | ||
| import org.apache.spark.sql.execution.datasources.v2.{AppendDataExec, OverwriteByExpressionExec} | ||
| import org.apache.spark.sql.hive.execution.{CreateHiveTableAsSelectCommand, InsertIntoHiveDirCommand, InsertIntoHiveTable} | ||
| import org.apache.spark.sql.sources.DataSourceRegister | ||
| import org.apache.spark.sql.vectorized.ColumnarBatch | ||
|
|
@@ -133,19 +132,33 @@ object GlutenWriterColumnarRules { | |
| } | ||
| } | ||
|
|
||
| case class NativeWritePostRule(session: SparkSession) extends Rule[SparkPlan] { | ||
| private[datasources] def injectFakeRowAdaptor(command: SparkPlan, child: SparkPlan): SparkPlan = { | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Is this API only called by
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This API is also needed in |
||
| child match { | ||
| // if the child is columnar, we can just wrap & transfer the columnar data | ||
| case c2r: ColumnarToRowExecBase => | ||
| command.withNewChildren(Array(FakeRowAdaptor(c2r.child))) | ||
| // If the child is aqe, we make aqe "support columnar", | ||
| // then aqe itself will guarantee to generate columnar outputs. | ||
| // So FakeRowAdaptor will always consumes columnar data, | ||
| // thus avoiding the case of c2r->aqe->r2c->writer | ||
| case aqe: AdaptiveSparkPlanExec => | ||
| command.withNewChildren( | ||
| Array( | ||
| FakeRowAdaptor( | ||
| AdaptiveSparkPlanExec( | ||
| aqe.inputPlan, | ||
| aqe.context, | ||
| aqe.preprocessingRules, | ||
| aqe.isSubquery, | ||
| supportsColumnar = true | ||
| )))) | ||
| case other => command.withNewChildren(Array(FakeRowAdaptor(other))) | ||
| } | ||
| } | ||
|
|
||
| private val NOOP_WRITE = "org.apache.spark.sql.execution.datasources.noop.NoopWrite$" | ||
| case class NativeWritePostRule(session: SparkSession) extends Rule[SparkPlan] { | ||
|
|
||
| override def apply(p: SparkPlan): SparkPlan = p match { | ||
| case rc @ AppendDataExec(_, _, write) | ||
| if write.getClass.getName == NOOP_WRITE && | ||
| BackendsApiManager.getSettings.enableNativeWriteFiles() => | ||
| injectFakeRowAdaptor(rc, rc.child) | ||
| case rc @ OverwriteByExpressionExec(_, _, write) | ||
| if write.getClass.getName == NOOP_WRITE && | ||
| BackendsApiManager.getSettings.enableNativeWriteFiles() => | ||
| injectFakeRowAdaptor(rc, rc.child) | ||
| case rc @ DataWritingCommandExec(cmd, child) => | ||
| // The same thread can set these properties in the last query submission. | ||
| val fields = child.output.toStructType.fields | ||
|
|
@@ -165,30 +178,6 @@ object GlutenWriterColumnarRules { | |
|
|
||
| case plan: SparkPlan => plan.withNewChildren(plan.children.map(apply)) | ||
| } | ||
|
|
||
| private def injectFakeRowAdaptor(command: SparkPlan, child: SparkPlan): SparkPlan = { | ||
| child match { | ||
| // if the child is columnar, we can just wrap&transfer the columnar data | ||
| case c2r: ColumnarToRowExecBase => | ||
| command.withNewChildren(Array(FakeRowAdaptor(c2r.child))) | ||
| // If the child is aqe, we make aqe "support columnar", | ||
| // then aqe itself will guarantee to generate columnar outputs. | ||
| // So FakeRowAdaptor will always consumes columnar data, | ||
| // thus avoiding the case of c2r->aqe->r2c->writer | ||
| case aqe: AdaptiveSparkPlanExec => | ||
| command.withNewChildren( | ||
| Array( | ||
| FakeRowAdaptor( | ||
| AdaptiveSparkPlanExec( | ||
| aqe.inputPlan, | ||
| aqe.context, | ||
| aqe.preprocessingRules, | ||
| aqe.isSubquery, | ||
| supportsColumnar = true | ||
| )))) | ||
| case other => command.withNewChildren(Array(FakeRowAdaptor(other))) | ||
| } | ||
| } | ||
| } | ||
|
|
||
| def injectSparkLocalProperty(spark: SparkSession, format: Option[String]): Unit = { | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,42 @@ | ||
| /* | ||
| * 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.datasources.noop | ||
|
|
||
| import org.apache.spark.sql.SparkSession | ||
| import org.apache.spark.sql.catalyst.rules.Rule | ||
| import org.apache.spark.sql.execution.SparkPlan | ||
| import org.apache.spark.sql.execution.datasources.GlutenWriterColumnarRules.injectFakeRowAdaptor | ||
| import org.apache.spark.sql.execution.datasources.v2.{AppendDataExec, OverwriteByExpressionExec} | ||
|
|
||
| /** | ||
| * A rule that injects a FakeRowAdaptor for NoopWrite. | ||
| * | ||
| * The current V2 Command does not support columnar. Therefore, when its child node is a | ||
| * ColumnarNode, Vanilla Spark inserts a ColumnarToRow conversion between V2 Command and its child. | ||
| * This rule replaces the inserted ColumnarToRow with a FakeRowAdaptor, effectively bypassing the | ||
| * ColumnarToRow operation for NoopWrite. Since NoopWrite does not actually perform any data | ||
| * operations, it can accept input data in either row-based or columnar format. | ||
| */ | ||
| case class GlutenNoopWriterRule(session: SparkSession) extends Rule[SparkPlan] { | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Such rule could be placed in this folder.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. We cannot move to that folder as the |
||
| override def apply(p: SparkPlan): SparkPlan = p match { | ||
| case rc @ AppendDataExec(_, _, NoopWrite) => | ||
| injectFakeRowAdaptor(rc, rc.child) | ||
| case rc @ OverwriteByExpressionExec(_, _, NoopWrite) => | ||
| injectFakeRowAdaptor(rc, rc.child) | ||
| case _ => p | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -16,6 +16,7 @@ | |
| */ | ||
| package org.apache.spark.sql.execution.adaptive.clickhouse | ||
|
|
||
| import org.apache.gluten.config.GlutenConfig | ||
| import org.apache.gluten.execution.{BroadcastHashJoinExecTransformerBase, ShuffledHashJoinExecTransformerBase, SortExecTransformer, SortMergeJoinExecTransformer} | ||
|
|
||
| import org.apache.spark.SparkConf | ||
|
|
@@ -25,14 +26,20 @@ import org.apache.spark.sql.GlutenTestConstants.GLUTEN_TEST | |
| import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight} | ||
| import org.apache.spark.sql.execution._ | ||
| import org.apache.spark.sql.execution.adaptive._ | ||
| import org.apache.spark.sql.execution.command.DataWritingCommandExec | ||
| import org.apache.spark.sql.execution.datasources.FakeRowAdaptor | ||
| import org.apache.spark.sql.execution.datasources.noop.NoopDataSource | ||
| import org.apache.spark.sql.execution.datasources.v2.V2TableWriteExec | ||
| import org.apache.spark.sql.execution.exchange._ | ||
| import org.apache.spark.sql.execution.joins.{BaseJoinExec, BroadcastHashJoinExec, ShuffledHashJoinExec, SortMergeJoinExec} | ||
| import org.apache.spark.sql.execution.metric.SQLShuffleReadMetricsReporter | ||
| import org.apache.spark.sql.execution.ui.SparkListenerSQLAdaptiveExecutionUpdate | ||
| import org.apache.spark.sql.functions.when | ||
| import org.apache.spark.sql.internal.SQLConf | ||
| import org.apache.spark.sql.internal.SQLConf.PartitionOverwriteMode | ||
| import org.apache.spark.sql.test.SQLTestData.TestData | ||
| import org.apache.spark.sql.types.{IntegerType, StructType} | ||
| import org.apache.spark.sql.util.QueryExecutionListener | ||
|
|
||
| import org.apache.log4j.Level | ||
|
|
||
|
|
@@ -42,7 +49,7 @@ class ClickHouseAdaptiveQueryExecSuite extends AdaptiveQueryExecSuite with Glute | |
|
|
||
| override def sparkConf: SparkConf = { | ||
| super.sparkConf | ||
| .set("spark.gluten.sql.columnar.forceShuffledHashJoin", "false") | ||
| .set(GlutenConfig.COLUMNAR_FORCE_SHUFFLED_HASH_JOIN_ENABLED.key, "false") | ||
| .set(SQLConf.SHUFFLE_PARTITIONS.key, "5") | ||
| } | ||
|
|
||
|
|
@@ -1196,6 +1203,86 @@ class ClickHouseAdaptiveQueryExecSuite extends AdaptiveQueryExecSuite with Glute | |
| } | ||
| } | ||
|
|
||
| testGluten("SPARK-32932: Do not use local shuffle read at final stage on write command") { | ||
| withSQLConf( | ||
| SQLConf.PARTITION_OVERWRITE_MODE.key -> PartitionOverwriteMode.DYNAMIC.toString, | ||
| SQLConf.SHUFFLE_PARTITIONS.key -> "5", | ||
| SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true" | ||
| ) { | ||
| val data = | ||
| for ( | ||
| i <- 1L to 10L; | ||
| j <- 1L to 3L | ||
| ) yield (i, j) | ||
|
|
||
| val df = data.toDF("i", "j").repartition($"j") | ||
| var noLocalread: Boolean = false | ||
| val listener = new QueryExecutionListener { | ||
| override def onSuccess(funcName: String, qe: QueryExecution, durationNs: Long): Unit = { | ||
| qe.executedPlan match { | ||
| case plan @ (_: DataWritingCommandExec | _: V2TableWriteExec) => | ||
| noLocalread = collect(plan) { | ||
|
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Remove the child plan check as we would add |
||
| case exec: AQEShuffleReadExec if exec.isLocalRead => exec | ||
| }.isEmpty | ||
| case _ => // ignore other events | ||
| } | ||
| } | ||
| override def onFailure( | ||
| funcName: String, | ||
| qe: QueryExecution, | ||
| exception: Exception): Unit = {} | ||
| } | ||
| spark.listenerManager.register(listener) | ||
|
|
||
| withTable("t") { | ||
| df.write.partitionBy("j").saveAsTable("t") | ||
| sparkContext.listenerBus.waitUntilEmpty() | ||
| assert(noLocalread) | ||
| noLocalread = false | ||
| } | ||
|
|
||
| // Test DataSource v2 | ||
| val format = classOf[NoopDataSource].getName | ||
| df.write.format(format).mode("overwrite").save() | ||
| sparkContext.listenerBus.waitUntilEmpty() | ||
| assert(noLocalread) | ||
| noLocalread = false | ||
|
|
||
| spark.listenerManager.unregister(listener) | ||
| } | ||
| } | ||
|
|
||
| testGluten( | ||
| "SPARK-30953: InsertAdaptiveSparkPlan should apply AQE on child plan of v2 write commands") { | ||
| withSQLConf( | ||
| SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", | ||
| SQLConf.ADAPTIVE_EXECUTION_FORCE_APPLY.key -> "true") { | ||
| var plan: SparkPlan = null | ||
| val listener = new QueryExecutionListener { | ||
| override def onSuccess(funcName: String, qe: QueryExecution, durationNs: Long): Unit = { | ||
| plan = qe.executedPlan | ||
| } | ||
| override def onFailure( | ||
| funcName: String, | ||
| qe: QueryExecution, | ||
| exception: Exception): Unit = {} | ||
| } | ||
| spark.listenerManager.register(listener) | ||
| withTable("t1") { | ||
| val format = classOf[NoopDataSource].getName | ||
| Seq((0, 1)).toDF("x", "y").write.format(format).mode("overwrite").save() | ||
|
|
||
| sparkContext.listenerBus.waitUntilEmpty() | ||
| assert(plan.isInstanceOf[V2TableWriteExec]) | ||
| val childPlan = plan.asInstanceOf[V2TableWriteExec].child | ||
| assert(childPlan.isInstanceOf[FakeRowAdaptor]) | ||
| assert(childPlan.asInstanceOf[FakeRowAdaptor].child.isInstanceOf[AdaptiveSparkPlanExec]) | ||
|
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Refine the child plan check |
||
|
|
||
| spark.listenerManager.unregister(listener) | ||
| } | ||
| } | ||
| } | ||
|
|
||
| testGluten("SPARK-35650: Coalesce number of partitions by AEQ") { | ||
| withSQLConf(SQLConf.COALESCE_PARTITIONS_MIN_PARTITION_NUM.key -> "1") { | ||
| Seq("REPARTITION", "REBALANCE(key)") | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.