You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by GitBox <gi...@apache.org> on 2022/07/20 23:37:17 UTC

[GitHub] [hudi] alexeykudinkin commented on a diff in pull request #5943: [HUDI-4186] Support Hudi with Spark 3.3.0

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


##########
hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala:
##########
@@ -171,12 +170,32 @@ trait SparkAdapter extends Serializable {
   }
 
   /**
-    * Create instance of [[ParquetFileFormat]]
-    */
+   * Create instance of [[ParquetFileFormat]]
+   */
   def createHoodieParquetFileFormat(appendPartitionValues: Boolean): Option[ParquetFileFormat]
 
   /**
    * Create instance of [[InterpretedPredicate]]
    */
   def createInterpretedPredicate(e: Expression): InterpretedPredicate
+
+  /**
+   * Create instance of [[HoodieFileScanRDD]]
+   */
+  def createHoodieFileScanRDD(@transient sparkSession: SparkSession,

Review Comment:
   There's no point in annotating `@transient` method params



##########
hudi-spark-datasource/hudi-spark3.2.x/src/main/antlr4/imports/SqlBase.g4:
##########
@@ -0,0 +1,1908 @@
+/*
+ * Licensed 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.
+ *
+ * This file is an adaptation of Presto's presto-parser/src/main/antlr4/com/facebook/presto/sql/parser/SqlBase.g4 grammar.
+ */
+
+// The parser file is forked from spark 3.2.0's SqlBase.g4.
+grammar SqlBase;

Review Comment:
   Did this change from Spark 3.2?



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/BaseFileOnlyRelation.scala:
##########
@@ -91,7 +93,9 @@ class BaseFileOnlyRelation(sqlContext: SQLContext,
       hadoopConf = HoodieDataSourceHelper.getConfigurationWithInternalSchema(new Configuration(conf), requiredSchema.internalSchema, metaClient.getBasePath, validCommits)
     )
 
-    new HoodieFileScanRDD(sparkSession, baseFileReader, fileSplits)
+    // SPARK-37273 FileScanRDD constructor changed in SPARK 3.3

Review Comment:
   I think we can spare this comment for just the adapter



##########
hudi-spark-datasource/README.md:
##########
@@ -21,17 +21,19 @@ This repo contains the code that integrate Hudi with Spark. The repo is split in
 
 `hudi-spark`
 `hudi-spark2`
-`hudi-spark3`
 `hudi-spark3.1.x`
+`hudi-spark3.2.x`
+`hudi-spark3`

Review Comment:
   Let's fix naming in this PR to make sure we avoid the need to rename the module and move 10k LOC of code when we'd plan to support next Spark version.
   
   Let's call it in-line with the other modules -- `hudi-spark3.3.x`



##########
hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/adapter/Spark3_3Adapter.scala:
##########
@@ -0,0 +1,92 @@
+/*
+ * 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, Literal}
+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, HoodieSpark3_3CatalystExpressionUtils, SparkSession}
+
+/**
+ * Implementation of [[SparkAdapter]] for Spark 3.3.x branch
+ */
+class Spark3_3Adapter extends BaseSpark3Adapter {
+
+  override def createAvroSerializer(rootCatalystType: DataType, rootAvroType: Schema, nullable: Boolean): HoodieAvroSerializer =
+    new HoodieSpark3_2AvroSerializer(rootCatalystType, rootAvroType, nullable)
+
+  override def createAvroDeserializer(rootAvroType: Schema, rootCatalystType: DataType): HoodieAvroDeserializer =
+    new HoodieSpark3_2AvroDeserializer(rootAvroType, rootCatalystType)
+
+  override def createCatalystExpressionUtils(): HoodieCatalystExpressionUtils = HoodieSpark3_3CatalystExpressionUtils
+
+  /**
+   * if the logical plan is a TimeTravelRelation LogicalPlan.
+   */
+  override def isRelationTimeTravel(plan: LogicalPlan): Boolean = {
+    plan.isInstanceOf[TimeTravelRelation]
+  }
+
+  /**
+   * Get the member of the TimeTravelRelation LogicalPlan.
+   */
+  override def getRelationTimeTravel(plan: LogicalPlan): Option[(LogicalPlan, Option[Expression], Option[String])] = {
+    plan match {
+      case timeTravel: TimeTravelRelation =>
+        Some((timeTravel.table, timeTravel.timestamp, timeTravel.version))
+      case _ =>
+        None
+    }
+  }
+
+  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 getDeleteFromTable(table: LogicalPlan, condition: Option[Expression]): DeleteFromTable = {
+    DeleteFromTable(table, condition.getOrElse(Literal.TrueLiteral))
+  }
+
+  override def getQueryParserFromExtendedSqlParser(session: SparkSession, delegate: ParserInterface,
+                                                   sqlText: String): LogicalPlan = {
+    new HoodieSpark3_3ExtendedSqlParser(session, delegate).parseQuery(sqlText)

Review Comment:
   Why can't we just get leverage existing `createExtendedSparkParser` API instead adding this new one?



##########
hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieAnalysis.scala:
##########
@@ -64,15 +64,18 @@ object HoodieAnalysis {
       val spark3ResolveReferences: RuleBuilder =
         session => ReflectionUtils.loadClass(spark3ResolveReferencesClass, session).asInstanceOf[Rule[LogicalPlan]]
 
-      val spark32ResolveAlterTableCommandsClass = "org.apache.spark.sql.hudi.ResolveHudiAlterTableCommandSpark32"
-      val spark32ResolveAlterTableCommands: RuleBuilder =
-        session => ReflectionUtils.loadClass(spark32ResolveAlterTableCommandsClass, session).asInstanceOf[Rule[LogicalPlan]]
+      val resolveAlterTableCommandsClass =
+        if (HoodieSparkUtils.gteqSpark3_3_0)
+          "org.apache.spark.sql.hudi.ResolveHudiAlterTableCommandSpark33"

Review Comment:
   Do you mind cleaning this up and renaming the classes to have Spark3x as prefix (same as other classes)



##########
hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/DeleteHoodieTableCommand.scala:
##########
@@ -36,9 +37,13 @@ case class DeleteHoodieTableCommand(deleteTable: DeleteFromTable) extends Hoodie
 
     // Remove meta fields from the data frame
     var df = removeMetaFields(Dataset.ofRows(sparkSession, table))
-    if (deleteTable.condition.isDefined) {
-      df = df.filter(Column(deleteTable.condition.get))
+    // SPARK-38626 DeleteFromTable.condition is changed from Option[Expression] to Expression in Spark 3.3
+    val condition: Expression = deleteTable.condition match {

Review Comment:
   Instead of doing that let's modify this class to hold the `Option[Expression]` instead of `DeleteFromCommand`



##########
hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieAnalysis.scala:
##########
@@ -413,10 +416,13 @@ case class HoodieResolveReferences(sparkSession: SparkSession) extends Rule[Logi
     // Resolve Delete Table
     case DeleteFromTable(table, condition)
       if sparkAdapter.isHoodieTable(table, sparkSession) && table.resolved =>
-      // Resolve condition
-      val resolvedCondition = condition.map(resolveExpressionFrom(table)(_))
+      val unwrappedCondition: Expression = condition match {

Review Comment:
   Let's hide this compatibility juggling w/in the SparkAdapter actually: we can create method like 
   
   ```
   def resolveDeleteFromCommand(dfc: DeleteFromCommand)
   ```
   And handle compatibility in there 



##########
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/HoodieSparkSqlTestBase.scala:
##########
@@ -140,8 +140,11 @@ class HoodieSparkSqlTestBase extends FunSuite with BeforeAndAfterAll {
       spark.sql(sql)
     } catch {
       case e: Throwable =>
-        assertResult(true)(e.getMessage.contains(errorMsg))
-        hasException = true
+        if (e.getMessage.contains(errorMsg)) {

Review Comment:
   Why changing this?



-- 
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: commits-unsubscribe@hudi.apache.org

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