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


##########
hudi-common/src/main/java/org/apache/hudi/common/util/CollectionUtils.java:
##########
@@ -69,6 +69,26 @@ public static boolean nonEmpty(Collection<?> c) {
     return !isNullOrEmpty(c);
   }
 
+  /**
+   * Reduces provided {@link Collection} using provided {@code reducer} 
applied to
+   * every element of the collection like following
+   *
+   * {@code reduce(reduce(reduce(identity, e1), e2), ...)}
+   *
+   * @param c target collection to be reduced
+   * @param identity element for reducing to start from
+   * @param reducer actual reducing operator
+   *
+   * @return result of the reduction of the collection using reducing operator
+   */
+  public static <T, U> U reduce(Collection<T> c, U identity, BiFunction<U, T, 
U> reducer) {
+    return c.stream()
+        .sequential()

Review Comment:
   [Reducing](https://en.wikipedia.org/wiki/Reduction_operator) is inherently 
sequential operation. In this case i'm just creating a convenience wrapper 
enforcing this in exchange of simplifying the API provided by the streams (if 
someone wants to do it in parallel they can use Streams API directly)



##########
hudi-common/src/main/java/org/apache/hudi/internal/schema/action/TableChange.java:
##########
@@ -83,10 +83,16 @@ abstract class BaseColumnChange implements TableChange {
     protected final InternalSchema internalSchema;
     protected final Map<Integer, Integer> id2parent;
     protected final Map<Integer, ArrayList<ColumnPositionChange>> 
positionChangeMap = new HashMap<>();
+    protected final boolean caseSensitive;
 
     BaseColumnChange(InternalSchema schema) {
+      this(schema, false);

Review Comment:
   To keep things compatible w/ how they are today



##########
hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/HoodieCatalystExpressionUtils.scala:
##########
@@ -75,7 +81,7 @@ trait HoodieCatalystExpressionUtils {
   def unapplyCastExpression(expr: Expression): Option[(Expression, DataType, 
Option[String], Boolean)]
 }
 
-object HoodieCatalystExpressionUtils {
+object HoodieCatalystExpressionUtils extends SparkAdapterSupport {

Review Comment:
   Yes, adapter is needed to match Cast expression (`MatchCast` object below)



##########
hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/catalyst/analysis/HoodieSpark2Analysis.scala:
##########
@@ -0,0 +1,208 @@
+/*
+ * 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 org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.expressions.{Alias, CurrentDate, 
CurrentTimestamp, Expression, ExtractValue, GetStructField, LambdaFunction}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Assignment, 
DeleteAction, InsertAction, LogicalPlan, MergeIntoTable, Project, UpdateAction, 
Window}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.util.toPrettySQL
+
+/**
+ * NOTE: This code is borrowed from Spark 3.1.3
+ *       This code is borrowed, so that we can have some advanced Spark SQL 
functionality (like Merge Into, for ex)
+ *       in Spark 2.x
+ *
+ *       PLEASE REFRAIN MAKING ANY CHANGES TO THIS CODE UNLESS ABSOLUTELY 
NECESSARY
+ */
+object HoodieSpark2Analysis {

Review Comment:
   It's supported. 
   
   The difference is that previously Hudi's bespoke MIT resolution logic was 
applied to all Spark versions not only Spark 2.x. Now,
    - For Spark 3.x we rely on Spark's own logic to resolve
    - For Spark 2.x we back-ported code from Spark 3.1.x



##########
hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/DeleteHoodieTableCommand.scala:
##########
@@ -20,35 +20,44 @@ package org.apache.spark.sql.hudi.command
 import org.apache.hudi.SparkAdapterSupport
 import org.apache.spark.sql._
 import org.apache.spark.sql.catalyst.catalog.HoodieCatalogTable
-import org.apache.spark.sql.catalyst.plans.logical.DeleteFromTable
-import org.apache.spark.sql.catalyst.expressions.Expression
-import org.apache.spark.sql.hudi.HoodieSqlCommonUtils._
+import org.apache.spark.sql.catalyst.plans.logical.{DeleteFromTable, Filter}
 import org.apache.spark.sql.hudi.ProvidesHoodieConfig
+import 
org.apache.spark.sql.hudi.command.HoodieLeafRunnableCommand.stripMetaFieldAttributes
 
-case class DeleteHoodieTableCommand(deleteTable: DeleteFromTable) extends 
HoodieLeafRunnableCommand
-  with SparkAdapterSupport with ProvidesHoodieConfig {
+case class DeleteHoodieTableCommand(dft: DeleteFromTable) extends 
HoodieLeafRunnableCommand
+  with SparkAdapterSupport
+  with ProvidesHoodieConfig {
 
-  private val table = deleteTable.table
+  override def run(sparkSession: SparkSession): Seq[Row] = {
+    val catalogTable = sparkAdapter.resolveHoodieTable(dft.table)
+      .map(HoodieCatalogTable(sparkSession, _))
+      .get
 
-  private val tableId = getTableIdentifier(table)
+    val tableId = catalogTable.table.qualifiedName
 
-  override def run(sparkSession: SparkSession): Seq[Row] = {
-    logInfo(s"start execute delete command for $tableId")
+    logInfo(s"Executing 'DELETE FROM' command for $tableId")
+
+    val condition = sparkAdapter.extractDeleteCondition(dft)
+
+    val targetLogicalPlan = stripMetaFieldAttributes(dft.table)

Review Comment:
   Good call. I don't think this will work in the exsting impl 



##########
hudi-spark-datasource/hudi-spark3.2plus-common/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieSpark32PlusAnalysis.scala:
##########
@@ -0,0 +1,167 @@
+/*
+ * 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.hudi.analysis
+
+import org.apache.hudi.{DataSourceReadOptions, DefaultSource, 
SparkAdapterSupport}
+import org.apache.spark.sql.HoodieSpark3CatalystPlanUtils.MatchResolvedTable
+import org.apache.spark.sql.catalyst.analysis.UnresolvedPartitionSpec
+import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogUtils}
+import org.apache.spark.sql.catalyst.plans.logcal.HoodieQuery
+import org.apache.spark.sql.catalyst.plans.logcal.HoodieQuery.parseOptions
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import 
org.apache.spark.sql.connector.catalog.CatalogV2Implicits.IdentifierHelper
+import org.apache.spark.sql.connector.catalog.{Table, V1Table}
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.execution.datasources.{DataSource, LogicalRelation}
+import org.apache.spark.sql.hudi.ProvidesHoodieConfig
+import 
org.apache.spark.sql.hudi.analysis.HoodieSpark32PlusAnalysis.{HoodieV1OrV2Table,
 ResolvesToHudiTable}
+import org.apache.spark.sql.hudi.catalog.HoodieInternalV2Table
+import 
org.apache.spark.sql.hudi.command.{AlterHoodieTableDropPartitionCommand, 
ShowHoodieTablePartitionsCommand, TruncateHoodieTableCommand}
+import org.apache.spark.sql.{AnalysisException, SQLContext, SparkSession}
+
+/**
+ * NOTE: PLEASE READ CAREFULLY
+ *
+ * Since Hudi relations don't currently implement DS V2 Read API, we have to 
fallback to V1 here.
+ * Such fallback will have considerable performance impact, therefore it's 
only performed in cases
+ * where V2 API have to be used. Currently only such use-case is using of 
Schema Evolution feature
+ *
+ * Check out HUDI-4178 for more details
+ */
+case class HoodieDataSourceV2ToV1Fallback(sparkSession: SparkSession) extends 
Rule[LogicalPlan]

Review Comment:
   B/c we're using DSVv2 only in `HoodieCatalog` that's Spark >= 3.2



##########
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/HoodieSparkSqlTestBase.scala:
##########
@@ -50,6 +50,9 @@ class HoodieSparkSqlTestBase extends FunSuite with 
BeforeAndAfterAll {
     dir
   }
 
+  // NOTE: We need to set "spark.testing" property to make sure Spark can 
appropriately
+  //       recognize environment as testing
+  System.setProperty("spark.testing", "true")

Review Comment:
   We can configure some of the configs that aren't configurable in prod env



-- 
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