alexeykudinkin commented on code in PR #5943:
URL: https://github.com/apache/hudi/pull/5943#discussion_r931566562


##########
hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/parser/HoodieCommonSqlParser.scala:
##########
@@ -57,6 +57,14 @@ class HoodieCommonSqlParser(session: SparkSession, delegate: 
ParserInterface)
 
   override def parseDataType(sqlText: String): DataType = 
delegate.parseDataType(sqlText)
 
+  /* SPARK-37266 Added parseQuery to ParserInterface in Spark 3.3.0. This is a 
patch to prevent
+   hackers from tampering text with persistent view, it won't be called in 
older Spark
+   Don't mark this as override for backward compatibility
+   Can't use sparkExtendedParser directly here due to the same reason */

Review Comment:
   Sorry, but i can't understand the java-doc: can you please elaborate on why 
this is here? 
   
    - What exactly are we trying to prevent from happening? 
    - What BWC are we referring to? 



##########
hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieAnalysis.scala:
##########
@@ -421,12 +433,10 @@ case class HoodieResolveReferences(sparkSession: 
SparkSession) extends Rule[Logi
       UpdateTable(table, resolvedAssignments, resolvedCondition)
 
     // Resolve Delete Table
-    case DeleteFromTable(table, condition)
+    case dft @ DeleteFromTable(table, condition)
       if sparkAdapter.isHoodieTable(table, sparkSession) && table.resolved =>
-      // Resolve condition
-      val resolvedCondition = condition.map(resolveExpressionFrom(table)(_))
-      // Return the resolved DeleteTable
-      DeleteFromTable(table, resolvedCondition)
+      val resolveExpression = resolveExpressionFrom(table, None)_

Review Comment:
   I'd suggest we keep syntax as it was (with parenthesis)



##########
hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/parser/HoodieCommonSqlParser.scala:
##########
@@ -57,6 +57,14 @@ class HoodieCommonSqlParser(session: SparkSession, delegate: 
ParserInterface)
 
   override def parseDataType(sqlText: String): DataType = 
delegate.parseDataType(sqlText)
 
+  /* SPARK-37266 Added parseQuery to ParserInterface in Spark 3.3.0. This is a 
patch to prevent
+   hackers from tampering text with persistent view, it won't be called in 
older Spark
+   Don't mark this as override for backward compatibility
+   Can't use sparkExtendedParser directly here due to the same reason */
+  def parseQuery(sqlText: String): LogicalPlan = parse(sqlText) { parser =>

Review Comment:
   Why are we doing double-parsing?



##########
hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_3Adapter.scala:
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.adapter
+
+import org.apache.avro.Schema
+import org.apache.hudi.Spark33HoodieFileScanRDD
+import org.apache.spark.sql.avro._
+import org.apache.spark.sql.catalyst.expressions.{AttributeReference, 
Expression}
+import org.apache.spark.sql.catalyst.parser.ParserInterface
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.execution.datasources.{FilePartition, FileScanRDD, 
PartitionedFile}
+import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, 
Spark33HoodieParquetFileFormat}
+import org.apache.spark.sql.parser.HoodieSpark3_3ExtendedSqlParser
+import org.apache.spark.sql.types.{DataType, StructType}
+import org.apache.spark.sql.{HoodieCatalystExpressionUtils, 
HoodieCatalystPlansUtils, HoodieSpark33CatalystPlanUtils, 
HoodieSpark33CatalystExpressionUtils, SparkSession}
+
+/**
+ * Implementation of [[SparkAdapter]] for Spark 3.3.x branch
+ */
+class Spark3_3Adapter extends BaseSpark3Adapter {
+
+  override def getCatalystExpressionUtils: HoodieCatalystExpressionUtils = 
HoodieSpark33CatalystExpressionUtils
+
+  override def getCatalystPlanUtils: HoodieCatalystPlansUtils = 
HoodieSpark33CatalystPlanUtils
+
+  override def createAvroSerializer(rootCatalystType: DataType, rootAvroType: 
Schema, nullable: Boolean): HoodieAvroSerializer =
+    new HoodieSpark3_3AvroSerializer(rootCatalystType, rootAvroType, nullable)
+
+  override def createAvroDeserializer(rootAvroType: Schema, rootCatalystType: 
DataType): HoodieAvroDeserializer =
+    new HoodieSpark3_3AvroDeserializer(rootAvroType, rootCatalystType)
+
+  override def createExtendedSparkParser: Option[(SparkSession, 
ParserInterface) => ParserInterface] = {
+    Some(
+      (spark: SparkSession, delegate: ParserInterface) => new 
HoodieSpark3_3ExtendedSqlParser(spark, delegate)
+    )
+  }
+
+  override def createHoodieParquetFileFormat(appendPartitionValues: Boolean): 
Option[ParquetFileFormat] = {
+    Some(new Spark33HoodieParquetFileFormat(appendPartitionValues))
+  }
+
+  override def createHoodieFileScanRDD(sparkSession: SparkSession,
+                                       readFunction: PartitionedFile => 
Iterator[InternalRow],
+                                       filePartitions: Seq[FilePartition],
+                                       readDataSchema: StructType,
+                                       metadataColumns: 
Seq[AttributeReference] = Seq.empty): FileScanRDD = {
+    new Spark33HoodieFileScanRDD(sparkSession, readFunction, filePartitions, 
readDataSchema, metadataColumns)
+  }
+
+  override def resolveDeleteFromTable(deleteFromTable: Command,
+                                      resolveExpression: Expression => 
Expression): DeleteFromTable = {
+    val deleteFromTableCommand = deleteFromTable.asInstanceOf[DeleteFromTable]
+    DeleteFromTable(deleteFromTableCommand.table, 
resolveExpression(deleteFromTableCommand.condition))
+  }
+
+  override def extractCondition(deleteFromTable: Command): Expression = {
+    deleteFromTable.asInstanceOf[DeleteFromTable].condition
+  }
+
+  override def getQueryParserFromExtendedSqlParser(session: SparkSession, 
delegate: ParserInterface,
+                                                   sqlText: String): 
LogicalPlan = {
+    new HoodieSpark3_3ExtendedSqlParser(session, delegate).parseQuery(sqlText)

Review Comment:
   This is not a query parser -- this is already parsed query



##########
hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/adapter/Spark2Adapter.scala:
##########
@@ -122,6 +123,30 @@ class Spark2Adapter extends SparkAdapter {
     InterpretedPredicate.create(e)
   }
 
+  override def createHoodieFileScanRDD(sparkSession: SparkSession,
+                                       readFunction: PartitionedFile => 
Iterator[InternalRow],
+                                       filePartitions: Seq[FilePartition],
+                                       readDataSchema: StructType,
+                                       metadataColumns: 
Seq[AttributeReference] = Seq.empty): FileScanRDD = {
+    new Spark2HoodieFileScanRDD(sparkSession, readFunction, filePartitions)
+  }
+
+  override def resolveDeleteFromTable(deleteFromTable: Command,

Review Comment:
   Given that we have `extractCondition` we can get rid of 
`resolveDeleteFromTable`



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to