viirya commented on a change in pull request #33008:
URL: https://github.com/apache/spark/pull/33008#discussion_r659253107



##########
File path: 
sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/SupportsDelta.java
##########
@@ -0,0 +1,35 @@
+/*
+ * 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.Experimental;
+import org.apache.spark.sql.connector.expressions.NamedReference;
+
+/**
+ * A mix-in interface for {@link RowLevelOperation}. Data sources can 
implement this interface
+ * to indicate they support handling deltas of rows.
+ *
+ * @since 3.2.0
+ */
+@Experimental
+public interface SupportsDelta extends RowLevelOperation {

Review comment:
       Do we also need an interface for those that need to replace groups?

##########
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteDeleteFromTable.scala
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.catalyst.analysis
+
+import java.util.UUID
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.InternalRowProjection
+import org.apache.spark.sql.catalyst.analysis.RowDeltaUtils._
+import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference, 
EqualNullSafe, Expression, Literal, Not}
+import org.apache.spark.sql.catalyst.plans.logical.{DeleteFromTable, Filter, 
LogicalPlan, Project, ReplaceData, WriteDelta}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.catalog.{SupportsDelete, 
SupportsRowLevelOperations}
+import org.apache.spark.sql.connector.expressions.{FieldReference, 
NamedReference}
+import org.apache.spark.sql.connector.write.{LogicalWriteInfo, 
LogicalWriteInfoImpl, RowLevelOperation, RowLevelOperationTable, SupportsDelta}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command.DELETE
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.types.{BooleanType, StructType}
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+/**
+ * Assigns a rewrite plan for v2 tables that support rewriting data to handle 
DELETE statements.
+ *
+ * If a table implements [[SupportsDelete]] and 
[[SupportsRowLevelOperations]], we assign a rewrite
+ * plan but the optimizer will check whether this particular DELETE statement 
can be handled
+ * by simply passing delete filters to the connector. If yes, the optimizer 
will then discard
+ * the rewrite plan.
+ */
+object RewriteDeleteFromTable extends Rule[LogicalPlan] {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+    case delete @ DeleteFromTable(aliasedTable, cond, None) if delete.resolved 
=>
+      EliminateSubqueryAliases(aliasedTable) match {
+        case relation @ DataSourceV2Relation(table: 
SupportsRowLevelOperations, _, _, _, _) =>
+          val rowLevelOperation = buildRowLevelOperation(table)
+          val rowLevelOperationTable = RowLevelOperationTable(table, 
rowLevelOperation)
+          val rewritePlan = rowLevelOperation match {
+            case _: SupportsDelta => buildWriteDeltaPlan(relation, 
rowLevelOperationTable, cond)
+            case _ => buildReplaceDataPlan(relation, rowLevelOperationTable, 
cond)
+          }
+          // keep the original relation in DELETE so that we can attempt to 
delete with metadata
+          DeleteFromTable(relation, cond, Some(rewritePlan))
+
+        case DataSourceV2Relation(_: SupportsDelete, _, _, _, _) =>
+          // don't assign a rewrite plan as the table supports deletes only 
with filters
+          delete
+
+        case DataSourceV2Relation(t, _, _, _, _) =>
+          throw new AnalysisException(s"Table $t does not support DELETE 
statements")
+
+        case _ =>
+          delete
+      }
+  }
+
+  // build a rewrite plan for sources that support replacing groups of data 
(e.g. files, partitions)
+  private def buildReplaceDataPlan(
+      relation: DataSourceV2Relation,
+      table: RowLevelOperationTable,
+      cond: Option[Expression]): LogicalPlan = {
+
+    // resolve all columns needed for DELETE including metadata columns for 
grouping data on write
+    val requiredWriteAttrs = resolveRefs(relation, 
table.operation.requiredWriteAttributes)
+
+    // construct a scan relation
+    val scanAttrs = toScanAttrs(relation, requiredWriteAttrs)
+    val scanRelation = relation.copy(table = table, output = scanAttrs)
+
+    // construct a plan that contains unmatched rows in matched groups that 
must be carried over
+    // such rows do not match the condition but have to be copied over as the 
source can replace
+    // only groups of rows
+    val deleteCond = cond.getOrElse(Literal.TrueLiteral)
+    val remainingRowsFilter = Not(EqualNullSafe(deleteCond, Literal(true, 
BooleanType)))
+    val remainingRowsPlan = Filter(remainingRowsFilter, scanRelation)
+
+    // pass only required columns for unmatched rows to the writer
+    val writeRelation = relation.copy(table = table, output = 
requiredWriteAttrs)
+    ReplaceData(writeRelation, remainingRowsPlan, relation)
+  }
+
+  // build a rewrite plan for sources that support row-level changes
+  private def buildWriteDeltaPlan(
+      relation: DataSourceV2Relation,
+      table: RowLevelOperationTable,
+      cond: Option[Expression]): LogicalPlan = {
+
+    val merge = table.operation.asInstanceOf[SupportsDelta]
+
+    // resolve all columns needed for merge (e.g. row ID and columns for 
grouping data on write)
+    val requiredWriteAttrs = resolveRefs(relation, 
table.operation.requiredWriteAttributes)
+    val rowIdAttrs = resolveRefs(relation, merge.rowId)
+
+    // construct a scan relation and include all required columns
+    val scanAttrs = toScanAttrs(relation, requiredWriteAttrs)
+    val scanRelation = relation.copy(table = table, output = scanAttrs)
+
+    // construct a plan that only contains records to delete
+    val deleteCond = cond.getOrElse(Literal.TrueLiteral)
+    val deletedRowsPlan = Filter(deleteCond, scanRelation)
+    val operation = Alias(Literal(DELETE_OPERATION), OPERATION_COLUMN)()
+    val project = Project(operation +: requiredWriteAttrs, deletedRowsPlan)
+
+    // pass operation and required columns to the writer as we are working 
with deltas
+    val operationAttr = resolveRef(project, FieldReference(OPERATION_COLUMN))
+    val writeRelationAttrs = operationAttr +: requiredWriteAttrs

Review comment:
       `requiredWriteAttrs` shared between scan and write relations? Do we need 
to use new expr ids for write relation?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org

Reply via email to