-
Notifications
You must be signed in to change notification settings - Fork 28.9k
[SPARK-22834][SQL] Make insertion commands have real children to fix UI issues #20020
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
9717709
d49d0f0
043cd55
cb10431
c1b542f
21b5318
bdffa6d
0e4d2e1
18ec016
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 |
|---|---|---|
|
|
@@ -20,30 +20,32 @@ package org.apache.spark.sql.execution.command | |
| import org.apache.hadoop.conf.Configuration | ||
|
|
||
| import org.apache.spark.SparkContext | ||
| import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan | ||
| import org.apache.spark.sql.{Row, SparkSession} | ||
| import org.apache.spark.sql.catalyst.expressions.Attribute | ||
| import org.apache.spark.sql.catalyst.plans.logical.{Command, LogicalPlan} | ||
| import org.apache.spark.sql.execution.SparkPlan | ||
| import org.apache.spark.sql.execution.datasources.BasicWriteJobStatsTracker | ||
| import org.apache.spark.sql.execution.datasources.FileFormatWriter | ||
| import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} | ||
| import org.apache.spark.util.SerializableConfiguration | ||
|
|
||
|
|
||
| /** | ||
| * A special `RunnableCommand` which writes data out and updates metrics. | ||
| * A special `Command` which writes data out and updates metrics. | ||
| */ | ||
| trait DataWritingCommand extends RunnableCommand { | ||
|
|
||
| trait DataWritingCommand extends Command { | ||
| /** | ||
| * The input query plan that produces the data to be written. | ||
| * IMPORTANT: the input query plan MUST be analyzed, so that we can carry its output columns | ||
| * to [[FileFormatWriter]]. | ||
| */ | ||
| def query: LogicalPlan | ||
|
|
||
| // We make the input `query` an inner child instead of a child in order to hide it from the | ||
| // optimizer. This is because optimizer may not preserve the output schema names' case, and we | ||
| // have to keep the original analyzed plan here so that we can pass the corrected schema to the | ||
| // writer. The schema of analyzed plan is what user expects(or specifies), so we should respect | ||
| // it when writing. | ||
| override protected def innerChildren: Seq[LogicalPlan] = query :: Nil | ||
| override final def children: Seq[LogicalPlan] = query :: Nil | ||
|
|
||
| override lazy val metrics: Map[String, SQLMetric] = { | ||
| // Output columns of the analyzed input query plan | ||
| def outputColumns: Seq[Attribute] | ||
|
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.
withTempDir { dir =>
val path = dir.getCanonicalPath
val cnt = 30
val table1Path = s"$path/table1"
val table3Path = s"$path/table3"
spark.range(cnt).selectExpr("cast(id as bigint) as col1", "cast(id % 3 as bigint) as col2")
.write.mode(SaveMode.Overwrite).parquet(table1Path)
withTable("table1", "table3") {
spark.sql(
s"CREATE TABLE table1(col1 bigint, col2 bigint) using parquet location '$table1Path/'")
spark.sql("CREATE TABLE table3(COL1 bigint, COL2 bigint) using parquet " +
"PARTITIONED BY (COL2) " +
s"CLUSTERED BY (COL1) INTO 2 BUCKETS location '$table3Path/'")
withView("view1") {
spark.sql("CREATE VIEW view1 as select col1, col2 from table1 where col1 > -20")
spark.sql("INSERT OVERWRITE TABLE table3 select COL1, COL2 from view1 CLUSTER BY COL1")
spark.table("table3").show
}
}
} |
||
|
|
||
| lazy val metrics: Map[String, SQLMetric] = { | ||
| val sparkContext = SparkContext.getActive.get | ||
| Map( | ||
| "numFiles" -> SQLMetrics.createMetric(sparkContext, "number of written files"), | ||
|
|
@@ -57,4 +59,6 @@ trait DataWritingCommand extends RunnableCommand { | |
| val serializableHadoopConf = new SerializableConfiguration(hadoopConf) | ||
| new BasicWriteJobStatsTracker(serializableHadoopConf, metrics) | ||
| } | ||
|
|
||
| def run(sparkSession: SparkSession, child: SparkPlan): Seq[Row] | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -26,7 +26,7 @@ import org.apache.spark.sql.catalyst.errors.TreeNodeException | |
| import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} | ||
| import org.apache.spark.sql.catalyst.plans.QueryPlan | ||
| import org.apache.spark.sql.catalyst.plans.logical.{Command, LogicalPlan} | ||
| import org.apache.spark.sql.execution.LeafExecNode | ||
| import org.apache.spark.sql.execution.{LeafExecNode, SparkPlan} | ||
| import org.apache.spark.sql.execution.debug._ | ||
| import org.apache.spark.sql.execution.metric.SQLMetric | ||
| import org.apache.spark.sql.execution.streaming.{IncrementalExecution, OffsetSeqMetadata} | ||
|
|
@@ -87,6 +87,42 @@ case class ExecutedCommandExec(cmd: RunnableCommand) extends LeafExecNode { | |
| } | ||
| } | ||
|
||
|
|
||
| /** | ||
| * A physical operator that executes the run method of a `DataWritingCommand` and | ||
| * saves the result to prevent multiple executions. | ||
| * | ||
| * @param cmd the `DataWritingCommand` this operator will run. | ||
| * @param child the physical plan child ran by the `DataWritingCommand`. | ||
| */ | ||
| case class DataWritingCommandExec(cmd: DataWritingCommand, child: SparkPlan) | ||
| extends SparkPlan { | ||
|
|
||
| override lazy val metrics: Map[String, SQLMetric] = cmd.metrics | ||
|
|
||
| protected[sql] lazy val sideEffectResult: Seq[InternalRow] = { | ||
| val converter = CatalystTypeConverters.createToCatalystConverter(schema) | ||
| val rows = cmd.run(sqlContext.sparkSession, child) | ||
|
|
||
| rows.map(converter(_).asInstanceOf[InternalRow]) | ||
| } | ||
|
|
||
| override def children: Seq[SparkPlan] = child :: Nil | ||
|
|
||
| override def output: Seq[Attribute] = cmd.output | ||
|
|
||
| override def nodeName: String = "Execute " + cmd.nodeName | ||
|
|
||
| override def executeCollect(): Array[InternalRow] = sideEffectResult.toArray | ||
|
|
||
| override def executeToIterator: Iterator[InternalRow] = sideEffectResult.toIterator | ||
|
|
||
| override def executeTake(limit: Int): Array[InternalRow] = sideEffectResult.take(limit).toArray | ||
|
|
||
| protected override def doExecute(): RDD[InternalRow] = { | ||
| sqlContext.sparkContext.parallelize(sideEffectResult, 1) | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * An explain command for users to see how a command will be executed. | ||
| * | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -456,17 +456,6 @@ case class DataSource( | |
| val caseSensitive = sparkSession.sessionState.conf.caseSensitiveAnalysis | ||
| PartitioningUtils.validatePartitionColumn(data.schema, partitionColumns, caseSensitive) | ||
|
|
||
|
|
||
| // SPARK-17230: Resolve the partition columns so InsertIntoHadoopFsRelationCommand does | ||
| // not need to have the query as child, to avoid to analyze an optimized query, | ||
| // because InsertIntoHadoopFsRelationCommand will be optimized first. | ||
| val partitionAttributes = partitionColumns.map { name => | ||
| data.output.find(a => equality(a.name, name)).getOrElse { | ||
| throw new AnalysisException( | ||
| s"Unable to resolve $name given [${data.output.map(_.name).mkString(", ")}]") | ||
| } | ||
| } | ||
|
|
||
| val fileIndex = catalogTable.map(_.identifier).map { tableIdent => | ||
| sparkSession.table(tableIdent).queryExecution.analyzed.collect { | ||
| case LogicalRelation(t: HadoopFsRelation, _, _, _) => t.location | ||
|
|
@@ -479,14 +468,15 @@ case class DataSource( | |
| outputPath = outputPath, | ||
| staticPartitions = Map.empty, | ||
| ifPartitionNotExists = false, | ||
| partitionColumns = partitionAttributes, | ||
| partitionColumns = partitionColumns.map(UnresolvedAttribute.quoted), | ||
| bucketSpec = bucketSpec, | ||
| fileFormat = format, | ||
| options = options, | ||
| query = data, | ||
|
||
| mode = mode, | ||
| catalogTable = catalogTable, | ||
| fileIndex = fileIndex) | ||
| fileIndex = fileIndex, | ||
| outputColumns = data.output) | ||
| } | ||
|
|
||
| /** | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
now shall we define query as a child here?