-
Notifications
You must be signed in to change notification settings - Fork 28.4k
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
[SPARK-22252][SQL] FileFormatWriter should respect the input query schema #19474
Changes from 1 commit
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,6 +20,7 @@ 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.execution.datasources.BasicWriteJobStatsTracker | ||
import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} | ||
import org.apache.spark.util.SerializableConfiguration | ||
|
@@ -30,6 +31,15 @@ import org.apache.spark.util.SerializableConfiguration | |
*/ | ||
trait DataWritingCommand extends RunnableCommand { | ||
|
||
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 change the output schema names, and we have to keep | ||
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. You will scare others. :) -> |
||
// 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 lazy val metrics: Map[String, SQLMetric] = { | ||
val sparkContext = SparkContext.getActive.get | ||
Map( | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -39,7 +39,7 @@ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec | |
import org.apache.spark.sql.catalyst.expressions.{UnsafeProjection, _} | ||
import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning | ||
import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateTimeUtils} | ||
import org.apache.spark.sql.execution.{SortExec, SparkPlan, SQLExecution} | ||
import org.apache.spark.sql.execution.{QueryExecution, SortExec, SQLExecution} | ||
import org.apache.spark.sql.types.StringType | ||
import org.apache.spark.util.{SerializableConfiguration, Utils} | ||
|
||
|
@@ -101,7 +101,7 @@ object FileFormatWriter extends Logging { | |
*/ | ||
def write( | ||
sparkSession: SparkSession, | ||
plan: SparkPlan, | ||
queryExecution: QueryExecution, | ||
fileFormat: FileFormat, | ||
committer: FileCommitProtocol, | ||
outputSpec: OutputSpec, | ||
|
@@ -117,7 +117,7 @@ object FileFormatWriter extends Logging { | |
job.setOutputValueClass(classOf[InternalRow]) | ||
FileOutputFormat.setOutputPath(job, new Path(outputSpec.outputPath)) | ||
|
||
val allColumns = plan.output | ||
val allColumns = queryExecution.logical.output | ||
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. I think it'd be good to leave a comment that we should not use optimized output here in case it will be changed in the future. 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. Btw, shall we use 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. Explicitly using |
||
val partitionSet = AttributeSet(partitionColumns) | ||
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. You might need to double check the |
||
val dataColumns = allColumns.filterNot(partitionSet.contains) | ||
|
||
|
@@ -158,7 +158,7 @@ object FileFormatWriter extends Logging { | |
// We should first sort by partition columns, then bucket id, and finally sorting columns. | ||
val requiredOrdering = partitionColumns ++ bucketIdExpression ++ sortColumns | ||
// the sort order doesn't matter | ||
val actualOrdering = plan.outputOrdering.map(_.child) | ||
val actualOrdering = queryExecution.executedPlan.outputOrdering.map(_.child) | ||
val orderingMatched = if (requiredOrdering.length > actualOrdering.length) { | ||
false | ||
} else { | ||
|
@@ -176,12 +176,12 @@ object FileFormatWriter extends Logging { | |
|
||
try { | ||
val rdd = if (orderingMatched) { | ||
plan.execute() | ||
queryExecution.toRdd | ||
} else { | ||
SortExec( | ||
requiredOrdering.map(SortOrder(_, Ascending)), | ||
global = false, | ||
child = plan).execute() | ||
child = queryExecution.executedPlan).execute() | ||
} | ||
val ret = new Array[WriteTaskResult](rdd.partitions.length) | ||
sparkSession.sparkContext.runJob( | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -17,10 +17,11 @@ | |
|
||
package org.apache.spark.sql.execution.datasources | ||
|
||
import org.apache.spark.sql.QueryTest | ||
import org.apache.spark.sql.{QueryTest, Row} | ||
import org.apache.spark.sql.test.SharedSQLContext | ||
|
||
class FileFormatWriterSuite extends QueryTest with SharedSQLContext { | ||
import testImplicits._ | ||
|
||
test("empty file should be skipped while write to file") { | ||
withTempPath { path => | ||
|
@@ -30,4 +31,12 @@ class FileFormatWriterSuite extends QueryTest with SharedSQLContext { | |
assert(partFiles.length === 2) | ||
} | ||
} | ||
|
||
test("FileFormatWriter should respect the input query schema") { | ||
withTable("t1", "t2") { | ||
spark.range(1).select('id as 'col1, 'id as 'col2).write.saveAsTable("t1") | ||
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. Also add another case here?
|
||
spark.sql("select COL1, COL2 from t1").write.saveAsTable("t2") | ||
checkAnswer(spark.table("t2"), Row(0, 0)) | ||
} | ||
} | ||
} |
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.
Add one line description for
query
?