Skip to content

Commit b973fbd

Browse files
committed
remove V1_BATCH_WRITE table capability
1 parent 50cf484 commit b973fbd

File tree

8 files changed

+166
-227
lines changed

8 files changed

+166
-227
lines changed

sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCapability.java

Lines changed: 1 addition & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -89,14 +89,5 @@ public enum TableCapability {
8989
/**
9090
* Signals that the table accepts input of any schema in a write operation.
9191
*/
92-
ACCEPT_ANY_SCHEMA,
93-
94-
/**
95-
* Signals that the table supports append writes using the V1 InsertableRelation interface.
96-
* <p>
97-
* Tables that return this capability must create a V1WriteBuilder and may also support additional
98-
* write modes, like {@link #TRUNCATE}, and {@link #OVERWRITE_BY_FILTER}, but cannot support
99-
* {@link #OVERWRITE_DYNAMIC}.
100-
*/
101-
V1_BATCH_WRITE
92+
ACCEPT_ANY_SCHEMA
10293
}

sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala

Lines changed: 6 additions & 35 deletions
Original file line numberDiff line numberDiff line change
@@ -17,21 +17,19 @@
1717

1818
package org.apache.spark.sql.execution.datasources.v2
1919

20-
import scala.collection.JavaConverters._
2120
import scala.collection.mutable
2221

2322
import org.apache.spark.sql.{AnalysisException, Strategy}
2423
import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, AttributeSet, Expression, PredicateHelper, SubqueryExpression}
2524
import org.apache.spark.sql.catalyst.planning.PhysicalOperation
26-
import org.apache.spark.sql.catalyst.plans.logical.{AlterTable, AppendData, CreateNamespace, CreateTableAsSelect, CreateV2Table, DeleteFromTable, DescribeTable, DropTable, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, RefreshTable, Repartition, ReplaceTable, ReplaceTableAsSelect, SetCatalogAndNamespace, ShowNamespaces, ShowTables}
27-
import org.apache.spark.sql.connector.catalog.{StagingTableCatalog, TableCapability}
25+
import org.apache.spark.sql.catalyst.plans.logical.{AlterTable, CreateNamespace, CreateTableAsSelect, CreateV2Table, DeleteFromTable, DescribeTable, DropTable, LogicalPlan, RefreshTable, Repartition, ReplaceTable, ReplaceTableAsSelect, SetCatalogAndNamespace, ShowNamespaces, ShowTables}
26+
import org.apache.spark.sql.connector.catalog.StagingTableCatalog
2827
import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, SupportsPushDownFilters, SupportsPushDownRequiredColumns}
2928
import org.apache.spark.sql.connector.read.streaming.{ContinuousStream, MicroBatchStream}
3029
import org.apache.spark.sql.execution.{FilterExec, ProjectExec, SparkPlan}
3130
import org.apache.spark.sql.execution.datasources.DataSourceStrategy
3231
import org.apache.spark.sql.execution.streaming.continuous.{ContinuousCoalesceExec, WriteToContinuousDataSource, WriteToContinuousDataSourceExec}
3332
import org.apache.spark.sql.sources
34-
import org.apache.spark.sql.util.CaseInsensitiveStringMap
3533

3634
object DataSourceV2Strategy extends Strategy with PredicateHelper {
3735

@@ -183,14 +181,13 @@ object DataSourceV2Strategy extends Strategy with PredicateHelper {
183181
CreateTableExec(catalog, ident, schema, parts, props, ifNotExists) :: Nil
184182

185183
case CreateTableAsSelect(catalog, ident, parts, query, props, options, ifNotExists) =>
186-
val writeOptions = new CaseInsensitiveStringMap(options.asJava)
187184
catalog match {
188185
case staging: StagingTableCatalog =>
189186
AtomicCreateTableAsSelectExec(
190-
staging, ident, parts, query, planLater(query), props, writeOptions, ifNotExists) :: Nil
187+
staging, ident, parts, query, planLater(query), props, options, ifNotExists) :: Nil
191188
case _ =>
192189
CreateTableAsSelectExec(
193-
catalog, ident, parts, query, planLater(query), props, writeOptions, ifNotExists) :: Nil
190+
catalog, ident, parts, query, planLater(query), props, options, ifNotExists) :: Nil
194191
}
195192

196193
case RefreshTable(catalog, ident) =>
@@ -205,7 +202,6 @@ object DataSourceV2Strategy extends Strategy with PredicateHelper {
205202
}
206203

207204
case ReplaceTableAsSelect(catalog, ident, parts, query, props, options, orCreate) =>
208-
val writeOptions = new CaseInsensitiveStringMap(options.asJava)
209205
catalog match {
210206
case staging: StagingTableCatalog =>
211207
AtomicReplaceTableAsSelectExec(
@@ -215,7 +211,7 @@ object DataSourceV2Strategy extends Strategy with PredicateHelper {
215211
query,
216212
planLater(query),
217213
props,
218-
writeOptions,
214+
options,
219215
orCreate = orCreate) :: Nil
220216
case _ =>
221217
ReplaceTableAsSelectExec(
@@ -225,35 +221,10 @@ object DataSourceV2Strategy extends Strategy with PredicateHelper {
225221
query,
226222
planLater(query),
227223
props,
228-
writeOptions,
224+
options,
229225
orCreate = orCreate) :: Nil
230226
}
231227

232-
case AppendData(r: DataSourceV2Relation, query, writeOptions, _) =>
233-
r.table.asWritable match {
234-
case v1 if v1.supports(TableCapability.V1_BATCH_WRITE) =>
235-
AppendDataExecV1(v1, writeOptions.asOptions, query) :: Nil
236-
case v2 =>
237-
AppendDataExec(v2, writeOptions.asOptions, planLater(query)) :: Nil
238-
}
239-
240-
case OverwriteByExpression(r: DataSourceV2Relation, deleteExpr, query, writeOptions, _) =>
241-
// fail if any filter cannot be converted. correctness depends on removing all matching data.
242-
val filters = splitConjunctivePredicates(deleteExpr).map {
243-
filter => DataSourceStrategy.translateFilter(deleteExpr).getOrElse(
244-
throw new AnalysisException(s"Cannot translate expression to source filter: $filter"))
245-
}.toArray
246-
r.table.asWritable match {
247-
case v1 if v1.supports(TableCapability.V1_BATCH_WRITE) =>
248-
OverwriteByExpressionExecV1(v1, filters, writeOptions.asOptions, query) :: Nil
249-
case v2 =>
250-
OverwriteByExpressionExec(v2, filters, writeOptions.asOptions, planLater(query)) :: Nil
251-
}
252-
253-
case OverwritePartitionsDynamic(r: DataSourceV2Relation, query, writeOptions, _) =>
254-
OverwritePartitionsDynamicExec(
255-
r.table.asWritable, writeOptions.asOptions, planLater(query)) :: Nil
256-
257228
case DeleteFromTable(r: DataSourceV2Relation, condition) =>
258229
if (condition.exists(SubqueryExpression.hasSubquery)) {
259230
throw new AnalysisException(

sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/TableCapabilityCheck.scala

Lines changed: 3 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -20,7 +20,6 @@ package org.apache.spark.sql.execution.datasources.v2
2020
import org.apache.spark.sql.AnalysisException
2121
import org.apache.spark.sql.catalyst.expressions.Literal
2222
import org.apache.spark.sql.catalyst.plans.logical.{AppendData, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic}
23-
import org.apache.spark.sql.connector.catalog.{SupportsWrite, Table}
2423
import org.apache.spark.sql.connector.catalog.TableCapability._
2524
import org.apache.spark.sql.execution.streaming.{StreamingRelation, StreamingRelationV2}
2625
import org.apache.spark.sql.types.BooleanType
@@ -33,10 +32,6 @@ object TableCapabilityCheck extends (LogicalPlan => Unit) {
3332

3433
private def failAnalysis(msg: String): Unit = throw new AnalysisException(msg)
3534

36-
private def supportsBatchWrite(table: Table): Boolean = {
37-
table.supportsAny(BATCH_WRITE, V1_BATCH_WRITE)
38-
}
39-
4035
override def apply(plan: LogicalPlan): Unit = {
4136
plan foreach {
4237
case r: DataSourceV2Relation if !r.table.supports(BATCH_READ) =>
@@ -48,7 +43,7 @@ object TableCapabilityCheck extends (LogicalPlan => Unit) {
4843

4944
// TODO: check STREAMING_WRITE capability. It's not doable now because we don't have a
5045
// a logical plan for streaming write.
51-
case AppendData(r: DataSourceV2Relation, _, _, _) if !supportsBatchWrite(r.table) =>
46+
case AppendData(r: DataSourceV2Relation, _, _, _) if !r.table.supports(BATCH_WRITE) =>
5247
failAnalysis(s"Table ${r.table.name()} does not support append in batch mode.")
5348

5449
case OverwritePartitionsDynamic(r: DataSourceV2Relation, _, _, _)
@@ -58,13 +53,13 @@ object TableCapabilityCheck extends (LogicalPlan => Unit) {
5853
case OverwriteByExpression(r: DataSourceV2Relation, expr, _, _, _) =>
5954
expr match {
6055
case Literal(true, BooleanType) =>
61-
if (!supportsBatchWrite(r.table) ||
56+
if (!r.table.supports(BATCH_WRITE) ||
6257
!r.table.supportsAny(TRUNCATE, OVERWRITE_BY_FILTER)) {
6358
failAnalysis(
6459
s"Table ${r.table.name()} does not support truncate in batch mode.")
6560
}
6661
case _ =>
67-
if (!supportsBatchWrite(r.table) || !r.table.supports(OVERWRITE_BY_FILTER)) {
62+
if (!r.table.supports(BATCH_WRITE) || !r.table.supports(OVERWRITE_BY_FILTER)) {
6863
failAnalysis(s"Table ${r.table.name()} does not support " +
6964
"overwrite by filter in batch mode.")
7065
}

sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V1FallbackWriters.scala

Lines changed: 13 additions & 53 deletions
Original file line numberDiff line numberDiff line change
@@ -17,32 +17,27 @@
1717

1818
package org.apache.spark.sql.execution.datasources.v2
1919

20-
import java.util.UUID
21-
22-
import org.apache.spark.SparkException
2320
import org.apache.spark.rdd.RDD
2421
import org.apache.spark.sql.Dataset
2522
import org.apache.spark.sql.catalyst.InternalRow
2623
import org.apache.spark.sql.catalyst.expressions.Attribute
2724
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
28-
import org.apache.spark.sql.connector.catalog.SupportsWrite
29-
import org.apache.spark.sql.connector.write.{SupportsOverwrite, SupportsTruncate, V1WriteBuilder, WriteBuilder}
30-
import org.apache.spark.sql.execution.SparkPlan
31-
import org.apache.spark.sql.sources.{AlwaysTrue, Filter, InsertableRelation}
32-
import org.apache.spark.sql.util.CaseInsensitiveStringMap
25+
import org.apache.spark.sql.execution.{LeafExecNode, SparkPlan}
26+
import org.apache.spark.sql.sources.{Filter, InsertableRelation}
3327

3428
/**
3529
* Physical plan node for append into a v2 table using V1 write interfaces.
3630
*
3731
* Rows in the output data set are appended.
3832
*/
3933
case class AppendDataExecV1(
40-
table: SupportsWrite,
41-
writeOptions: CaseInsensitiveStringMap,
42-
plan: LogicalPlan) extends V1FallbackWriters {
34+
v1Relation: InsertableRelation,
35+
plan: LogicalPlan) extends LeafExecNode with SupportsV1Write {
36+
37+
override def output: Seq[Attribute] = Nil
4338

4439
override protected def doExecute(): RDD[InternalRow] = {
45-
writeWithV1(newWriteBuilder().buildForV1Write())
40+
writeWithV1(v1Relation)
4641
}
4742
}
4843

@@ -58,57 +53,22 @@ case class AppendDataExecV1(
5853
* AlwaysTrue to delete all rows.
5954
*/
6055
case class OverwriteByExpressionExecV1(
61-
table: SupportsWrite,
62-
deleteWhere: Array[Filter],
63-
writeOptions: CaseInsensitiveStringMap,
64-
plan: LogicalPlan) extends V1FallbackWriters {
65-
66-
private def isTruncate(filters: Array[Filter]): Boolean = {
67-
filters.length == 1 && filters(0).isInstanceOf[AlwaysTrue]
68-
}
69-
70-
override protected def doExecute(): RDD[InternalRow] = {
71-
newWriteBuilder() match {
72-
case builder: SupportsTruncate if isTruncate(deleteWhere) =>
73-
writeWithV1(builder.truncate().asV1Builder.buildForV1Write())
74-
75-
case builder: SupportsOverwrite =>
76-
writeWithV1(builder.overwrite(deleteWhere).asV1Builder.buildForV1Write())
56+
v1Relation: InsertableRelation,
57+
deleteWhere: Seq[Filter],
58+
plan: LogicalPlan) extends LeafExecNode with SupportsV1Write {
7759

78-
case _ =>
79-
throw new SparkException(s"Table does not support overwrite by expression: $table")
80-
}
81-
}
82-
}
83-
84-
/** Some helper interfaces that use V2 write semantics through the V1 writer interface. */
85-
sealed trait V1FallbackWriters extends SupportsV1Write {
8660
override def output: Seq[Attribute] = Nil
87-
override final def children: Seq[SparkPlan] = Nil
88-
89-
def table: SupportsWrite
90-
def writeOptions: CaseInsensitiveStringMap
9161

92-
protected implicit class toV1WriteBuilder(builder: WriteBuilder) {
93-
def asV1Builder: V1WriteBuilder = builder match {
94-
case v1: V1WriteBuilder => v1
95-
case other => throw new IllegalStateException(
96-
s"The returned writer ${other} was no longer a V1WriteBuilder.")
97-
}
98-
}
99-
100-
protected def newWriteBuilder(): V1WriteBuilder = {
101-
val writeBuilder = table.newWriteBuilder(writeOptions)
102-
.withInputDataSchema(plan.schema)
103-
.withQueryId(UUID.randomUUID().toString)
104-
writeBuilder.asV1Builder
62+
override protected def doExecute(): RDD[InternalRow] = {
63+
writeWithV1(v1Relation)
10564
}
10665
}
10766

10867
/**
10968
* A trait that allows Tables that use V1 Writer interfaces to append data.
11069
*/
11170
trait SupportsV1Write extends SparkPlan {
71+
11272
// TODO: We should be able to work on SparkPlans at this point.
11373
def plan: LogicalPlan
11474

Lines changed: 95 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,95 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
18+
package org.apache.spark.sql.execution.datasources.v2
19+
20+
import java.util.UUID
21+
22+
import org.apache.spark.sql.{AnalysisException, Strategy}
23+
import org.apache.spark.sql.catalyst.expressions.PredicateHelper
24+
import org.apache.spark.sql.catalyst.plans.logical.{AppendData, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic}
25+
import org.apache.spark.sql.connector.catalog.Table
26+
import org.apache.spark.sql.connector.write.{SupportsDynamicOverwrite, SupportsOverwrite, SupportsTruncate, V1WriteBuilder, WriteBuilder}
27+
import org.apache.spark.sql.execution.SparkPlan
28+
import org.apache.spark.sql.execution.datasources.DataSourceStrategy
29+
import org.apache.spark.sql.sources.{AlwaysTrue, Filter}
30+
import org.apache.spark.sql.types.StructType
31+
32+
object V2WriteStrategy extends Strategy with PredicateHelper {
33+
import DataSourceV2Implicits._
34+
35+
override def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match {
36+
case WriteToDataSourceV2(writer, query) =>
37+
WriteToDataSourceV2Exec(writer, planLater(query)) :: Nil
38+
39+
case AppendData(r: DataSourceV2Relation, query, writeOptions, _) =>
40+
val writeBuilder = newWriteBuilder(r.table, writeOptions, query.schema)
41+
writeBuilder match {
42+
case v1: V1WriteBuilder =>
43+
AppendDataExecV1(v1.buildForV1Write(), query) :: Nil
44+
case _ =>
45+
AppendDataExec(writeBuilder.buildForBatch(), planLater(query)) :: Nil
46+
}
47+
48+
case OverwriteByExpression(r: DataSourceV2Relation, deleteExpr, query, writeOptions, _) =>
49+
// fail if any filter cannot be converted. correctness depends on removing all matching data.
50+
val filters = splitConjunctivePredicates(deleteExpr).map {
51+
filter => DataSourceStrategy.translateFilter(deleteExpr).getOrElse(
52+
throw new AnalysisException(s"Cannot translate expression to source filter: $filter"))
53+
}.toArray
54+
55+
val writeBuilder = newWriteBuilder(r.table, writeOptions, query.schema)
56+
val configured = writeBuilder match {
57+
case builder: SupportsTruncate if isTruncate(filters) => builder.truncate()
58+
case builder: SupportsOverwrite => builder.overwrite(filters)
59+
case _ =>
60+
throw new IllegalArgumentException(
61+
s"Table does not support overwrite by expression: ${r.table.name}")
62+
}
63+
64+
configured match {
65+
case v1: V1WriteBuilder =>
66+
OverwriteByExpressionExecV1(v1.buildForV1Write(), filters, query) :: Nil
67+
case _ =>
68+
OverwriteByExpressionExec(configured.buildForBatch(), filters, planLater(query)) :: Nil
69+
}
70+
71+
case OverwritePartitionsDynamic(r: DataSourceV2Relation, query, writeOptions, _) =>
72+
val writeBuilder = newWriteBuilder(r.table, writeOptions, query.schema)
73+
val configured = writeBuilder match {
74+
case builder: SupportsDynamicOverwrite =>
75+
builder.overwriteDynamicPartitions()
76+
case _ =>
77+
throw new IllegalArgumentException(
78+
s"Table does not support dynamic partition overwrite: ${r.table.name}")
79+
}
80+
OverwritePartitionsDynamicExec(configured.buildForBatch(), planLater(query)) :: Nil
81+
}
82+
83+
def newWriteBuilder(
84+
table: Table,
85+
options: Map[String, String],
86+
inputSchema: StructType): WriteBuilder = {
87+
table.asWritable.newWriteBuilder(options.asOptions)
88+
.withInputDataSchema(inputSchema)
89+
.withQueryId(UUID.randomUUID().toString)
90+
}
91+
92+
private def isTruncate(filters: Array[Filter]): Boolean = {
93+
filters.length == 1 && filters(0).isInstanceOf[AlwaysTrue]
94+
}
95+
}

0 commit comments

Comments
 (0)