-
Notifications
You must be signed in to change notification settings - Fork 28.6k
[SPARK-41407][SQL] Pull out v1 write to WriteFiles #38939
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 |
---|---|---|
@@ -0,0 +1,33 @@ | ||
/* | ||
* 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.internal; | ||
|
||
import java.io.Serializable; | ||
|
||
/** | ||
* Write spec is a input parameter of | ||
* {@link org.apache.spark.sql.execution.SparkPlan#executeWrite}. | ||
* | ||
* <p> | ||
* This is an empty interface, the concrete class which implements | ||
* {@link org.apache.spark.sql.execution.SparkPlan#doExecuteWrite} | ||
* should define its own class and use it. | ||
* | ||
* @since 3.4.0 | ||
*/ | ||
public interface WriteSpec extends Serializable {} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -34,9 +34,10 @@ import org.apache.spark.sql.catalyst.plans.QueryPlan | |
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan | ||
import org.apache.spark.sql.catalyst.plans.physical._ | ||
import org.apache.spark.sql.catalyst.trees.{BinaryLike, LeafLike, TreeNodeTag, UnaryLike} | ||
import org.apache.spark.sql.connector.write.WriterCommitMessage | ||
import org.apache.spark.sql.errors.QueryExecutionErrors | ||
import org.apache.spark.sql.execution.metric.SQLMetric | ||
import org.apache.spark.sql.internal.SQLConf | ||
import org.apache.spark.sql.internal.{SQLConf, WriteSpec} | ||
import org.apache.spark.sql.vectorized.ColumnarBatch | ||
import org.apache.spark.util.NextIterator | ||
import org.apache.spark.util.io.{ChunkedByteBuffer, ChunkedByteBufferOutputStream} | ||
|
@@ -223,6 +224,19 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ | |
doExecuteColumnar() | ||
} | ||
|
||
/** | ||
* Returns the result of writes as an RDD[WriterCommitMessage] variable by delegating to | ||
* `doExecuteWrite` after preparations. | ||
* | ||
* Concrete implementations of SparkPlan should override `doExecuteWrite`. | ||
*/ | ||
def executeWrite(writeSpec: WriteSpec): RDD[WriterCommitMessage] = executeQuery { | ||
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. regardless of how hard to implement, ideally which information should be in the WriteFiles operator and which should be passed as parameters? 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. Let me list the required things of current v1 write files:
According to the existed datasource v1 writes command. case class InsertIntoHadoopFsRelationCommand(
outputPath: Path,
staticPartitions: TablePartitionSpec,
ifPartitionNotExists: Boolean,
partitionColumns: Seq[Attribute],
bucketSpec: Option[BucketSpec],
fileFormat: FileFormat,
options: Map[String, String],
query: LogicalPlan,
mode: SaveMode,
catalogTable: Option[CatalogTable],
fileIndex: Option[FileIndex],
outputColumnNames: Seq[String]) Due to we can not get physical plan at logical side, and ConcurrentOutputWriterSpec depend on physical. It should be held in FileCommitProtocol should be held in According to the usage of hadoop job (FileCommitProtocol.setup(Job)), I tend to make In sum:
Notes: the aboved does not consider how hard to implement, just based on semantic level. |
||
if (isCanonicalizedPlan) { | ||
throw SparkException.internalError("A canonicalized plan is not supposed to be executed.") | ||
} | ||
doExecuteWrite(writeSpec) | ||
} | ||
|
||
/** | ||
* Executes a query after preparing the query and adding query plan information to created RDDs | ||
* for visualization. | ||
|
@@ -324,6 +338,16 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ | |
s" mismatch:\n${this}") | ||
} | ||
|
||
/** | ||
* Produces the result of the writes as an `RDD[WriterCommitMessage]` | ||
* | ||
* Overridden by concrete implementations of SparkPlan. | ||
*/ | ||
protected def doExecuteWrite(writeSpec: WriteSpec): RDD[WriterCommitMessage] = { | ||
throw SparkException.internalError(s"Internal Error ${this.getClass} has write support" + | ||
s" mismatch:\n${this}") | ||
} | ||
|
||
/** | ||
* Converts the output of this plan to row-based if it is columnar plan. | ||
*/ | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -145,6 +145,12 @@ case class CreateDataSourceTableAsSelectCommand( | |
outputColumnNames: Seq[String]) | ||
extends V1WriteCommand { | ||
|
||
override def fileFormatProvider: Boolean = { | ||
table.provider.forall { provider => | ||
classOf[FileFormat].isAssignableFrom(DataSource.providingClass(provider, conf)) | ||
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.
|
||
} | ||
} | ||
|
||
override lazy val partitionColumns: Seq[Attribute] = { | ||
val unresolvedPartitionColumns = table.partitionColumnNames.map(UnresolvedAttribute.quoted) | ||
DataSource.resolvePartitionColumns( | ||
|
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.
@cloud-fan after we introduce Writefiles, the error stack for writing changes.
before:
after:
So I added the root cause mesagge into the wrapped
SparkException
.