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-28554][SQL] Adds a v1 fallback writer implementation for v2 data source codepaths #25348
Closed
Closed
Changes from all commits
Commits
Show all changes
25 commits
Select commit
Hold shift + click to select a range
d5798fd
Adds a v1 fallback writer implementation for v2 data source codepaths
brkyvz 59094e9
Update WriteToDataSourceV2Exec.scala
brkyvz 1587d31
Merge branch 'master' of github.com:apache/spark into v1WriteFallback
brkyvz 5aed803
some changes but doubtful
brkyvz bcdf8c5
Merge branch 'master' of github.com:apache/spark into v1WriteFallback
brkyvz a1284a1
Revert "some changes but doubtful"
brkyvz ef2ec72
Address comments and separate whatever's possible
brkyvz 335a92d
update docs
brkyvz 3e35d5c
minor move for better separation
brkyvz 41c4c0a
use implicit class
brkyvz 138f2b9
use insertable relation instead
brkyvz 078d0f1
Merge branch 'master' of github.com:apache/spark into v1WriteFallback
brkyvz d816824
address comments
brkyvz c62a60d
Merge branch 'master' of github.com:apache/spark into v1WriteFallback
brkyvz cfdb036
Added basic tests
brkyvz 442836b
Update V1WriteBuilder.scala
brkyvz 7396f24
fix tests
brkyvz 20b906d
Merge branch 'v1WriteFallback' of github.com:brkyvz/spark into v1Writ…
brkyvz a93bf8c
Update V1WriteBuilder.scala
brkyvz 83dbd78
Merge branch 'master' into v1WriteFallback
brkyvz 9bfb76e
Merge branch 'master' of github.com:apache/spark into v1WriteFallback
brkyvz 00347ee
Add table capability to do V1_BATCH_WRITE
brkyvz d4d6276
test refactor
brkyvz 749ae85
Update V1WriteFallbackSuite.scala
brkyvz 27598ce
address nits
brkyvz File filter
Filter by extension
Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
121 changes: 121 additions & 0 deletions
121
...core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V1FallbackWriters.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,121 @@ | ||
/* | ||
* 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.v2 | ||
|
||
import java.util.UUID | ||
|
||
import scala.collection.JavaConverters._ | ||
|
||
import org.apache.spark.SparkException | ||
import org.apache.spark.rdd.RDD | ||
import org.apache.spark.sql.{Dataset, SaveMode} | ||
import org.apache.spark.sql.catalyst.InternalRow | ||
import org.apache.spark.sql.catalyst.expressions.Attribute | ||
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan | ||
import org.apache.spark.sql.execution.SparkPlan | ||
import org.apache.spark.sql.sources.{AlwaysTrue, CreatableRelationProvider, Filter, InsertableRelation} | ||
import org.apache.spark.sql.sources.v2.{SupportsWrite, Table} | ||
import org.apache.spark.sql.sources.v2.writer._ | ||
import org.apache.spark.sql.util.CaseInsensitiveStringMap | ||
|
||
/** | ||
* Physical plan node for append into a v2 table using V1 write interfaces. | ||
* | ||
* Rows in the output data set are appended. | ||
*/ | ||
case class AppendDataExecV1( | ||
brkyvz marked this conversation as resolved.
Show resolved
Hide resolved
|
||
table: SupportsWrite, | ||
writeOptions: CaseInsensitiveStringMap, | ||
plan: LogicalPlan) extends V1FallbackWriters { | ||
brkyvz marked this conversation as resolved.
Show resolved
Hide resolved
|
||
|
||
override protected def doExecute(): RDD[InternalRow] = { | ||
writeWithV1(newWriteBuilder().buildForV1Write()) | ||
} | ||
} | ||
|
||
/** | ||
* Physical plan node for overwrite into a v2 table with V1 write interfaces. Note that when this | ||
* interface is used, the atomicity of the operation depends solely on the target data source. | ||
* | ||
* Overwrites data in a table matched by a set of filters. Rows matching all of the filters will be | ||
* deleted and rows in the output data set are appended. | ||
* | ||
* This plan is used to implement SaveMode.Overwrite. The behavior of SaveMode.Overwrite is to | ||
* truncate the table -- delete all rows -- and append the output data set. This uses the filter | ||
* AlwaysTrue to delete all rows. | ||
*/ | ||
case class OverwriteByExpressionExecV1( | ||
table: SupportsWrite, | ||
deleteWhere: Array[Filter], | ||
writeOptions: CaseInsensitiveStringMap, | ||
plan: LogicalPlan) extends V1FallbackWriters { | ||
|
||
private def isTruncate(filters: Array[Filter]): Boolean = { | ||
filters.length == 1 && filters(0).isInstanceOf[AlwaysTrue] | ||
} | ||
|
||
override protected def doExecute(): RDD[InternalRow] = { | ||
newWriteBuilder() match { | ||
case builder: SupportsTruncate if isTruncate(deleteWhere) => | ||
writeWithV1(builder.truncate().asV1Builder.buildForV1Write()) | ||
brkyvz marked this conversation as resolved.
Show resolved
Hide resolved
|
||
|
||
case builder: SupportsOverwrite => | ||
writeWithV1(builder.overwrite(deleteWhere).asV1Builder.buildForV1Write()) | ||
|
||
case _ => | ||
throw new SparkException(s"Table does not support overwrite by expression: $table") | ||
} | ||
} | ||
} | ||
|
||
/** Some helper interfaces that use V2 write semantics through the V1 writer interface. */ | ||
sealed trait V1FallbackWriters extends SupportsV1Write { | ||
override def output: Seq[Attribute] = Nil | ||
override final def children: Seq[SparkPlan] = Nil | ||
|
||
def table: SupportsWrite | ||
def writeOptions: CaseInsensitiveStringMap | ||
|
||
protected implicit class toV1WriteBuilder(builder: WriteBuilder) { | ||
def asV1Builder: V1WriteBuilder = builder match { | ||
case v1: V1WriteBuilder => v1 | ||
case other => throw new IllegalStateException( | ||
s"The returned writer ${other} was no longer a V1WriteBuilder.") | ||
} | ||
} | ||
|
||
protected def newWriteBuilder(): V1WriteBuilder = { | ||
val writeBuilder = table.newWriteBuilder(writeOptions) | ||
.withInputDataSchema(plan.schema) | ||
.withQueryId(UUID.randomUUID().toString) | ||
writeBuilder.asV1Builder | ||
} | ||
} | ||
|
||
/** | ||
* A trait that allows Tables that use V1 Writer interfaces to append data. | ||
*/ | ||
trait SupportsV1Write extends SparkPlan { | ||
// TODO: We should be able to work on SparkPlans at this point. | ||
def plan: LogicalPlan | ||
|
||
protected def writeWithV1(relation: InsertableRelation): RDD[InternalRow] = { | ||
relation.insert(Dataset.ofRows(sqlContext.sparkSession, plan), overwrite = false) | ||
brkyvz marked this conversation as resolved.
Show resolved
Hide resolved
|
||
sparkContext.emptyRDD | ||
} | ||
} |
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
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.
Why we reuse the v2
CreateTableAsSelectExec
for v1 fallback, but notAppendDataExec
?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.
because table creation happens after certain operations with side effects in CTAS and RTAS.
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.
I think this is because CTAS and RTAS plans don't have a table instance. CTAS and RTAS create a table at runtime and then write to it.
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.
This explains why we must reuse
CreateTableAsSelectExec
for v1 fallback, but why can't we reuseAppendDataExec
for v1 fallback as well?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.
It's just cleaner to have a separation of which code path was used. (Shows up separately in the SQL tab, etc)
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.
If end-users look at the SQL tab and see
AppendDataExecV1
, they would expect to see v1 version of CTAS physical plan as well, and may report a bug if they don't see it.BTW I think there are other ways to implement this feature (users know if v1 fallback is triggered from SQL tab), e.g. we can use SQLMetrics to report it, which can be updated at runtime and support CTAS as well.
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.
I don't think most (98%) of Spark users already don't know what the physical nodes stand for, etc. I was thinking of using metrics as well. We just wanted to keep the nodes separate, because the semantics are a bit different for things like
OverwriteByExpression
, where the source's implementation of the Overwrite + Append (v2 behavior) may not be atomic, etcThere 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.
good point on having different physical nodes for different semantics. It seems to me that the semantic of append is the same between v1 and v2. Shall we reuse
AppendDataExec
? Then we can make a good story: If semantic is the same, reuse the v2 plan (CREATE TABLE, CTAS, APPEND). Otherwise, create a new physical node (OVERWRITE).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.
I like the idea to use the same physical plan node when the semantics match, but I'm not sure that it is possible to make strong guarantees about not changing v1 if we do. The problem is that there are multiple v1 plan nodes for the same operation, which could have slightly different behavior.
We could take the time to inspect the v1 implementations and convert, but that adds risk and takes time. It also isn't needed to migrate to v2, and wouldn't speed up the migration, unless I'm missing something. So it probably doesn't provide enough value to make doing it worth while.