Skip to content

Commit 620e6f5

Browse files
author
xy_xin
committed
Make SupportsDelete to be simple mix-in of DSV2
1 parent bc9daf9 commit 620e6f5

File tree

9 files changed

+59
-172
lines changed

9 files changed

+59
-172
lines changed
Lines changed: 39 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,39 @@
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.sources.v2;
19+
20+
import org.apache.spark.sql.sources.Filter;
21+
22+
public interface SupportsDelete {
23+
/**
24+
* Delete data from a data source table that matches filter expressions.
25+
* <p>
26+
* Rows are deleted from the data source iff all of the filter expressions match. That is, the
27+
* expressions must be interpreted as a set of filters that are ANDed together.
28+
* <p>
29+
* Implementations may reject a delete operation if the delete isn't possible without significant
30+
* effort. For example, partitioned data sources may reject deletes that do not filter by
31+
* partition columns because the filter may require rewriting files without deleted records.
32+
* To reject a delete implementations should throw {@link IllegalArgumentException} with a clear
33+
* error message that identifies which expression was rejected.
34+
*
35+
* @param filters filter expressions, used to select rows to delete when all expressions match
36+
* @throws IllegalArgumentException If the delete is rejected due to required effort
37+
*/
38+
void deleteWhere(Filter[] filters);
39+
}

sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/SupportsMaintenance.java

Lines changed: 0 additions & 35 deletions
This file was deleted.

sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/maintain/Maintainer.java

Lines changed: 0 additions & 31 deletions
This file was deleted.

sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/maintain/MaintainerBuilder.java

Lines changed: 0 additions & 40 deletions
This file was deleted.

sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/maintain/SupportsDelete.java

Lines changed: 0 additions & 28 deletions
This file was deleted.

sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Implicits.scala

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -40,12 +40,12 @@ object DataSourceV2Implicits {
4040
}
4141
}
4242

43-
def asMaintainable: SupportsMaintenance = {
43+
def asDeletable: SupportsDelete = {
4444
table match {
45-
case support: SupportsMaintenance =>
45+
case support: SupportsDelete =>
4646
support
4747
case _ =>
48-
throw new AnalysisException(s"Table does not support maintenance: ${table.name}")
48+
throw new AnalysisException(s"Table does not support deletes: ${table.name}")
4949
}
5050
}
5151

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

Lines changed: 2 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -23,8 +23,7 @@ import scala.collection.mutable
2323
import org.apache.spark.sql.{AnalysisException, Strategy}
2424
import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, AttributeSet, Expression, PredicateHelper, SubqueryExpression}
2525
import org.apache.spark.sql.catalyst.planning.PhysicalOperation
26-
import org.apache.spark.sql.catalyst.plans.logical.{AlterTable, AppendData, CreateTableAsSelect, CreateV2Table, DropTable, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, Repartition}
27-
import org.apache.spark.sql.catalyst.plans.logical._
26+
import org.apache.spark.sql.catalyst.plans.logical.{AlterTable, AppendData, CreateTableAsSelect, CreateV2Table, DeleteFromTable, DropTable, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, Repartition}
2827
import org.apache.spark.sql.execution.{FilterExec, ProjectExec, SparkPlan}
2928
import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, LogicalRelation}
3029
import org.apache.spark.sql.execution.streaming.continuous.{ContinuousCoalesceExec, WriteToContinuousDataSource, WriteToContinuousDataSourceExec}
@@ -191,7 +190,7 @@ object DataSourceV2Strategy extends Strategy with PredicateHelper {
191190
filter => DataSourceStrategy.translateFilter(deleteExpr).getOrElse(
192191
throw new AnalysisException(s"Cannot translate expression to source filter: $filter"))
193192
}.toArray
194-
DeleteFromTableExec(r.table.asMaintainable, r.options, filters, planLater(r)) :: Nil
193+
DeleteFromTableExec(r.table.asDeletable, r.options, filters, planLater(r)) :: Nil
195194

196195
case WriteToContinuousDataSource(writer, query) =>
197196
WriteToContinuousDataSourceExec(writer, planLater(query)) :: Nil

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

Lines changed: 3 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -17,30 +17,23 @@
1717

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

20-
import org.apache.spark.SparkException
2120
import org.apache.spark.rdd.RDD
2221
import org.apache.spark.sql.catalyst.InternalRow
2322
import org.apache.spark.sql.catalyst.expressions.Attribute
2423
import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode}
2524
import org.apache.spark.sql.sources.Filter
26-
import org.apache.spark.sql.sources.v2.SupportsMaintenance
27-
import org.apache.spark.sql.sources.v2.maintain.SupportsDelete
25+
import org.apache.spark.sql.sources.v2.SupportsDelete
2826
import org.apache.spark.sql.util.CaseInsensitiveStringMap
2927

3028
case class DeleteFromTableExec(
31-
table: SupportsMaintenance,
29+
table: SupportsDelete,
3230
options: CaseInsensitiveStringMap,
3331
deleteWhere: Array[Filter],
3432
query: SparkPlan) extends UnaryExecNode {
3533

3634
override protected def doExecute(): RDD[InternalRow] = {
37-
table.newMaintainerBuilder(options).build() match {
38-
case maintainer: SupportsDelete =>
39-
maintainer.delete(deleteWhere)
40-
case _ =>
41-
throw new SparkException(s"Table does not support delete: $table")
42-
}
4335

36+
table.deleteWhere(deleteWhere)
4437
sparkContext.emptyRDD
4538
}
4639

sql/core/src/test/scala/org/apache/spark/sql/sources/v2/SimpleInMemoryTable.scala

Lines changed: 12 additions & 22 deletions
Original file line numberDiff line numberDiff line change
@@ -26,7 +26,6 @@ import org.apache.spark.sql.{Column, SparkSession}
2626
import org.apache.spark.sql.catalyst.InternalRow
2727
import org.apache.spark.sql.catalyst.expressions.{EqualNullSafe, Literal, Not}
2828
import org.apache.spark.sql.sources.Filter
29-
import org.apache.spark.sql.sources.v2.maintain.{Maintainer, MaintainerBuilder, SupportsDelete}
3029
import org.apache.spark.sql.sources.v2.reader._
3130
import org.apache.spark.sql.sources.v2.writer._
3231
import org.apache.spark.sql.types._
@@ -41,7 +40,7 @@ private[v2] class InMemoryTable(
4140
val name: String,
4241
val schema: StructType,
4342
override val properties: util.Map[String, String])
44-
extends Table with SupportsRead with SupportsWrite with SupportsMaintenance {
43+
extends Table with SupportsRead with SupportsWrite with SupportsDelete {
4544

4645
def this(
4746
name: String,
@@ -92,26 +91,6 @@ private[v2] class InMemoryTable(
9291
}
9392
}
9493

95-
override def newMaintainerBuilder(options: CaseInsensitiveStringMap): MaintainerBuilder = {
96-
() => {
97-
Delete
98-
}
99-
}
100-
101-
private object Delete extends Maintainer with SupportsDelete {
102-
103-
override def delete(filters: Array[Filter]): Unit = {
104-
val filtered = data.map {
105-
rows =>
106-
val newRows = filter(rows.rows, filters)
107-
val newBufferedRows = new BufferedRows()
108-
newBufferedRows.rows.appendAll(newRows)
109-
newBufferedRows
110-
}.filter(_.rows.nonEmpty)
111-
replaceData(filtered)
112-
}
113-
}
114-
11594
def filter(rows: mutable.ArrayBuffer[InternalRow],
11695
filters: Array[Filter]): Array[InternalRow] = {
11796
if (rows.isEmpty) {
@@ -164,6 +143,17 @@ private[v2] class InMemoryTable(
164143
override def abort(messages: Array[WriterCommitMessage]): Unit = {
165144
}
166145
}
146+
147+
override def deleteWhere(filters: Array[Filter]): Unit = {
148+
val filtered = data.map {
149+
rows =>
150+
val newRows = filter(rows.rows, filters)
151+
val newBufferedRows = new BufferedRows()
152+
newBufferedRows.rows.appendAll(newRows)
153+
newBufferedRows
154+
}.filter(_.rows.nonEmpty)
155+
replaceData(filtered)
156+
}
167157
}
168158

169159
private class BufferedRows extends WriterCommitMessage with InputPartition with Serializable {

0 commit comments

Comments
 (0)