-
Notifications
You must be signed in to change notification settings - Fork 28.1k
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-33808][SQL] DataSource V2: Build logical writes in the optimizer #30806
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 |
---|---|---|
|
@@ -25,6 +25,7 @@ import org.apache.spark.sql.catalyst.util.CharVarcharUtils | |
import org.apache.spark.sql.connector.catalog._ | ||
import org.apache.spark.sql.connector.catalog.TableChange.{AddColumn, ColumnChange} | ||
import org.apache.spark.sql.connector.expressions.Transform | ||
import org.apache.spark.sql.connector.write.Write | ||
import org.apache.spark.sql.types.{BooleanType, DataType, MetadataBuilder, StringType, StructType} | ||
|
||
/** | ||
|
@@ -65,7 +66,8 @@ case class AppendData( | |
table: NamedRelation, | ||
query: LogicalPlan, | ||
writeOptions: Map[String, String], | ||
isByName: Boolean) extends V2WriteCommand { | ||
isByName: Boolean, | ||
write: Option[Write] = None) extends V2WriteCommand { | ||
override def withNewQuery(newQuery: LogicalPlan): AppendData = copy(query = newQuery) | ||
override def withNewTable(newTable: NamedRelation): AppendData = copy(table = newTable) | ||
} | ||
|
@@ -94,7 +96,8 @@ case class OverwriteByExpression( | |
deleteExpr: Expression, | ||
query: LogicalPlan, | ||
writeOptions: Map[String, String], | ||
isByName: Boolean) extends V2WriteCommand { | ||
isByName: Boolean, | ||
write: Option[Write] = None) extends V2WriteCommand { | ||
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. Making this optional allows us to reuse the same plan before we construct a write and after. Having |
||
override lazy val resolved: Boolean = { | ||
table.resolved && query.resolved && outputResolved && deleteExpr.resolved | ||
} | ||
|
@@ -132,7 +135,8 @@ case class OverwritePartitionsDynamic( | |
table: NamedRelation, | ||
query: LogicalPlan, | ||
writeOptions: Map[String, String], | ||
isByName: Boolean) extends V2WriteCommand { | ||
isByName: Boolean, | ||
write: Option[Write] = None) extends V2WriteCommand { | ||
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. Not related to this PR. I'm thinking that if we should have an optional 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 that's a good idea. 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. Quick question: 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. For |
||
override def withNewQuery(newQuery: LogicalPlan): OverwritePartitionsDynamic = { | ||
copy(query = newQuery) | ||
} | ||
|
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.connector.write; | ||
|
||
import org.apache.spark.annotation.Unstable; | ||
import org.apache.spark.sql.connector.catalog.TableCapability; | ||
import org.apache.spark.sql.sources.InsertableRelation; | ||
|
||
/** | ||
* A logical write that should be executed using V1 InsertableRelation interface. | ||
* <p> | ||
* Tables that have {@link TableCapability#V1_BATCH_WRITE} in the list of their capabilities | ||
* must build {@link V1Write}. | ||
*/ | ||
@Unstable | ||
public interface V1Write extends Write { | ||
InsertableRelation toInsertableRelation(); | ||
} |
This file was deleted.
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -24,8 +24,9 @@ import org.apache.spark.sql.catalyst.analysis.{ResolvedNamespace, ResolvedPartit | |
import org.apache.spark.sql.catalyst.expressions.{And, Expression, NamedExpression, PredicateHelper, SubqueryExpression} | ||
import org.apache.spark.sql.catalyst.planning.PhysicalOperation | ||
import org.apache.spark.sql.catalyst.plans.logical._ | ||
import org.apache.spark.sql.connector.catalog.{CatalogV2Util, StagingTableCatalog, SupportsNamespaces, SupportsPartitionManagement, TableCapability, TableCatalog, TableChange} | ||
import org.apache.spark.sql.connector.catalog.{CatalogV2Util, StagingTableCatalog, SupportsNamespaces, SupportsPartitionManagement, SupportsWrite, TableCapability, TableCatalog, TableChange} | ||
import org.apache.spark.sql.connector.read.streaming.{ContinuousStream, MicroBatchStream} | ||
import org.apache.spark.sql.connector.write.V1Write | ||
import org.apache.spark.sql.execution.{FilterExec, LeafExecNode, LocalTableScanExec, ProjectExec, RowDataSourceScanExec, SparkPlan} | ||
import org.apache.spark.sql.execution.datasources.DataSourceStrategy | ||
import org.apache.spark.sql.execution.streaming.continuous.{WriteToContinuousDataSource, WriteToContinuousDataSourceExec} | ||
|
@@ -195,33 +196,42 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat | |
orCreate = orCreate) :: Nil | ||
} | ||
|
||
case AppendData(r: DataSourceV2Relation, query, writeOptions, _) => | ||
r.table.asWritable match { | ||
case v1 if v1.supports(TableCapability.V1_BATCH_WRITE) => | ||
AppendDataExecV1(v1, writeOptions.asOptions, query, refreshCache(r)) :: Nil | ||
case v2 => | ||
AppendDataExec(v2, writeOptions.asOptions, planLater(query), refreshCache(r)) :: Nil | ||
case AppendData(r @ DataSourceV2Relation(v1: SupportsWrite, _, _, _, _), query, writeOptions, | ||
_, Some(write)) if v1.supports(TableCapability.V1_BATCH_WRITE) => | ||
write match { | ||
case v1Write: V1Write => | ||
AppendDataExecV1(v1, writeOptions.asOptions, query, refreshCache(r), v1Write) :: Nil | ||
case v2Write => | ||
throw new AnalysisException( | ||
s"Table ${v1.name} declares ${TableCapability.V1_BATCH_WRITE} capability but " + | ||
s"${v2Write.getClass.getName} is not an instance of ${classOf[V1Write].getName}") | ||
} | ||
|
||
case OverwriteByExpression(r: DataSourceV2Relation, deleteExpr, query, writeOptions, _) => | ||
// fail if any filter cannot be converted. correctness depends on removing all matching data. | ||
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 removed the filter conversion as it is done earlier now. |
||
val filters = splitConjunctivePredicates(deleteExpr).map { | ||
filter => DataSourceStrategy.translateFilter(deleteExpr, | ||
supportNestedPredicatePushdown = true).getOrElse( | ||
throw new AnalysisException(s"Cannot translate expression to source filter: $filter")) | ||
}.toArray | ||
r.table.asWritable match { | ||
case v1 if v1.supports(TableCapability.V1_BATCH_WRITE) => | ||
OverwriteByExpressionExecV1(v1, filters, writeOptions.asOptions, | ||
query, refreshCache(r)) :: Nil | ||
case v2 => | ||
OverwriteByExpressionExec(v2, filters, | ||
writeOptions.asOptions, planLater(query), refreshCache(r)) :: Nil | ||
case AppendData(r @ DataSourceV2Relation(v2: SupportsWrite, _, _, _, _), query, writeOptions, | ||
_, Some(write)) => | ||
AppendDataExec(v2, writeOptions.asOptions, planLater(query), refreshCache(r), write) :: Nil | ||
|
||
case OverwriteByExpression(r @ DataSourceV2Relation(v1: SupportsWrite, _, _, _, _), _, query, | ||
writeOptions, _, Some(write)) if v1.supports(TableCapability.V1_BATCH_WRITE) => | ||
write match { | ||
case v1Write: V1Write => | ||
OverwriteByExpressionExecV1( | ||
v1, writeOptions.asOptions, query, refreshCache(r), v1Write) :: Nil | ||
case v2Write => | ||
throw new AnalysisException( | ||
s"Table ${v1.name} declares ${TableCapability.V1_BATCH_WRITE} capability but " + | ||
s"${v2Write.getClass.getName} is not an instance of ${classOf[V1Write].getName}") | ||
} | ||
|
||
case OverwritePartitionsDynamic(r: DataSourceV2Relation, query, writeOptions, _) => | ||
case OverwriteByExpression(r @ DataSourceV2Relation(v2: SupportsWrite, _, _, _, _), _, query, | ||
writeOptions, _, Some(write)) => | ||
OverwriteByExpressionExec( | ||
v2, writeOptions.asOptions, planLater(query), refreshCache(r), write) :: Nil | ||
|
||
case OverwritePartitionsDynamic(r: DataSourceV2Relation, query, writeOptions, _, Some(write)) => | ||
OverwritePartitionsDynamicExec( | ||
r.table.asWritable, writeOptions.asOptions, planLater(query), refreshCache(r)) :: Nil | ||
r.table.asWritable, writeOptions.asOptions, planLater(query), | ||
refreshCache(r), write) :: Nil | ||
|
||
case DeleteFromTable(relation, condition) => | ||
relation match { | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -17,17 +17,14 @@ | |
|
||
package org.apache.spark.sql.execution.datasources.v2 | ||
|
||
import java.util.UUID | ||
|
||
import org.apache.spark.SparkException | ||
import org.apache.spark.sql.Dataset | ||
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.connector.catalog.SupportsWrite | ||
import org.apache.spark.sql.connector.write.{LogicalWriteInfoImpl, SupportsOverwrite, SupportsTruncate, V1WriteBuilder, WriteBuilder} | ||
import org.apache.spark.sql.connector.write.V1Write | ||
import org.apache.spark.sql.execution.SparkPlan | ||
import org.apache.spark.sql.sources.{AlwaysTrue, Filter, InsertableRelation} | ||
import org.apache.spark.sql.sources.InsertableRelation | ||
import org.apache.spark.sql.util.CaseInsensitiveStringMap | ||
|
||
/** | ||
|
@@ -39,12 +36,8 @@ case class AppendDataExecV1( | |
table: SupportsWrite, | ||
writeOptions: CaseInsensitiveStringMap, | ||
plan: LogicalPlan, | ||
refreshCache: () => Unit) extends V1FallbackWriters { | ||
|
||
override protected def run(): Seq[InternalRow] = { | ||
writeWithV1(newWriteBuilder().buildForV1Write(), refreshCache = refreshCache) | ||
} | ||
} | ||
refreshCache: () => Unit, | ||
write: V1Write) extends V1FallbackWriters | ||
|
||
/** | ||
* Physical plan node for overwrite into a v2 table with V1 write interfaces. Note that when this | ||
|
@@ -59,29 +52,10 @@ case class AppendDataExecV1( | |
*/ | ||
case class OverwriteByExpressionExecV1( | ||
table: SupportsWrite, | ||
deleteWhere: Array[Filter], | ||
writeOptions: CaseInsensitiveStringMap, | ||
plan: LogicalPlan, | ||
refreshCache: () => Unit) extends V1FallbackWriters { | ||
|
||
private def isTruncate(filters: Array[Filter]): Boolean = { | ||
filters.length == 1 && filters(0).isInstanceOf[AlwaysTrue] | ||
} | ||
|
||
override protected def run(): Seq[InternalRow] = { | ||
newWriteBuilder() match { | ||
case builder: SupportsTruncate if isTruncate(deleteWhere) => | ||
writeWithV1(builder.truncate().asV1Builder.buildForV1Write(), refreshCache = refreshCache) | ||
|
||
case builder: SupportsOverwrite => | ||
writeWithV1(builder.overwrite(deleteWhere).asV1Builder.buildForV1Write(), | ||
refreshCache = refreshCache) | ||
|
||
case _ => | ||
throw new SparkException(s"Table does not support overwrite by expression: $table") | ||
} | ||
} | ||
} | ||
refreshCache: () => Unit, | ||
write: V1Write) extends V1FallbackWriters | ||
|
||
/** Some helper interfaces that use V2 write semantics through the V1 writer interface. */ | ||
sealed trait V1FallbackWriters extends V2CommandExec with SupportsV1Write { | ||
|
@@ -90,23 +64,13 @@ sealed trait V1FallbackWriters extends V2CommandExec with SupportsV1Write { | |
|
||
def table: SupportsWrite | ||
def writeOptions: CaseInsensitiveStringMap | ||
def refreshCache: () => Unit | ||
def write: V1Write | ||
|
||
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 info = LogicalWriteInfoImpl( | ||
queryId = UUID.randomUUID().toString, | ||
schema = plan.schema, | ||
options = writeOptions) | ||
val writeBuilder = table.newWriteBuilder(info) | ||
|
||
writeBuilder.asV1Builder | ||
override def run(): Seq[InternalRow] = { | ||
val writtenRows = writeWithV1(write.toInsertableRelation) | ||
refreshCache() | ||
writtenRows | ||
} | ||
} | ||
|
||
|
@@ -116,12 +80,8 @@ sealed trait V1FallbackWriters extends V2CommandExec with SupportsV1Write { | |
trait SupportsV1Write extends SparkPlan { | ||
def plan: LogicalPlan | ||
|
||
protected def writeWithV1( | ||
relation: InsertableRelation, | ||
refreshCache: () => Unit = () => ()): Seq[InternalRow] = { | ||
protected def writeWithV1(relation: InsertableRelation): Seq[InternalRow] = { | ||
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. Nicely simplified |
||
relation.insert(Dataset.ofRows(sqlContext.sparkSession, plan), overwrite = false) | ||
refreshCache() | ||
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. Refresh moved to 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. cc @sunchao 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. lgtm |
||
|
||
Nil | ||
} | ||
} |
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.
Shall we add
override lazy val resolved = ... && write.isDefined
inV2WriteCommand
? It's safer to make sure that the analyzer creates theWrite
object.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.
Sounds like a good idea but we actually construct the
Write
object in the optimizer after the operator optimization is done to ensure we operate on optimal expressions.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.
ah I see, let's leave it then.