diff --git a/azure-pipelines.yml b/azure-pipelines.yml
index 4401ba6236ab8..cb9787bb3f930 100644
--- a/azure-pipelines.yml
+++ b/azure-pipelines.yml
@@ -72,7 +72,7 @@ parameters:
- '!hudi-utilities'
variables:
- BUILD_PROFILES: '-Dscala-2.11 -Dspark2 -Dflink1.14'
+ BUILD_PROFILES: '-Dscala-2.11 -Dspark2.4 -Dflink1.14'
PLUGIN_OPTS: '-Dcheckstyle.skip=true -Drat.skip=true -Djacoco.skip=true -ntp -B -V -Pwarn-log -Dorg.slf4j.simpleLogger.log.org.apache.maven.plugins.shade=warn -Dorg.slf4j.simpleLogger.log.org.apache.maven.plugins.dependency=warn'
MVN_OPTS_INSTALL: '-DskipTests $(BUILD_PROFILES) $(PLUGIN_OPTS)'
MVN_OPTS_TEST: '-fae -Pwarn-log $(BUILD_PROFILES) $(PLUGIN_OPTS)'
diff --git a/hudi-cli/pom.xml b/hudi-cli/pom.xml
index 2bc5213f22315..d283d17ea93e1 100644
--- a/hudi-cli/pom.xml
+++ b/hudi-cli/pom.xml
@@ -136,7 +136,11 @@
org.scala-langscala-library
- ${scala.version}
+
+
+
+ org.scala-lang.modules
+ scala-collection-compat_${scala.binary.version}
diff --git a/hudi-client/hudi-flink-client/pom.xml b/hudi-client/hudi-flink-client/pom.xml
index 47280bf72d6f7..e3ddc8b9ed326 100644
--- a/hudi-client/hudi-flink-client/pom.xml
+++ b/hudi-client/hudi-flink-client/pom.xml
@@ -36,20 +36,20 @@
-
- org.apache.logging.log4j
- log4j-1.2-api
-
-
- org.apache.logging.log4j
- log4j-slf4j-impl
-
-
- org.slf4j
- slf4j-api
-
+
+ org.apache.logging.log4j
+ log4j-1.2-api
+
+
+ org.apache.logging.log4j
+ log4j-slf4j-impl
+
+
+ org.slf4j
+ slf4j-api
+
-
+
org.apache.hudihudi-client-common
diff --git a/hudi-client/hudi-spark-client/pom.xml b/hudi-client/hudi-spark-client/pom.xml
index 8d913d60ca430..cf3b075636ae3 100644
--- a/hudi-client/hudi-spark-client/pom.xml
+++ b/hudi-client/hudi-spark-client/pom.xml
@@ -34,7 +34,11 @@
org.scala-langscala-library
- ${scala.version}
+
+
+
+ org.scala-lang.modules
+ scala-collection-compat_${scala.binary.version}
diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/HoodieCatalogUtils.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/HoodieCatalogUtils.scala
new file mode 100644
index 0000000000000..c65957515546f
--- /dev/null
+++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/HoodieCatalogUtils.scala
@@ -0,0 +1,24 @@
+/*
+ * 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
+
+/**
+ * NOTE: Since support for [[TableCatalog]] was only added in Spark 3, this trait
+ * is going to be an empty one simply serving as a placeholder (for compatibility w/ Spark 2)
+ */
+trait HoodieCatalogUtils {}
diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/HoodieCatalystPlansUtils.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/HoodieCatalystPlansUtils.scala
index 1733b03ab8edb..e7e529b12545a 100644
--- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/HoodieCatalystPlansUtils.scala
+++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/HoodieCatalystPlansUtils.scala
@@ -17,11 +17,11 @@
package org.apache.spark.sql
-import org.apache.spark.sql.catalyst.{AliasIdentifier, TableIdentifier}
import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression}
import org.apache.spark.sql.catalyst.plans.JoinType
import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan}
+import org.apache.spark.sql.catalyst.{AliasIdentifier, TableIdentifier}
import org.apache.spark.sql.internal.SQLConf
trait HoodieCatalystPlansUtils {
diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala
index eaad4d471e0d3..1dd96958aa0f6 100644
--- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala
+++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala
@@ -33,7 +33,7 @@ import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat
import org.apache.spark.sql.execution.datasources.{FilePartition, FileScanRDD, LogicalRelation, PartitionedFile, SparkParsePartitionUtil}
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types.{DataType, StructType}
-import org.apache.spark.sql.{HoodieCatalystExpressionUtils, HoodieCatalystPlansUtils, Row, SparkSession}
+import org.apache.spark.sql.{HoodieCatalogUtils, HoodieCatalystExpressionUtils, HoodieCatalystPlansUtils, Row, SparkSession}
import org.apache.spark.storage.StorageLevel
import java.util.Locale
@@ -44,13 +44,19 @@ import java.util.Locale
trait SparkAdapter extends Serializable {
/**
- * Creates instance of [[HoodieCatalystExpressionUtils]] providing for common utils operating
+ * Returns an instance of [[HoodieCatalogUtils]] providing for common utils operating on Spark's
+ * [[TableCatalog]]s
+ */
+ def getCatalogUtils: HoodieCatalogUtils
+
+ /**
+ * Returns an instance of [[HoodieCatalystExpressionUtils]] providing for common utils operating
* on Catalyst [[Expression]]s
*/
def getCatalystExpressionUtils: HoodieCatalystExpressionUtils
/**
- * Creates instance of [[HoodieCatalystPlansUtils]] providing for common utils operating
+ * Returns an instance of [[HoodieCatalystPlansUtils]] providing for common utils operating
* on Catalyst [[LogicalPlan]]s
*/
def getCatalystPlanUtils: HoodieCatalystPlansUtils
diff --git a/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/SerDeHelper.java b/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/SerDeHelper.java
index fe6174057bbdc..aebda533e324a 100644
--- a/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/SerDeHelper.java
+++ b/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/SerDeHelper.java
@@ -23,9 +23,9 @@
import com.fasterxml.jackson.databind.JsonNode;
import com.fasterxml.jackson.databind.ObjectMapper;
-import org.apache.hadoop.hbase.exceptions.IllegalArgumentIOException;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.internal.schema.InternalSchema;
import org.apache.hudi.internal.schema.Type;
import org.apache.hudi.internal.schema.Types;
@@ -179,7 +179,7 @@ private static void toJson(Type type, JsonGenerator generator) throws IOExceptio
if (!type.isNestedType()) {
generator.writeString(type.toString());
} else {
- throw new IllegalArgumentIOException(String.format("cannot write unknown types: %s", type));
+ throw new HoodieIOException(String.format("cannot write unknown types: %s", type));
}
}
}
diff --git a/hudi-examples/hudi-examples-spark/pom.xml b/hudi-examples/hudi-examples-spark/pom.xml
index 6e2b2f4069353..3c59acdfb7555 100644
--- a/hudi-examples/hudi-examples-spark/pom.xml
+++ b/hudi-examples/hudi-examples-spark/pom.xml
@@ -112,7 +112,11 @@
org.scala-langscala-library
- ${scala.version}
+
+
+
+ org.scala-lang.modules
+ scala-collection-compat_${scala.binary.version}
@@ -134,12 +138,6 @@
-
- org.apache.hudi
- hudi-cli
- ${project.version}
-
-
org.apache.hudihudi-client-common
diff --git a/hudi-spark-datasource/hudi-spark-common/pom.xml b/hudi-spark-datasource/hudi-spark-common/pom.xml
index 2ab2b24e748f0..e3c221fa36f5a 100644
--- a/hudi-spark-datasource/hudi-spark-common/pom.xml
+++ b/hudi-spark-datasource/hudi-spark-common/pom.xml
@@ -146,7 +146,11 @@
org.scala-langscala-library
- ${scala.version}
+
+
+
+ org.scala-lang.modules
+ scala-collection-compat_${scala.binary.version}
diff --git a/hudi-spark-datasource/hudi-spark/pom.xml b/hudi-spark-datasource/hudi-spark/pom.xml
index ddce2ae63770f..6b9fbe031ed41 100644
--- a/hudi-spark-datasource/hudi-spark/pom.xml
+++ b/hudi-spark-datasource/hudi-spark/pom.xml
@@ -165,12 +165,6 @@
-
-
- org.scala-lang
- scala-library
- ${scala.version}
-
@@ -178,71 +172,46 @@
log4j-1.2-api
-
-
- org.apache.hudi
- hudi-client-common
- ${project.version}
-
+
org.apache.hudihudi-spark-client${project.version}
+
org.apache.hudi
- hudi-common
- ${project.version}
-
-
- org.apache.hudi
- hudi-hadoop-mr
+ ${hudi.spark.module}_${scala.binary.version}${project.version}
+
+
+
org.apache.hudi
- hudi-hive-sync
+ hudi-client-common${project.version}
+
org.apache.hudi
- hudi-sync-common
+ hudi-common${project.version}org.apache.hudi
- hudi-spark-common_${scala.binary.version}
+ hudi-hadoop-mr${project.version}
-
-
- org.apache.curator
- *
-
-
-
org.apache.hudi
- ${hudi.spark.module}_${scala.binary.version}
+ hudi-hive-sync${project.version}
-
-
- org.apache.hudi
- *
-
-
-
org.apache.hudi
- ${hudi.spark.common.module}
+ hudi-sync-common${project.version}
-
-
- org.apache.hudi
- *
-
-
diff --git a/hudi-spark-datasource/hudi-spark/run_hoodie_app.sh b/hudi-spark-datasource/hudi-spark/run_hoodie_app.sh
index 9782aa359556f..ba5eb6ed56521 100755
--- a/hudi-spark-datasource/hudi-spark/run_hoodie_app.sh
+++ b/hudi-spark-datasource/hudi-spark/run_hoodie_app.sh
@@ -23,7 +23,7 @@ function error_exit {
DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )"
#Ensure we pick the right jar even for hive11 builds
-HUDI_JAR=`ls -c $DIR/../../packaging/hudi-spark-bundle/target/hudi-spark-bundle*.jar | grep -v sources | head -1`
+HUDI_JAR=`ls -c $DIR/../../packaging/hudi-spark-bundle/target/hudi-spark*-bundle*.jar | grep -v sources | head -1`
if [ -z "$HADOOP_CONF_DIR" ]; then
echo "setting hadoop conf dir"
diff --git a/hudi-spark-datasource/hudi-spark/run_hoodie_generate_app.sh b/hudi-spark-datasource/hudi-spark/run_hoodie_generate_app.sh
index a2769517b9eb4..15c6c0d48cc2e 100755
--- a/hudi-spark-datasource/hudi-spark/run_hoodie_generate_app.sh
+++ b/hudi-spark-datasource/hudi-spark/run_hoodie_generate_app.sh
@@ -23,7 +23,7 @@ function error_exit {
DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )"
#Ensure we pick the right jar even for hive11 builds
-HUDI_JAR=`ls -c $DIR/../../packaging/hudi-spark-bundle/target/hudi-spark-bundle*.jar | grep -v sources | head -1`
+HUDI_JAR=`ls -c $DIR/../../packaging/hudi-spark-bundle/target/hudi-spark*-bundle*.jar | grep -v sources | head -1`
if [ -z "$HADOOP_CONF_DIR" ]; then
echo "setting hadoop conf dir"
diff --git a/hudi-spark-datasource/hudi-spark/run_hoodie_streaming_app.sh b/hudi-spark-datasource/hudi-spark/run_hoodie_streaming_app.sh
index 9a81a4c0684e3..0501ff8f43bde 100755
--- a/hudi-spark-datasource/hudi-spark/run_hoodie_streaming_app.sh
+++ b/hudi-spark-datasource/hudi-spark/run_hoodie_streaming_app.sh
@@ -23,7 +23,7 @@ function error_exit {
DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )"
#Ensure we pick the right jar even for hive11 builds
-HUDI_JAR=`ls -c $DIR/../../packaging/hudi-spark-bundle/target/hudi-spark-bundle*.jar | grep -v sources | head -1`
+HUDI_JAR=`ls -c $DIR/../../packaging/hudi-spark-bundle/target/hudi-spark*-bundle*.jar | grep -v sources | head -1`
if [ -z "$HADOOP_CONF_DIR" ]; then
echo "setting hadoop conf dir"
diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieAnalysis.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieAnalysis.scala
index 4f4446ac281c6..ecc5d4a901c58 100644
--- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieAnalysis.scala
+++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieAnalysis.scala
@@ -79,10 +79,6 @@ object HoodieAnalysis {
val spark3Analysis: RuleBuilder =
session => ReflectionUtils.loadClass(spark3AnalysisClass, session).asInstanceOf[Rule[LogicalPlan]]
- val spark3ResolveReferencesClass = "org.apache.spark.sql.hudi.analysis.HoodieSpark3ResolveReferences"
- val spark3ResolveReferences: RuleBuilder =
- session => ReflectionUtils.loadClass(spark3ResolveReferencesClass, session).asInstanceOf[Rule[LogicalPlan]]
-
val resolveAlterTableCommandsClass =
if (HoodieSparkUtils.gteqSpark3_3)
"org.apache.spark.sql.hudi.Spark33ResolveHudiAlterTableCommand"
@@ -94,10 +90,10 @@ object HoodieAnalysis {
//
// It's critical for this rules to follow in this order, so that DataSource V2 to V1 fallback
// is performed prior to other rules being evaluated
- rules ++= Seq(dataSourceV2ToV1Fallback, spark3Analysis, spark3ResolveReferences, resolveAlterTableCommands)
+ rules ++= Seq(dataSourceV2ToV1Fallback, spark3Analysis, resolveAlterTableCommands)
} else if (HoodieSparkUtils.gteqSpark3_1) {
- val spark31ResolveAlterTableCommandsClass = "org.apache.spark.sql.hudi.Spark312ResolveHudiAlterTableCommand"
+ val spark31ResolveAlterTableCommandsClass = "org.apache.spark.sql.hudi.Spark31ResolveHudiAlterTableCommand"
val spark31ResolveAlterTableCommands: RuleBuilder =
session => ReflectionUtils.loadClass(spark31ResolveAlterTableCommandsClass, session).asInstanceOf[Rule[LogicalPlan]]
diff --git a/hudi-spark-datasource/hudi-spark2/pom.xml b/hudi-spark-datasource/hudi-spark2/pom.xml
index 8bdab01ba1c51..ce7741f1ae54c 100644
--- a/hudi-spark-datasource/hudi-spark2/pom.xml
+++ b/hudi-spark-datasource/hudi-spark2/pom.xml
@@ -166,12 +166,6 @@
-
-
- org.scala-lang
- scala-library
- ${scala.version}
-
diff --git a/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/adapter/Spark2Adapter.scala b/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/adapter/Spark2Adapter.scala
index 91ab71ef1c3af..d7d498431b0b6 100644
--- a/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/adapter/Spark2Adapter.scala
+++ b/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/adapter/Spark2Adapter.scala
@@ -44,6 +44,10 @@ import scala.collection.mutable.ArrayBuffer
*/
class Spark2Adapter extends SparkAdapter {
+ override def getCatalogUtils: HoodieCatalogUtils = {
+ throw new UnsupportedOperationException("Catalog utilities are not supported in Spark 2.x");
+ }
+
override def getCatalystExpressionUtils: HoodieCatalystExpressionUtils = HoodieSpark2CatalystExpressionUtils
override def getCatalystPlanUtils: HoodieCatalystPlansUtils = HoodieSpark2CatalystPlanUtils
diff --git a/hudi-spark-datasource/hudi-spark3-common/pom.xml b/hudi-spark-datasource/hudi-spark3-common/pom.xml
index 19f73c4ca625b..d09523cc8dc29 100644
--- a/hudi-spark-datasource/hudi-spark3-common/pom.xml
+++ b/hudi-spark-datasource/hudi-spark3-common/pom.xml
@@ -158,11 +158,6 @@
-
- org.scala-lang
- scala-library
- ${scala12.version}
- org.apache.spark
diff --git a/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/HoodieSpark3CatalogUtils.scala b/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/HoodieSpark3CatalogUtils.scala
new file mode 100644
index 0000000000000..33cb8ae373e16
--- /dev/null
+++ b/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/HoodieSpark3CatalogUtils.scala
@@ -0,0 +1,45 @@
+/*
+ * 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
+
+import org.apache.hudi.SparkAdapterSupport
+import org.apache.spark.sql.connector.expressions.{NamedReference, Transform}
+
+trait HoodieSpark3CatalogUtils extends HoodieCatalogUtils {
+
+ /**
+ * Decomposes [[org.apache.spark.sql.connector.expressions.BucketTransform]] extracting its
+ * arguments to accommodate for API changes in Spark 3.3 returning:
+ *
+ *
+ *
Number of the buckets
+ *
Seq of references (to be bucketed by)
+ *
Seq of sorted references
+ *
+ */
+ def unapplyBucketTransform(t: Transform): Option[(Int, Seq[NamedReference], Seq[NamedReference])]
+}
+
+object HoodieSpark3CatalogUtils extends SparkAdapterSupport {
+
+ object MatchBucketTransform {
+ def unapply(t: Transform): Option[(Int, Seq[NamedReference], Seq[NamedReference])] =
+ sparkAdapter.getCatalogUtils.asInstanceOf[HoodieSpark3CatalogUtils]
+ .unapplyBucketTransform(t)
+ }
+}
diff --git a/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/HoodieSpark3CatalystPlanUtils.scala b/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/HoodieSpark3CatalystPlanUtils.scala
index 45b7a296da6f8..6ad0ee57e081f 100644
--- a/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/HoodieSpark3CatalystPlanUtils.scala
+++ b/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/HoodieSpark3CatalystPlanUtils.scala
@@ -19,7 +19,7 @@ package org.apache.spark.sql
import org.apache.hudi.spark3.internal.ReflectUtil
import org.apache.spark.sql.catalyst.analysis.{TableOutputResolver, UnresolvedRelation}
-import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, Like}
+import org.apache.spark.sql.catalyst.expressions.Attribute
import org.apache.spark.sql.catalyst.plans.JoinType
import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoStatement, Join, JoinHint, LogicalPlan}
import org.apache.spark.sql.catalyst.{AliasIdentifier, TableIdentifier}
@@ -28,7 +28,7 @@ import org.apache.spark.sql.execution.command.ExplainCommand
import org.apache.spark.sql.execution.{ExtendedMode, SimpleMode}
import org.apache.spark.sql.internal.SQLConf
-abstract class HoodieSpark3CatalystPlanUtils extends HoodieCatalystPlansUtils {
+trait HoodieSpark3CatalystPlanUtils extends HoodieCatalystPlansUtils {
def resolveOutputColumns(tableName: String,
expected: Seq[Attribute],
diff --git a/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/adapter/BaseSpark3Adapter.scala b/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/adapter/BaseSpark3Adapter.scala
index 77df665b98def..1c8151ef91867 100644
--- a/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/adapter/BaseSpark3Adapter.scala
+++ b/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/adapter/BaseSpark3Adapter.scala
@@ -20,7 +20,6 @@ package org.apache.spark.sql.adapter
import org.apache.hudi.Spark3RowSerDe
import org.apache.hudi.client.utils.SparkRowSerDe
import org.apache.spark.internal.Logging
-import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.avro.{HoodieAvroSchemaConverters, HoodieSparkAvroSchemaConverters}
import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
import org.apache.spark.sql.catalyst.encoders.RowEncoder
@@ -33,6 +32,7 @@ import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
import org.apache.spark.sql.hudi.SparkAdapter
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.{HoodieSpark3CatalogUtils, SparkSession}
import org.apache.spark.storage.StorageLevel
import org.apache.spark.storage.StorageLevel._
@@ -43,6 +43,8 @@ import scala.util.control.NonFatal
*/
abstract class BaseSpark3Adapter extends SparkAdapter with Logging {
+ def getCatalogUtils: HoodieSpark3CatalogUtils
+
override def createSparkRowSerDe(schema: StructType): SparkRowSerDe = {
val encoder = RowEncoder(schema).resolveAndBind()
new Spark3RowSerDe(encoder)
diff --git a/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/hudi/catalog/TableCreationMode.java b/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/HoodieUnaryLikeSham.scala
similarity index 63%
rename from hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/hudi/catalog/TableCreationMode.java
rename to hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/HoodieUnaryLikeSham.scala
index 8b54775be149e..e64709e7d83e5 100644
--- a/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/hudi/catalog/TableCreationMode.java
+++ b/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/HoodieUnaryLikeSham.scala
@@ -1,23 +1,31 @@
-/*
- * 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.catalog;
-
-public enum TableCreationMode {
- CREATE, CREATE_OR_REPLACE, STAGE_CREATE, STAGE_REPLACE
-}
+/*
+ * 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.plans.logical
+
+import org.apache.spark.sql.catalyst.trees.TreeNode
+
+/**
+ * NOTE: This is a sham trait providing a stub method definition which is not used anywhere.
+ * This is required just to be able to compile the code that relies on [[UnaryLike]]
+ * (introduced in Spark 3.2) against Spark < 3.2
+ */
+trait HoodieUnaryLikeSham[T <: TreeNode[T]] {
+ self: TreeNode[T] =>
+ protected def withNewChildInternal(newChild: T): T
+}
diff --git a/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/connector/catalog/HoodieIdentifier.scala b/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/connector/catalog/HoodieIdentifier.scala
similarity index 100%
rename from hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/connector/catalog/HoodieIdentifier.scala
rename to hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/connector/catalog/HoodieIdentifier.scala
diff --git a/hudi-spark-datasource/hudi-spark3.1.x/pom.xml b/hudi-spark-datasource/hudi-spark3.1.x/pom.xml
index d4b428d258073..de37e841bab84 100644
--- a/hudi-spark-datasource/hudi-spark3.1.x/pom.xml
+++ b/hudi-spark-datasource/hudi-spark3.1.x/pom.xml
@@ -149,11 +149,6 @@
-
- org.scala-lang
- scala-library
- ${scala12.version}
- org.apache.spark
diff --git a/hudi-spark-datasource/hudi-spark3.1.x/src/main/java/org/apache/spark/sql/execution/datasources/parquet/Spark312HoodieVectorizedParquetRecordReader.java b/hudi-spark-datasource/hudi-spark3.1.x/src/main/java/org/apache/spark/sql/execution/datasources/parquet/Spark31HoodieVectorizedParquetRecordReader.java
similarity index 97%
rename from hudi-spark-datasource/hudi-spark3.1.x/src/main/java/org/apache/spark/sql/execution/datasources/parquet/Spark312HoodieVectorizedParquetRecordReader.java
rename to hudi-spark-datasource/hudi-spark3.1.x/src/main/java/org/apache/spark/sql/execution/datasources/parquet/Spark31HoodieVectorizedParquetRecordReader.java
index 3f86eeedffa47..d5108b94fceb0 100644
--- a/hudi-spark-datasource/hudi-spark3.1.x/src/main/java/org/apache/spark/sql/execution/datasources/parquet/Spark312HoodieVectorizedParquetRecordReader.java
+++ b/hudi-spark-datasource/hudi-spark3.1.x/src/main/java/org/apache/spark/sql/execution/datasources/parquet/Spark31HoodieVectorizedParquetRecordReader.java
@@ -35,7 +35,7 @@
import java.util.HashMap;
import java.util.Map;
-public class Spark312HoodieVectorizedParquetRecordReader extends VectorizedParquetRecordReader {
+public class Spark31HoodieVectorizedParquetRecordReader extends VectorizedParquetRecordReader {
// save the col type change info.
private Map> typeChangeInfos;
@@ -62,7 +62,7 @@ public class Spark312HoodieVectorizedParquetRecordReader extends VectorizedParqu
private int batchIdx = 0;
private int numBatched = 0;
- public Spark312HoodieVectorizedParquetRecordReader(
+ public Spark31HoodieVectorizedParquetRecordReader(
ZoneId convertTz,
String datetimeRebaseMode,
String int96RebaseMode,
diff --git a/hudi-spark-datasource/hudi-spark3.1.x/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister b/hudi-spark-datasource/hudi-spark3.1.x/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
index 8fbcd663c623d..faf4a1d516c61 100644
--- a/hudi-spark-datasource/hudi-spark3.1.x/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
+++ b/hudi-spark-datasource/hudi-spark3.1.x/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
@@ -16,4 +16,4 @@
# limitations under the License.
-org.apache.hudi.Spark3xDefaultSource
\ No newline at end of file
+org.apache.hudi.Spark31DefaultSource
\ No newline at end of file
diff --git a/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/hudi/Spark3xDefaultSource.scala b/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/hudi/Spark31DefaultSource.scala
similarity index 93%
rename from hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/hudi/Spark3xDefaultSource.scala
rename to hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/hudi/Spark31DefaultSource.scala
index 6f941dabcb9cb..61b15b44a6785 100644
--- a/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/hudi/Spark3xDefaultSource.scala
+++ b/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/hudi/Spark31DefaultSource.scala
@@ -19,6 +19,6 @@ package org.apache.hudi
import org.apache.spark.sql.sources.DataSourceRegister
-class Spark3xDefaultSource extends DefaultSource with DataSourceRegister {
+class Spark31DefaultSource extends DefaultSource with DataSourceRegister {
override def shortName(): String = "hudi"
}
diff --git a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TimeTravelRelation.scala b/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/HoodieSpark31CatalogUtils.scala
similarity index 58%
rename from hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TimeTravelRelation.scala
rename to hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/HoodieSpark31CatalogUtils.scala
index f243a7a86174f..8eaf768c8f7d3 100644
--- a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TimeTravelRelation.scala
+++ b/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/HoodieSpark31CatalogUtils.scala
@@ -15,19 +15,16 @@
* limitations under the License.
*/
-package org.apache.spark.sql.catalyst.plans.logical
+package org.apache.spark.sql
-import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression}
+import org.apache.spark.sql.connector.expressions.{BucketTransform, NamedReference, Transform}
-case class TimeTravelRelation(
- table: LogicalPlan,
- timestamp: Option[Expression],
- version: Option[String]) extends Command {
- override def children: Seq[LogicalPlan] = Seq.empty
+object HoodieSpark31CatalogUtils extends HoodieSpark3CatalogUtils {
- override def output: Seq[Attribute] = Nil
+ override def unapplyBucketTransform(t: Transform): Option[(Int, Seq[NamedReference], Seq[NamedReference])] =
+ t match {
+ case BucketTransform(numBuckets, ref) => Some(numBuckets, Seq(ref), Seq.empty)
+ case _ => None
+ }
- override lazy val resolved: Boolean = false
-
- def withNewChildrenInternal(newChildren: IndexedSeq[LogicalPlan]): LogicalPlan = this
}
diff --git a/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_1Adapter.scala b/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_1Adapter.scala
index 2697c6658ab18..bed6ff33508ba 100644
--- a/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_1Adapter.scala
+++ b/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_1Adapter.scala
@@ -30,13 +30,15 @@ import org.apache.spark.sql.execution.datasources.{FilePartition, FileScanRDD, P
import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Spark31HoodieParquetFileFormat}
import org.apache.spark.sql.hudi.SparkAdapter
import org.apache.spark.sql.types.{DataType, StructType}
-import org.apache.spark.sql.{HoodieCatalystExpressionUtils, HoodieCatalystPlansUtils, HoodieSpark31CatalystExpressionUtils, HoodieSpark31CatalystPlanUtils, SparkSession}
+import org.apache.spark.sql.{HoodieCatalystExpressionUtils, HoodieCatalystPlansUtils, HoodieSpark31CatalogUtils, HoodieSpark31CatalystExpressionUtils, HoodieSpark31CatalystPlanUtils, HoodieSpark3CatalogUtils, SparkSession}
/**
* Implementation of [[SparkAdapter]] for Spark 3.1.x
*/
class Spark3_1Adapter extends BaseSpark3Adapter {
+ override def getCatalogUtils: HoodieSpark3CatalogUtils = HoodieSpark31CatalogUtils
+
override def getCatalystExpressionUtils: HoodieCatalystExpressionUtils = HoodieSpark31CatalystExpressionUtils
override def getCatalystPlanUtils: HoodieCatalystPlansUtils = HoodieSpark31CatalystPlanUtils
diff --git a/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/avro/HoodieSpark3_1AvroDeserializer.scala b/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/avro/HoodieSpark3_1AvroDeserializer.scala
index bf6fcbee7e9c1..7de9a8b28d08d 100644
--- a/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/avro/HoodieSpark3_1AvroDeserializer.scala
+++ b/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/avro/HoodieSpark3_1AvroDeserializer.scala
@@ -18,12 +18,19 @@
package org.apache.spark.sql.avro
import org.apache.avro.Schema
+import org.apache.spark.sql.catalyst.NoopFilters
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.internal.SQLConf.LegacyBehaviorPolicy
import org.apache.spark.sql.types.DataType
class HoodieSpark3_1AvroDeserializer(rootAvroType: Schema, rootCatalystType: DataType)
extends HoodieAvroDeserializer {
- private val avroDeserializer = new AvroDeserializer(rootAvroType, rootCatalystType)
+ private val avroDeserializer = {
+ val avroRebaseModeInRead = LegacyBehaviorPolicy.withName(SQLConf.get.getConf(SQLConf.LEGACY_AVRO_REBASE_MODE_IN_READ))
+ new AvroDeserializer(rootAvroType, rootCatalystType, avroRebaseModeInRead, new NoopFilters)
+ }
+
def deserialize(data: Any): Option[Any] = avroDeserializer.deserialize(data)
}
diff --git a/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark31HoodieParquetFileFormat.scala b/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark31HoodieParquetFileFormat.scala
index e99850bef06b8..ca41490fc0ac2 100644
--- a/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark31HoodieParquetFileFormat.scala
+++ b/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark31HoodieParquetFileFormat.scala
@@ -246,7 +246,7 @@ class Spark31HoodieParquetFileFormat(private val shouldAppendPartitionValues: Bo
if (enableVectorizedReader) {
val vectorizedReader =
if (shouldUseInternalSchema) {
- new Spark312HoodieVectorizedParquetRecordReader(
+ new Spark31HoodieVectorizedParquetRecordReader(
convertTz.orNull,
datetimeRebaseMode.toString,
int96RebaseMode.toString,
diff --git a/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/hudi/Spark312ResolveHudiAlterTableCommand.scala b/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/hudi/Spark31ResolveHudiAlterTableCommand.scala
similarity index 93%
rename from hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/hudi/Spark312ResolveHudiAlterTableCommand.scala
rename to hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/hudi/Spark31ResolveHudiAlterTableCommand.scala
index e9c80c359a110..3324a5f91b0da 100644
--- a/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/hudi/Spark312ResolveHudiAlterTableCommand.scala
+++ b/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/hudi/Spark31ResolveHudiAlterTableCommand.scala
@@ -17,21 +17,19 @@
package org.apache.spark.sql.hudi
import org.apache.hudi.common.config.HoodieCommonConfig
-
-import java.util.Locale
-import org.apache.hudi.config.HoodieWriteConfig
import org.apache.hudi.internal.schema.action.TableChange.ColumnChangeID
-import org.apache.spark.sql.{AnalysisException, SparkSession}
import org.apache.spark.sql.catalyst.TableIdentifier
import org.apache.spark.sql.catalyst.catalog.CatalogTable
-import org.apache.spark.sql.connector.catalog.{CatalogV2Util, TableChange}
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.connector.catalog.CatalogV2Util.failNullType
import org.apache.spark.sql.connector.catalog.TableChange._
-import org.apache.spark.sql.hudi.command.AlterTableCommand312
+import org.apache.spark.sql.connector.catalog.{CatalogV2Util, TableChange}
+import org.apache.spark.sql.hudi.command.Spark31AlterTableCommand
import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.{AnalysisException, SparkSession}
+import java.util.Locale
import scala.collection.mutable
/**
@@ -39,7 +37,7 @@ import scala.collection.mutable
* for alter table column commands.
* TODO: we should remove this file when we support datasourceV2 for hoodie on spark3.1x
*/
-case class Spark312ResolveHudiAlterTableCommand(sparkSession: SparkSession) extends Rule[LogicalPlan] {
+case class Spark31ResolveHudiAlterTableCommand(sparkSession: SparkSession) extends Rule[LogicalPlan] {
import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp {
case add @ HoodieAlterTableAddColumnsStatement(asTable(table), cols) =>
@@ -54,7 +52,7 @@ case class Spark312ResolveHudiAlterTableCommand(sparkSession: SparkSession) exte
col.position.orNull)
}
val newChanges = normalizeChanges(changes, table.schema)
- AlterTableCommand312(table, newChanges, ColumnChangeID.ADD)
+ Spark31AlterTableCommand(table, newChanges, ColumnChangeID.ADD)
} else {
// throw back to spark
AlterTableAddColumnsStatement(add.tableName, add.columnsToAdd)
@@ -75,7 +73,7 @@ case class Spark312ResolveHudiAlterTableCommand(sparkSession: SparkSession) exte
val positionChange = a.position.map { newPosition =>
TableChange.updateColumnPosition(colName, newPosition)
}
- AlterTableCommand312(table, normalizeChanges(typeChange.toSeq ++ nullabilityChange ++ commentChange ++ positionChange, table.schema), ColumnChangeID.UPDATE)
+ Spark31AlterTableCommand(table, normalizeChanges(typeChange.toSeq ++ nullabilityChange ++ commentChange ++ positionChange, table.schema), ColumnChangeID.UPDATE)
} else {
// throw back to spark
AlterTableAlterColumnStatement(a.tableName, a.column, a.dataType, a.nullable, a.comment, a.position)
@@ -83,7 +81,7 @@ case class Spark312ResolveHudiAlterTableCommand(sparkSession: SparkSession) exte
case rename @ HoodieAlterTableRenameColumnStatement(asTable(table), col, newName) =>
if (isHoodieTable(table) && schemaEvolutionEnabled){
val changes = Seq(TableChange.renameColumn(col.toArray, newName))
- AlterTableCommand312(table, normalizeChanges(changes, table.schema), ColumnChangeID.UPDATE)
+ Spark31AlterTableCommand(table, normalizeChanges(changes, table.schema), ColumnChangeID.UPDATE)
} else {
// throw back to spark
AlterTableRenameColumnStatement(rename.tableName, rename.column, rename.newName)
@@ -91,7 +89,7 @@ case class Spark312ResolveHudiAlterTableCommand(sparkSession: SparkSession) exte
case drop @ HoodieAlterTableDropColumnsStatement(asTable(table), cols) =>
if (isHoodieTable(table) && schemaEvolutionEnabled) {
val changes = cols.map(col => TableChange.deleteColumn(col.toArray))
- AlterTableCommand312(table, normalizeChanges(changes, table.schema), ColumnChangeID.DELETE)
+ Spark31AlterTableCommand(table, normalizeChanges(changes, table.schema), ColumnChangeID.DELETE)
} else {
// throw back to spark
AlterTableDropColumnsStatement(drop.tableName, drop.columnsToDrop)
@@ -101,7 +99,7 @@ case class Spark312ResolveHudiAlterTableCommand(sparkSession: SparkSession) exte
val changes = props.map { case (key, value) =>
TableChange.setProperty(key, value)
}.toSeq
- AlterTableCommand312(table, normalizeChanges(changes, table.schema), ColumnChangeID.PROPERTY_CHANGE)
+ Spark31AlterTableCommand(table, normalizeChanges(changes, table.schema), ColumnChangeID.PROPERTY_CHANGE)
} else {
// throw back to spark
AlterTableSetPropertiesStatement(set.tableName, set.properties)
@@ -109,7 +107,7 @@ case class Spark312ResolveHudiAlterTableCommand(sparkSession: SparkSession) exte
case unset @ HoodieAlterTableUnsetPropertiesStatement(asTable(table), keys, _) =>
if (isHoodieTable(table) && schemaEvolutionEnabled) {
val changes = keys.map(key => TableChange.removeProperty(key))
- AlterTableCommand312(table, normalizeChanges(changes, table.schema), ColumnChangeID.PROPERTY_CHANGE)
+ Spark31AlterTableCommand(table, normalizeChanges(changes, table.schema), ColumnChangeID.PROPERTY_CHANGE)
} else {
// throw back to spark
AlterTableUnsetPropertiesStatement(unset.tableName, unset.propertyKeys, unset.ifExists)
diff --git a/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/hudi/command/AlterTableCommand312.scala b/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/hudi/command/Spark31AlterTableCommand.scala
similarity index 94%
rename from hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/hudi/command/AlterTableCommand312.scala
rename to hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/hudi/command/Spark31AlterTableCommand.scala
index 5d78018f02465..9a5366b12f564 100644
--- a/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/hudi/command/AlterTableCommand312.scala
+++ b/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/hudi/command/Spark31AlterTableCommand.scala
@@ -52,7 +52,7 @@ import scala.collection.JavaConverters._
import scala.util.control.NonFatal
// TODO: we should remove this file when we support datasourceV2 for hoodie on spark3.1x
-case class AlterTableCommand312(table: CatalogTable, changes: Seq[TableChange], changeType: ColumnChangeID) extends RunnableCommand with Logging {
+case class Spark31AlterTableCommand(table: CatalogTable, changes: Seq[TableChange], changeType: ColumnChangeID) extends RunnableCommand with Logging {
override def run(sparkSession: SparkSession): Seq[Row] = {
changeType match {
case ColumnChangeID.ADD => applyAddAction(sparkSession)
@@ -72,7 +72,7 @@ case class AlterTableCommand312(table: CatalogTable, changes: Seq[TableChange],
val addChange = TableChanges.ColumnAddChange.get(oldSchema)
changes.map(_.asInstanceOf[AddColumn]).foreach { addColumn =>
val names = addColumn.fieldNames()
- val parentName = AlterTableCommand312.getParentName(names)
+ val parentName = Spark31AlterTableCommand.getParentName(names)
// add col change
val colType = SparkInternalSchemaConverter.buildTypeFromStructType(addColumn.dataType(), true, new AtomicInteger(0))
addChange.addColumns(parentName, names.last, colType, addColumn.comment())
@@ -92,7 +92,7 @@ case class AlterTableCommand312(table: CatalogTable, changes: Seq[TableChange],
} else {
historySchema
}
- AlterTableCommand312.commitWithSchema(newSchema, verifiedHistorySchema, table, sparkSession)
+ Spark31AlterTableCommand.commitWithSchema(newSchema, verifiedHistorySchema, table, sparkSession)
logInfo("column add finished")
}
@@ -101,7 +101,7 @@ case class AlterTableCommand312(table: CatalogTable, changes: Seq[TableChange],
val deleteChange = TableChanges.ColumnDeleteChange.get(oldSchema)
changes.map(_.asInstanceOf[DeleteColumn]).foreach { c =>
val originalColName = c.fieldNames().mkString(".");
- AlterTableCommand312.checkSchemaChange(Seq(originalColName), table)
+ Spark31AlterTableCommand.checkSchemaChange(Seq(originalColName), table)
deleteChange.deleteColumn(originalColName)
}
val newSchema = SchemaChangeUtils.applyTableChanges2Schema(oldSchema, deleteChange)
@@ -112,7 +112,7 @@ case class AlterTableCommand312(table: CatalogTable, changes: Seq[TableChange],
} else {
historySchema
}
- AlterTableCommand312.commitWithSchema(newSchema, verifiedHistorySchema, table, sparkSession)
+ Spark31AlterTableCommand.commitWithSchema(newSchema, verifiedHistorySchema, table, sparkSession)
logInfo("column delete finished")
}
@@ -128,13 +128,13 @@ case class AlterTableCommand312(table: CatalogTable, changes: Seq[TableChange],
updateChange.updateColumnComment(updateComment.fieldNames().mkString("."), updateComment.newComment())
case updateName: TableChange.RenameColumn =>
val originalColName = updateName.fieldNames().mkString(".")
- AlterTableCommand312.checkSchemaChange(Seq(originalColName), table)
+ Spark31AlterTableCommand.checkSchemaChange(Seq(originalColName), table)
updateChange.renameColumn(originalColName, updateName.newName())
case updateNullAbility: TableChange.UpdateColumnNullability =>
updateChange.updateColumnNullability(updateNullAbility.fieldNames().mkString("."), updateNullAbility.nullable())
case updatePosition: TableChange.UpdateColumnPosition =>
val names = updatePosition.fieldNames()
- val parentName = AlterTableCommand312.getParentName(names)
+ val parentName = Spark31AlterTableCommand.getParentName(names)
updatePosition.position() match {
case after: TableChange.After =>
updateChange.addPositionChange(names.mkString("."),
@@ -151,7 +151,7 @@ case class AlterTableCommand312(table: CatalogTable, changes: Seq[TableChange],
} else {
historySchema
}
- AlterTableCommand312.commitWithSchema(newSchema, verifiedHistorySchema, table, sparkSession)
+ Spark31AlterTableCommand.commitWithSchema(newSchema, verifiedHistorySchema, table, sparkSession)
logInfo("column update finished")
}
@@ -187,7 +187,7 @@ case class AlterTableCommand312(table: CatalogTable, changes: Seq[TableChange],
}
def getInternalSchemaAndHistorySchemaStr(sparkSession: SparkSession): (InternalSchema, String) = {
- val path = AlterTableCommand312.getTableLocation(table, sparkSession)
+ val path = Spark31AlterTableCommand.getTableLocation(table, sparkSession)
val hadoopConf = sparkSession.sessionState.newHadoopConf()
val metaClient = HoodieTableMetaClient.builder().setBasePath(path)
.setConf(hadoopConf).build()
@@ -202,7 +202,7 @@ case class AlterTableCommand312(table: CatalogTable, changes: Seq[TableChange],
}
}
-object AlterTableCommand312 extends Logging {
+object Spark31AlterTableCommand extends Logging {
/**
* Generate an commit with new schema to change the table's schema.
diff --git a/hudi-spark-datasource/hudi-spark3.2.x/pom.xml b/hudi-spark-datasource/hudi-spark3.2.x/pom.xml
index 45064515217d1..7eebeb2200016 100644
--- a/hudi-spark-datasource/hudi-spark3.2.x/pom.xml
+++ b/hudi-spark-datasource/hudi-spark3.2.x/pom.xml
@@ -172,11 +172,6 @@
-
- org.scala-lang
- scala-library
- ${scala12.version}
- org.apache.spark
@@ -268,17 +263,22 @@
+
org.apache.hudihudi-spark3-common${project.version}
-
-
- org.apache.spark
- *
-
-
+ compile
+
+
+
+ org.apache.hudi
+ hudi-spark3.2plus-common
+ ${project.version}
+ compile
+
+
org.apache.hudi
diff --git a/hudi-spark-datasource/hudi-spark3.2.x/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister b/hudi-spark-datasource/hudi-spark3.2.x/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
index 33ab03f55477b..c8dd99a95c27a 100644
--- a/hudi-spark-datasource/hudi-spark3.2.x/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
+++ b/hudi-spark-datasource/hudi-spark3.2.x/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
@@ -16,4 +16,4 @@
# limitations under the License.
-org.apache.hudi.Spark3DefaultSource
\ No newline at end of file
+org.apache.hudi.Spark32PlusDefaultSource
\ No newline at end of file
diff --git a/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/hudi/Spark32HoodieFileScanRDD.scala b/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/hudi/Spark32HoodieFileScanRDD.scala
index d7eafd71743eb..f407710e497de 100644
--- a/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/hudi/Spark32HoodieFileScanRDD.scala
+++ b/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/hudi/Spark32HoodieFileScanRDD.scala
@@ -18,12 +18,9 @@
package org.apache.hudi
-import org.apache.hudi.HoodieUnsafeRDD
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.expressions.AttributeReference
import org.apache.spark.sql.execution.datasources.{FilePartition, FileScanRDD, PartitionedFile}
-import org.apache.spark.sql.types.StructType
class Spark32HoodieFileScanRDD(@transient private val sparkSession: SparkSession,
read: PartitionedFile => Iterator[InternalRow],
diff --git a/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/HoodieSpark32CatalogUtils.scala b/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/HoodieSpark32CatalogUtils.scala
new file mode 100644
index 0000000000000..53ae6239b2822
--- /dev/null
+++ b/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/HoodieSpark32CatalogUtils.scala
@@ -0,0 +1,29 @@
+/*
+ * 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
+import org.apache.spark.sql.connector.expressions.{BucketTransform, NamedReference, Transform}
+
+object HoodieSpark32CatalogUtils extends HoodieSpark3CatalogUtils {
+
+ override def unapplyBucketTransform(t: Transform): Option[(Int, Seq[NamedReference], Seq[NamedReference])] =
+ t match {
+ case BucketTransform(numBuckets, ref) => Some(numBuckets, Seq(ref), Seq.empty)
+ case _ => None
+ }
+
+}
diff --git a/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_2Adapter.scala b/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_2Adapter.scala
index bd2d6e4f7bffc..8e15745d57bb1 100644
--- a/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_2Adapter.scala
+++ b/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_2Adapter.scala
@@ -17,24 +17,26 @@
package org.apache.spark.sql.adapter
-import org.apache.hudi.Spark32HoodieFileScanRDD
import org.apache.avro.Schema
+import org.apache.hudi.Spark32HoodieFileScanRDD
+import org.apache.spark.sql._
import org.apache.spark.sql.avro._
+import org.apache.spark.sql.catalyst.InternalRow
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.{Command, DeleteFromTable, LogicalPlan}
-import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.plans.logical.{Command, DeleteFromTable}
+import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Spark32PlusHoodieParquetFileFormat}
import org.apache.spark.sql.execution.datasources.{FilePartition, FileScanRDD, PartitionedFile}
-import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Spark32HoodieParquetFileFormat}
import org.apache.spark.sql.parser.HoodieSpark3_2ExtendedSqlParser
import org.apache.spark.sql.types.{DataType, StructType}
-import org.apache.spark.sql._
/**
* Implementation of [[SparkAdapter]] for Spark 3.2.x branch
*/
class Spark3_2Adapter extends BaseSpark3Adapter {
+ override def getCatalogUtils: HoodieSpark3CatalogUtils = HoodieSpark32CatalogUtils
+
override def getCatalystExpressionUtils: HoodieCatalystExpressionUtils = HoodieSpark32CatalystExpressionUtils
override def getCatalystPlanUtils: HoodieCatalystPlansUtils = HoodieSpark32CatalystPlanUtils
@@ -52,7 +54,7 @@ class Spark3_2Adapter extends BaseSpark3Adapter {
}
override def createHoodieParquetFileFormat(appendPartitionValues: Boolean): Option[ParquetFileFormat] = {
- Some(new Spark32HoodieParquetFileFormat(appendPartitionValues))
+ Some(new Spark32PlusHoodieParquetFileFormat(appendPartitionValues))
}
override def createHoodieFileScanRDD(sparkSession: SparkSession,
diff --git a/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32DataSourceUtils.scala b/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32DataSourceUtils.scala
deleted file mode 100644
index 6d1c76380f216..0000000000000
--- a/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32DataSourceUtils.scala
+++ /dev/null
@@ -1,77 +0,0 @@
-/*
- * 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.execution.datasources.parquet
-
-import org.apache.spark.sql.SPARK_VERSION_METADATA_KEY
-import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.internal.SQLConf.LegacyBehaviorPolicy
-import org.apache.spark.util.Utils
-
-object Spark32DataSourceUtils {
-
- /**
- * NOTE: This method was copied from Spark 3.2.0, and is required to maintain runtime
- * compatibility against Spark 3.2.0
- */
- // scalastyle:off
- def int96RebaseMode(lookupFileMeta: String => String,
- modeByConfig: String): LegacyBehaviorPolicy.Value = {
- if (Utils.isTesting && SQLConf.get.getConfString("spark.test.forceNoRebase", "") == "true") {
- return LegacyBehaviorPolicy.CORRECTED
- }
- // If there is no version, we return the mode specified by the config.
- Option(lookupFileMeta(SPARK_VERSION_METADATA_KEY)).map { version =>
- // Files written by Spark 3.0 and earlier follow the legacy hybrid calendar and we need to
- // rebase the INT96 timestamp values.
- // Files written by Spark 3.1 and latter may also need the rebase if they were written with
- // the "LEGACY" rebase mode.
- if (version < "3.1.0" || lookupFileMeta("org.apache.spark.legacyINT96") != null) {
- LegacyBehaviorPolicy.LEGACY
- } else {
- LegacyBehaviorPolicy.CORRECTED
- }
- }.getOrElse(LegacyBehaviorPolicy.withName(modeByConfig))
- }
- // scalastyle:on
-
- /**
- * NOTE: This method was copied from Spark 3.2.0, and is required to maintain runtime
- * compatibility against Spark 3.2.0
- */
- // scalastyle:off
- def datetimeRebaseMode(lookupFileMeta: String => String,
- modeByConfig: String): LegacyBehaviorPolicy.Value = {
- if (Utils.isTesting && SQLConf.get.getConfString("spark.test.forceNoRebase", "") == "true") {
- return LegacyBehaviorPolicy.CORRECTED
- }
- // If there is no version, we return the mode specified by the config.
- Option(lookupFileMeta(SPARK_VERSION_METADATA_KEY)).map { version =>
- // Files written by Spark 2.4 and earlier follow the legacy hybrid calendar and we need to
- // rebase the datetime values.
- // Files written by Spark 3.0 and latter may also need the rebase if they were written with
- // the "LEGACY" rebase mode.
- if (version < "3.0.0" || lookupFileMeta("org.apache.spark.legacyDateTime") != null) {
- LegacyBehaviorPolicy.LEGACY
- } else {
- LegacyBehaviorPolicy.CORRECTED
- }
- }.getOrElse(LegacyBehaviorPolicy.withName(modeByConfig))
- }
- // scalastyle:on
-
-}
diff --git a/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/hudi/Spark32ResolveHudiAlterTableCommand.scala b/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/hudi/Spark32ResolveHudiAlterTableCommand.scala
index cfc857145e175..271701032ff52 100644
--- a/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/hudi/Spark32ResolveHudiAlterTableCommand.scala
+++ b/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/hudi/Spark32ResolveHudiAlterTableCommand.scala
@@ -18,7 +18,6 @@
package org.apache.spark.sql.hudi
import org.apache.hudi.common.config.HoodieCommonConfig
-import org.apache.hudi.config.HoodieWriteConfig
import org.apache.hudi.internal.schema.action.TableChange.ColumnChangeID
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.catalyst.analysis.ResolvedTable
diff --git a/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/hudi/catalog/BasicStagedTable.scala b/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/hudi/catalog/BasicStagedTable.scala
deleted file mode 100644
index 67d9e1ebb2bf8..0000000000000
--- a/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/hudi/catalog/BasicStagedTable.scala
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * 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.catalog
-
-import org.apache.hudi.exception.HoodieException
-import org.apache.spark.sql.connector.catalog._
-import org.apache.spark.sql.connector.expressions.Transform
-import org.apache.spark.sql.connector.write.{LogicalWriteInfo, WriteBuilder}
-import org.apache.spark.sql.types.StructType
-
-import java.util
-
-/**
- * Basic implementation that represents a table which is staged for being committed.
- * @param ident table ident
- * @param table table
- * @param catalog table catalog
- */
-case class BasicStagedTable(ident: Identifier,
- table: Table,
- catalog: TableCatalog) extends SupportsWrite with StagedTable {
- override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = {
- info match {
- case supportsWrite: SupportsWrite => supportsWrite.newWriteBuilder(info)
- case _ => throw new HoodieException(s"Table `${ident.name}` does not support writes.")
- }
- }
-
- override def abortStagedChanges(): Unit = catalog.dropTable(ident)
-
- override def commitStagedChanges(): Unit = {}
-
- override def name(): String = ident.name()
-
- override def schema(): StructType = table.schema()
-
- override def partitioning(): Array[Transform] = table.partitioning()
-
- override def capabilities(): util.Set[TableCapability] = table.capabilities()
-
- override def properties(): util.Map[String, String] = table.properties()
-}
diff --git a/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/hudi/catalog/HoodieInternalV2Table.scala b/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/hudi/catalog/HoodieInternalV2Table.scala
deleted file mode 100644
index 9eb4a773f8d4f..0000000000000
--- a/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/hudi/catalog/HoodieInternalV2Table.scala
+++ /dev/null
@@ -1,129 +0,0 @@
-/*
- * 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.catalog
-
-import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient}
-import org.apache.spark.sql.catalyst.TableIdentifier
-import org.apache.spark.sql.catalyst.catalog.{CatalogTable, HoodieCatalogTable}
-import org.apache.spark.sql.connector.catalog.TableCapability._
-import org.apache.spark.sql.connector.catalog.{SupportsWrite, Table, TableCapability, V1Table, V2TableWithV1Fallback}
-import org.apache.spark.sql.connector.expressions.{FieldReference, IdentityTransform, Transform}
-import org.apache.spark.sql.connector.write._
-import org.apache.spark.sql.hudi.ProvidesHoodieConfig
-import org.apache.spark.sql.sources.{Filter, InsertableRelation}
-import org.apache.spark.sql.types.StructType
-import org.apache.spark.sql.util.CaseInsensitiveStringMap
-import org.apache.spark.sql.{DataFrame, SaveMode, SparkSession}
-
-import java.util
-import scala.collection.JavaConverters.{mapAsJavaMapConverter, setAsJavaSetConverter}
-
-case class HoodieInternalV2Table(spark: SparkSession,
- path: String,
- catalogTable: Option[CatalogTable] = None,
- tableIdentifier: Option[String] = None,
- options: CaseInsensitiveStringMap = CaseInsensitiveStringMap.empty())
- extends Table with SupportsWrite with V2TableWithV1Fallback {
-
- lazy val hoodieCatalogTable: HoodieCatalogTable = if (catalogTable.isDefined) {
- HoodieCatalogTable(spark, catalogTable.get)
- } else {
- val metaClient: HoodieTableMetaClient = HoodieTableMetaClient.builder()
- .setBasePath(path)
- .setConf(SparkSession.active.sessionState.newHadoopConf)
- .build()
-
- val tableConfig: HoodieTableConfig = metaClient.getTableConfig
- val tableName: String = tableConfig.getTableName
-
- HoodieCatalogTable(spark, TableIdentifier(tableName))
- }
-
- private lazy val tableSchema: StructType = hoodieCatalogTable.tableSchema
-
- override def name(): String = hoodieCatalogTable.table.identifier.unquotedString
-
- override def schema(): StructType = tableSchema
-
- override def capabilities(): util.Set[TableCapability] = Set(
- BATCH_READ, V1_BATCH_WRITE, OVERWRITE_BY_FILTER, TRUNCATE, ACCEPT_ANY_SCHEMA
- ).asJava
-
- override def properties(): util.Map[String, String] = {
- hoodieCatalogTable.catalogProperties.asJava
- }
-
- override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = {
- new HoodieV1WriteBuilder(info.options, hoodieCatalogTable, spark)
- }
-
- override def v1Table: CatalogTable = hoodieCatalogTable.table
-
- def v1TableWrapper: V1Table = V1Table(v1Table)
-
- override def partitioning(): Array[Transform] = {
- hoodieCatalogTable.partitionFields.map { col =>
- new IdentityTransform(new FieldReference(Seq(col)))
- }.toArray
- }
-
-}
-
-private class HoodieV1WriteBuilder(writeOptions: CaseInsensitiveStringMap,
- hoodieCatalogTable: HoodieCatalogTable,
- spark: SparkSession)
- extends SupportsTruncate with SupportsOverwrite with ProvidesHoodieConfig {
-
- private var forceOverwrite = false
-
- override def truncate(): HoodieV1WriteBuilder = {
- forceOverwrite = true
- this
- }
-
- override def overwrite(filters: Array[Filter]): WriteBuilder = {
- forceOverwrite = true
- this
- }
-
- override def build(): V1Write = new V1Write {
- override def toInsertableRelation: InsertableRelation = {
- new InsertableRelation {
- override def insert(data: DataFrame, overwrite: Boolean): Unit = {
- val mode = if (forceOverwrite && hoodieCatalogTable.partitionFields.isEmpty) {
- // insert overwrite non-partition table
- SaveMode.Overwrite
- } else {
- // for insert into or insert overwrite partition we use append mode.
- SaveMode.Append
- }
- alignOutputColumns(data).write.format("org.apache.hudi")
- .mode(mode)
- .options(buildHoodieConfig(hoodieCatalogTable) ++
- buildHoodieInsertConfig(hoodieCatalogTable, spark, forceOverwrite, Map.empty, Map.empty))
- .save()
- }
- }
- }
- }
-
- private def alignOutputColumns(data: DataFrame): DataFrame = {
- val schema = hoodieCatalogTable.tableSchema
- spark.createDataFrame(data.toJavaRDD, schema)
- }
-}
diff --git a/hudi-spark-datasource/hudi-spark3.2plus-common/pom.xml b/hudi-spark-datasource/hudi-spark3.2plus-common/pom.xml
new file mode 100644
index 0000000000000..6034d44e17012
--- /dev/null
+++ b/hudi-spark-datasource/hudi-spark3.2plus-common/pom.xml
@@ -0,0 +1,234 @@
+
+
+
+
+ hudi-spark-datasource
+ org.apache.hudi
+ 0.13.0-SNAPSHOT
+
+ 4.0.0
+
+ hudi-spark3.2plus-common
+
+
+ ${project.parent.parent.basedir}
+ 8
+ 8
+
+
+
+
+
+ src/main/resources
+
+
+
+
+
+ net.alchim31.maven
+ scala-maven-plugin
+ ${scala-maven-plugin.version}
+
+
+ -nobootcp
+
+ false
+
+
+
+ org.apache.maven.plugins
+ maven-compiler-plugin
+
+
+
+
+
+
+ org.apache.maven.plugins
+ maven-dependency-plugin
+
+
+ copy-dependencies
+ prepare-package
+
+ copy-dependencies
+
+
+ ${project.build.directory}/lib
+ true
+ true
+ true
+
+
+
+
+
+ net.alchim31.maven
+ scala-maven-plugin
+
+
+ scala-compile-first
+ process-resources
+
+ add-source
+ compile
+
+
+
+ scala-test-compile
+ process-test-resources
+
+ testCompile
+
+
+
+
+ 1.8
+ 1.8
+
+
+
+ org.apache.maven.plugins
+ maven-compiler-plugin
+
+
+ compile
+
+ compile
+
+
+
+
+ 1.8
+ 1.8
+
+
+
+ org.apache.maven.plugins
+ maven-jar-plugin
+
+
+
+ test-jar
+
+ test-compile
+
+
+
+ false
+
+
+
+ org.apache.maven.plugins
+ maven-surefire-plugin
+
+ ${skip.hudi-spark3.unit.tests}
+
+
+
+ org.apache.rat
+ apache-rat-plugin
+
+
+ org.scalastyle
+ scalastyle-maven-plugin
+
+
+ org.jacoco
+ jacoco-maven-plugin
+
+
+
+
+
+
+ org.apache.spark
+ spark-sql_2.12
+ ${spark3.version}
+ provided
+ true
+
+
+
+
+ org.apache.hudi
+ hudi-spark-client
+ ${project.version}
+
+
+
+ org.apache.hudi
+ hudi-spark3-common
+ ${project.version}
+
+
+
+
+ org.apache.hudi
+ hudi-tests-common
+ ${project.version}
+ test
+
+
+ org.apache.hudi
+ hudi-client-common
+ ${project.version}
+ tests
+ test-jar
+ test
+
+
+ org.apache.hudi
+ hudi-spark-client
+ ${project.version}
+ tests
+ test-jar
+ test
+
+
+ org.apache.hudi
+ hudi-common
+ ${project.version}
+ tests
+ test-jar
+ test
+
+
+ org.apache.hudi
+ hudi-spark-common_${scala.binary.version}
+ ${project.version}
+ tests
+ test-jar
+ test
+
+
+
+ org.junit.jupiter
+ junit-jupiter-api
+ test
+
+
+ org.junit.jupiter
+ junit-jupiter-params
+ test
+
+
+
+
diff --git a/hudi-spark-datasource/hudi-spark3.2.x/src/main/java/org/apache/spark/sql/execution/datasources/parquet/Spark32HoodieVectorizedParquetRecordReader.java b/hudi-spark-datasource/hudi-spark3.2plus-common/src/main/java/org/apache/spark/sql/execution/datasources/parquet/Spark32PlusHoodieVectorizedParquetRecordReader.java
similarity index 97%
rename from hudi-spark-datasource/hudi-spark3.2.x/src/main/java/org/apache/spark/sql/execution/datasources/parquet/Spark32HoodieVectorizedParquetRecordReader.java
rename to hudi-spark-datasource/hudi-spark3.2plus-common/src/main/java/org/apache/spark/sql/execution/datasources/parquet/Spark32PlusHoodieVectorizedParquetRecordReader.java
index abff023104c93..6ce054c5955f3 100644
--- a/hudi-spark-datasource/hudi-spark3.2.x/src/main/java/org/apache/spark/sql/execution/datasources/parquet/Spark32HoodieVectorizedParquetRecordReader.java
+++ b/hudi-spark-datasource/hudi-spark3.2plus-common/src/main/java/org/apache/spark/sql/execution/datasources/parquet/Spark32PlusHoodieVectorizedParquetRecordReader.java
@@ -35,7 +35,7 @@
import java.util.HashMap;
import java.util.Map;
-public class Spark32HoodieVectorizedParquetRecordReader extends VectorizedParquetRecordReader {
+public class Spark32PlusHoodieVectorizedParquetRecordReader extends VectorizedParquetRecordReader {
// save the col type change info.
private Map> typeChangeInfos;
@@ -62,7 +62,7 @@ public class Spark32HoodieVectorizedParquetRecordReader extends VectorizedParque
private int batchIdx = 0;
private int numBatched = 0;
- public Spark32HoodieVectorizedParquetRecordReader(
+ public Spark32PlusHoodieVectorizedParquetRecordReader(
ZoneId convertTz,
String datetimeRebaseMode,
String datetimeRebaseTz,
diff --git a/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/hudi/Spark3DefaultSource.scala b/hudi-spark-datasource/hudi-spark3.2plus-common/src/main/scala/org/apache/hudi/Spark32PlusDefaultSource.scala
similarity index 94%
rename from hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/hudi/Spark3DefaultSource.scala
rename to hudi-spark-datasource/hudi-spark3.2plus-common/src/main/scala/org/apache/hudi/Spark32PlusDefaultSource.scala
index 3bc3446d1f120..339c6a33083e2 100644
--- a/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/hudi/Spark3DefaultSource.scala
+++ b/hudi-spark-datasource/hudi-spark3.2plus-common/src/main/scala/org/apache/hudi/Spark32PlusDefaultSource.scala
@@ -25,7 +25,7 @@ import org.apache.spark.sql.sources.DataSourceRegister
* there are no regressions in performance
* Please check out HUDI-4178 for more details
*/
-class Spark3DefaultSource extends DefaultSource with DataSourceRegister /* with TableProvider */ {
+class Spark32PlusDefaultSource extends DefaultSource with DataSourceRegister /* with TableProvider */ {
override def shortName(): String = "hudi"
diff --git a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33DataSourceUtils.scala b/hudi-spark-datasource/hudi-spark3.2plus-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32PlusDataSourceUtils.scala
similarity index 98%
rename from hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33DataSourceUtils.scala
rename to hudi-spark-datasource/hudi-spark3.2plus-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32PlusDataSourceUtils.scala
index 2aa85660eb511..5c3f5a976c25f 100644
--- a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33DataSourceUtils.scala
+++ b/hudi-spark-datasource/hudi-spark3.2plus-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32PlusDataSourceUtils.scala
@@ -22,7 +22,7 @@ import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.internal.SQLConf.LegacyBehaviorPolicy
import org.apache.spark.util.Utils
-object Spark33DataSourceUtils {
+object Spark32PlusDataSourceUtils {
/**
* NOTE: This method was copied from Spark 3.2.0, and is required to maintain runtime
diff --git a/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32HoodieParquetFileFormat.scala b/hudi-spark-datasource/hudi-spark3.2plus-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32PlusHoodieParquetFileFormat.scala
similarity index 96%
rename from hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32HoodieParquetFileFormat.scala
rename to hudi-spark-datasource/hudi-spark3.2plus-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32PlusHoodieParquetFileFormat.scala
index 7135f19e95e2d..52d450029e3bd 100644
--- a/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32HoodieParquetFileFormat.scala
+++ b/hudi-spark-datasource/hudi-spark3.2plus-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32PlusHoodieParquetFileFormat.scala
@@ -41,7 +41,7 @@ import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection
import org.apache.spark.sql.catalyst.expressions.{Cast, JoinedRow}
import org.apache.spark.sql.catalyst.util.DateTimeUtils
-import org.apache.spark.sql.execution.datasources.parquet.Spark32HoodieParquetFileFormat._
+import org.apache.spark.sql.execution.datasources.parquet.Spark32PlusHoodieParquetFileFormat._
import org.apache.spark.sql.execution.datasources.{DataSourceUtils, PartitionedFile, RecordReaderIterator}
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.sources._
@@ -60,7 +60,7 @@ import java.net.URI
*
Schema on-read
*
*/
-class Spark32HoodieParquetFileFormat(private val shouldAppendPartitionValues: Boolean) extends ParquetFileFormat {
+class Spark32PlusHoodieParquetFileFormat(private val shouldAppendPartitionValues: Boolean) extends ParquetFileFormat {
override def buildReaderWithPartitionValues(sparkSession: SparkSession,
dataSchema: StructType,
@@ -180,7 +180,7 @@ class Spark32HoodieParquetFileFormat(private val shouldAppendPartitionValues: Bo
} else {
// Spark 3.2.0
val datetimeRebaseMode =
- Spark32DataSourceUtils.datetimeRebaseMode(footerFileMetaData.getKeyValueMetaData.get, datetimeRebaseModeInRead)
+ Spark32PlusDataSourceUtils.datetimeRebaseMode(footerFileMetaData.getKeyValueMetaData.get, datetimeRebaseModeInRead)
createParquetFilters(
parquetSchema,
pushDownDate,
@@ -247,7 +247,7 @@ class Spark32HoodieParquetFileFormat(private val shouldAppendPartitionValues: Bo
DataSourceUtils.int96RebaseSpec(footerFileMetaData.getKeyValueMetaData.get, int96RebaseModeInRead)
val datetimeRebaseSpec =
DataSourceUtils.datetimeRebaseSpec(footerFileMetaData.getKeyValueMetaData.get, datetimeRebaseModeInRead)
- new Spark32HoodieVectorizedParquetRecordReader(
+ new Spark32PlusHoodieVectorizedParquetRecordReader(
convertTz.orNull,
datetimeRebaseSpec.mode.toString,
datetimeRebaseSpec.timeZone,
@@ -275,9 +275,9 @@ class Spark32HoodieParquetFileFormat(private val shouldAppendPartitionValues: Bo
} else {
// Spark 3.2.0
val datetimeRebaseMode =
- Spark32DataSourceUtils.datetimeRebaseMode(footerFileMetaData.getKeyValueMetaData.get, datetimeRebaseModeInRead)
+ Spark32PlusDataSourceUtils.datetimeRebaseMode(footerFileMetaData.getKeyValueMetaData.get, datetimeRebaseModeInRead)
val int96RebaseMode =
- Spark32DataSourceUtils.int96RebaseMode(footerFileMetaData.getKeyValueMetaData.get, int96RebaseModeInRead)
+ Spark32PlusDataSourceUtils.int96RebaseMode(footerFileMetaData.getKeyValueMetaData.get, int96RebaseModeInRead)
createVectorizedParquetRecordReader(
convertTz.orNull,
datetimeRebaseMode.toString,
@@ -337,9 +337,9 @@ class Spark32HoodieParquetFileFormat(private val shouldAppendPartitionValues: Bo
int96RebaseSpec)
} else {
val datetimeRebaseMode =
- Spark32DataSourceUtils.datetimeRebaseMode(footerFileMetaData.getKeyValueMetaData.get, datetimeRebaseModeInRead)
+ Spark32PlusDataSourceUtils.datetimeRebaseMode(footerFileMetaData.getKeyValueMetaData.get, datetimeRebaseModeInRead)
val int96RebaseMode =
- Spark32DataSourceUtils.int96RebaseMode(footerFileMetaData.getKeyValueMetaData.get, int96RebaseModeInRead)
+ Spark32PlusDataSourceUtils.int96RebaseMode(footerFileMetaData.getKeyValueMetaData.get, int96RebaseModeInRead)
createParquetReadSupport(
convertTz,
/* enableVectorizedReader = */ false,
@@ -397,7 +397,7 @@ class Spark32HoodieParquetFileFormat(private val shouldAppendPartitionValues: Bo
}
-object Spark32HoodieParquetFileFormat {
+object Spark32PlusHoodieParquetFileFormat {
/**
* NOTE: This method is specific to Spark 3.2.0
diff --git a/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieSpark3Analysis.scala b/hudi-spark-datasource/hudi-spark3.2plus-common/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieSpark3Analysis.scala
similarity index 80%
rename from hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieSpark3Analysis.scala
rename to hudi-spark-datasource/hudi-spark3.2plus-common/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieSpark3Analysis.scala
index fb5b048926929..10d2208edc446 100644
--- a/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieSpark3Analysis.scala
+++ b/hudi-spark-datasource/hudi-spark3.2plus-common/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieSpark3Analysis.scala
@@ -17,8 +17,8 @@
package org.apache.spark.sql.hudi.analysis
-import org.apache.hudi.{DefaultSource, SparkAdapterSupport}
import org.apache.hudi.common.table.HoodieTableMetaClient
+import org.apache.hudi.{DefaultSource, SparkAdapterSupport}
import org.apache.spark.sql.catalyst.TableIdentifier
import org.apache.spark.sql.catalyst.analysis.{ResolvedTable, UnresolvedPartitionSpec}
import org.apache.spark.sql.catalyst.catalog.{CatalogTable, HoodieCatalogTable}
@@ -129,50 +129,6 @@ class HoodieSpark3Analysis(sparkSession: SparkSession) extends Rule[LogicalPlan]
}
}
-/**
- * Rule for resolve hoodie's extended syntax or rewrite some logical plan.
- */
-case class HoodieSpark3ResolveReferences(sparkSession: SparkSession) extends Rule[LogicalPlan]
- with SparkAdapterSupport with ProvidesHoodieConfig {
-
- def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperatorsUp {
- // Fill schema for Create Table without specify schema info
- case c @ CreateV2Table(tableCatalog, tableName, schema, partitioning, properties, _)
- if sparkAdapter.isHoodieTable(properties.asJava) =>
-
- if (schema.isEmpty && partitioning.nonEmpty) {
- failAnalysis("It is not allowed to specify partition columns when the table schema is " +
- "not defined. When the table schema is not provided, schema and partition columns " +
- "will be inferred.")
- }
-
- val tablePath = getTableLocation(properties,
- TableIdentifier(tableName.name(), tableName.namespace().lastOption), sparkSession)
-
- val tableExistInCatalog = tableCatalog.tableExists(tableName)
- // Only when the table has not exist in catalog, we need to fill the schema info for creating table.
- if (!tableExistInCatalog && tableExistsInPath(tablePath, sparkSession.sessionState.newHadoopConf())) {
- val metaClient = HoodieTableMetaClient.builder()
- .setBasePath(tablePath)
- .setConf(sparkSession.sessionState.newHadoopConf())
- .build()
- val tableSchema = HoodieSqlCommonUtils.getTableSqlSchema(metaClient)
- if (tableSchema.isDefined && schema.isEmpty) {
- // Fill the schema with the schema from the table
- c.copy(tableSchema = tableSchema.get)
- } else if (tableSchema.isDefined && schema != tableSchema.get) {
- throw new AnalysisException(s"Specified schema in create table statement is not equal to the table schema." +
- s"You should not specify the schema for an exist table: $tableName ")
- } else {
- c
- }
- } else {
- c
- }
- case p => p
- }
-}
-
/**
* Rule replacing resolved Spark's commands (not working for Hudi tables out-of-the-box) with
* corresponding Hudi implementations
diff --git a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/hudi/catalog/BasicStagedTable.scala b/hudi-spark-datasource/hudi-spark3.2plus-common/src/main/scala/org/apache/spark/sql/hudi/catalog/BasicStagedTable.scala
similarity index 76%
rename from hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/hudi/catalog/BasicStagedTable.scala
rename to hudi-spark-datasource/hudi-spark3.2plus-common/src/main/scala/org/apache/spark/sql/hudi/catalog/BasicStagedTable.scala
index 67d9e1ebb2bf8..41d31e591da8f 100644
--- a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/hudi/catalog/BasicStagedTable.scala
+++ b/hudi-spark-datasource/hudi-spark3.2plus-common/src/main/scala/org/apache/spark/sql/hudi/catalog/BasicStagedTable.scala
@@ -1,12 +1,13 @@
/*
- * 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
+ * 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
+ * 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,
@@ -27,8 +28,9 @@ import java.util
/**
* Basic implementation that represents a table which is staged for being committed.
- * @param ident table ident
- * @param table table
+ *
+ * @param ident table ident
+ * @param table table
* @param catalog table catalog
*/
case class BasicStagedTable(ident: Identifier,
diff --git a/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/hudi/catalog/HoodieCatalog.scala b/hudi-spark-datasource/hudi-spark3.2plus-common/src/main/scala/org/apache/spark/sql/hudi/catalog/HoodieCatalog.scala
similarity index 94%
rename from hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/hudi/catalog/HoodieCatalog.scala
rename to hudi-spark-datasource/hudi-spark3.2plus-common/src/main/scala/org/apache/spark/sql/hudi/catalog/HoodieCatalog.scala
index 671fafedec080..eeef56d3cff74 100644
--- a/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/hudi/catalog/HoodieCatalog.scala
+++ b/hudi-spark-datasource/hudi-spark3.2plus-common/src/main/scala/org/apache/spark/sql/hudi/catalog/HoodieCatalog.scala
@@ -23,17 +23,17 @@ import org.apache.hudi.exception.HoodieException
import org.apache.hudi.sql.InsertMode
import org.apache.hudi.sync.common.util.ConfigUtils
import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions, SparkAdapterSupport}
+import org.apache.spark.sql.HoodieSpark3CatalogUtils.MatchBucketTransform
import org.apache.spark.sql.catalyst.TableIdentifier
import org.apache.spark.sql.catalyst.analysis.{NoSuchTableException, TableAlreadyExistsException, UnresolvedAttribute}
import org.apache.spark.sql.catalyst.catalog.HoodieCatalogTable.needFilterProps
-import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogTable, CatalogTableType, CatalogUtils, HoodieCatalogTable}
+import org.apache.spark.sql.catalyst.catalog._
import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.IdentifierHelper
import org.apache.spark.sql.connector.catalog.TableChange.{AddColumn, ColumnChange, UpdateColumnComment, UpdateColumnType}
import org.apache.spark.sql.connector.catalog._
-import org.apache.spark.sql.connector.expressions.{BucketTransform, FieldReference, IdentityTransform, Transform}
+import org.apache.spark.sql.connector.expressions.{FieldReference, IdentityTransform, Transform}
import org.apache.spark.sql.execution.datasources.DataSource
import org.apache.spark.sql.hudi.analysis.HoodieV1OrV2Table
-import org.apache.spark.sql.hudi.catalog.HoodieCatalog.convertTransforms
import org.apache.spark.sql.hudi.command._
import org.apache.spark.sql.hudi.{HoodieSqlCommonUtils, ProvidesHoodieConfig}
import org.apache.spark.sql.types.{StructField, StructType}
@@ -346,20 +346,25 @@ class HoodieCatalog extends DelegatingCatalogExtension
}
object HoodieCatalog {
- def convertTransforms(partitions: Seq[Transform]): (Seq[String], Option[BucketSpec]) = {
+ def convertTransforms(transforms: Seq[Transform]): (Seq[String], Option[BucketSpec]) = {
val identityCols = new mutable.ArrayBuffer[String]
var bucketSpec = Option.empty[BucketSpec]
- partitions.map {
+ transforms.foreach {
case IdentityTransform(FieldReference(Seq(col))) =>
identityCols += col
+ case MatchBucketTransform(numBuckets, col, sortCol) =>
+ if (bucketSpec.nonEmpty) {
+ throw new HoodieException("Multiple bucket transformations are not supported")
+ } else if (sortCol.isEmpty) {
+ bucketSpec = Some(BucketSpec(numBuckets, col.map(_.fieldNames.mkString(".")), Nil))
+ } else {
+ bucketSpec = Some(BucketSpec(numBuckets, col.map(_.fieldNames.mkString(".")),
+ sortCol.map(_.fieldNames.mkString("."))))
+ }
- case BucketTransform(numBuckets, FieldReference(Seq(col))) =>
- bucketSpec = Some(BucketSpec(numBuckets, col :: Nil, Nil))
-
- case _ =>
- throw new HoodieException(s"Partitioning by expressions is not supported.")
+ case t => throw new HoodieException(s"Partitioning by transformation `$t` is not supported")
}
(identityCols, bucketSpec)
diff --git a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/hudi/catalog/HoodieInternalV2Table.scala b/hudi-spark-datasource/hudi-spark3.2plus-common/src/main/scala/org/apache/spark/sql/hudi/catalog/HoodieInternalV2Table.scala
similarity index 97%
rename from hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/hudi/catalog/HoodieInternalV2Table.scala
rename to hudi-spark-datasource/hudi-spark3.2plus-common/src/main/scala/org/apache/spark/sql/hudi/catalog/HoodieInternalV2Table.scala
index 9eb4a773f8d4f..9968095f3a5d3 100644
--- a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/hudi/catalog/HoodieInternalV2Table.scala
+++ b/hudi-spark-datasource/hudi-spark3.2plus-common/src/main/scala/org/apache/spark/sql/hudi/catalog/HoodieInternalV2Table.scala
@@ -21,7 +21,7 @@ import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient}
import org.apache.spark.sql.catalyst.TableIdentifier
import org.apache.spark.sql.catalyst.catalog.{CatalogTable, HoodieCatalogTable}
import org.apache.spark.sql.connector.catalog.TableCapability._
-import org.apache.spark.sql.connector.catalog.{SupportsWrite, Table, TableCapability, V1Table, V2TableWithV1Fallback}
+import org.apache.spark.sql.connector.catalog._
import org.apache.spark.sql.connector.expressions.{FieldReference, IdentityTransform, Transform}
import org.apache.spark.sql.connector.write._
import org.apache.spark.sql.hudi.ProvidesHoodieConfig
diff --git a/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/hudi/catalog/HoodieStagedTable.scala b/hudi-spark-datasource/hudi-spark3.2plus-common/src/main/scala/org/apache/spark/sql/hudi/catalog/HoodieStagedTable.scala
similarity index 85%
rename from hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/hudi/catalog/HoodieStagedTable.scala
rename to hudi-spark-datasource/hudi-spark3.2plus-common/src/main/scala/org/apache/spark/sql/hudi/catalog/HoodieStagedTable.scala
index e18f23ebde03f..380c816e34895 100644
--- a/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/hudi/catalog/HoodieStagedTable.scala
+++ b/hudi-spark-datasource/hudi-spark3.2plus-common/src/main/scala/org/apache/spark/sql/hudi/catalog/HoodieStagedTable.scala
@@ -1,12 +1,13 @@
/*
- * 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
+ * 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
+ * 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,
@@ -29,7 +30,7 @@ import org.apache.spark.sql.types.StructType
import java.net.URI
import java.util
-import scala.collection.JavaConverters.{mapAsScalaMapConverter, setAsJavaSetConverter}
+import scala.jdk.CollectionConverters.{mapAsScalaMapConverter, setAsJavaSetConverter}
case class HoodieStagedTable(ident: Identifier,
locUriAndTableType: (URI, CatalogTableType),
diff --git a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/hudi/catalog/TableCreationMode.java b/hudi-spark-datasource/hudi-spark3.2plus-common/src/main/scala/org/apache/spark/sql/hudi/catalog/TableCreationMode.java
similarity index 97%
rename from hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/hudi/catalog/TableCreationMode.java
rename to hudi-spark-datasource/hudi-spark3.2plus-common/src/main/scala/org/apache/spark/sql/hudi/catalog/TableCreationMode.java
index 8b54775be149e..30c7d4cfc798f 100644
--- a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/hudi/catalog/TableCreationMode.java
+++ b/hudi-spark-datasource/hudi-spark3.2plus-common/src/main/scala/org/apache/spark/sql/hudi/catalog/TableCreationMode.java
@@ -1,23 +1,23 @@
-/*
- * 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.catalog;
-
-public enum TableCreationMode {
- CREATE, CREATE_OR_REPLACE, STAGE_CREATE, STAGE_REPLACE
-}
+/*
+ * 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.catalog;
+
+public enum TableCreationMode {
+ CREATE, CREATE_OR_REPLACE, STAGE_CREATE, STAGE_REPLACE
+}
diff --git a/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/hudi/command/AlterTableCommand.scala b/hudi-spark-datasource/hudi-spark3.2plus-common/src/main/scala/org/apache/spark/sql/hudi/command/AlterTableCommand.scala
similarity index 100%
rename from hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/hudi/command/AlterTableCommand.scala
rename to hudi-spark-datasource/hudi-spark3.2plus-common/src/main/scala/org/apache/spark/sql/hudi/command/AlterTableCommand.scala
diff --git a/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TimeTravelRelation.scala b/hudi-spark-datasource/hudi-spark3.2plus-common/src/main/scala/org/apache/spark/sql/hudi/logical/TimeTravelRelation.scala
similarity index 100%
rename from hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TimeTravelRelation.scala
rename to hudi-spark-datasource/hudi-spark3.2plus-common/src/main/scala/org/apache/spark/sql/hudi/logical/TimeTravelRelation.scala
diff --git a/hudi-spark-datasource/hudi-spark3.3.x/pom.xml b/hudi-spark-datasource/hudi-spark3.3.x/pom.xml
index 8818c63de1858..3979581a0a660 100644
--- a/hudi-spark-datasource/hudi-spark3.3.x/pom.xml
+++ b/hudi-spark-datasource/hudi-spark3.3.x/pom.xml
@@ -172,11 +172,6 @@
-
- org.scala-lang
- scala-library
- ${scala12.version}
- org.apache.spark
@@ -248,12 +243,20 @@
+
org.apache.hudihudi-spark3-common${project.version}
+
+
+ org.apache.hudi
+ hudi-spark3.2plus-common
+ ${project.version}
+
+
org.apache.hudi
diff --git a/hudi-spark-datasource/hudi-spark3.3.x/src/main/java/org/apache/spark/sql/execution/datasources/parquet/Spark33HoodieVectorizedParquetRecordReader.java b/hudi-spark-datasource/hudi-spark3.3.x/src/main/java/org/apache/spark/sql/execution/datasources/parquet/Spark33HoodieVectorizedParquetRecordReader.java
deleted file mode 100644
index 28d69aa005499..0000000000000
--- a/hudi-spark-datasource/hudi-spark3.3.x/src/main/java/org/apache/spark/sql/execution/datasources/parquet/Spark33HoodieVectorizedParquetRecordReader.java
+++ /dev/null
@@ -1,191 +0,0 @@
-/*
- * 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.execution.datasources.parquet;
-
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hudi.client.utils.SparkInternalSchemaConverter;
-import org.apache.hudi.common.util.collection.Pair;
-import org.apache.spark.memory.MemoryMode;
-import org.apache.spark.sql.catalyst.InternalRow;
-import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector;
-import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
-import org.apache.spark.sql.execution.vectorized.WritableColumnVector;
-import org.apache.spark.sql.types.DataType;
-import org.apache.spark.sql.types.StructType;
-import org.apache.spark.sql.vectorized.ColumnarBatch;
-
-import java.io.IOException;
-import java.time.ZoneId;
-import java.util.HashMap;
-import java.util.Map;
-
-public class Spark33HoodieVectorizedParquetRecordReader extends VectorizedParquetRecordReader {
-
- // save the col type change info.
- private Map> typeChangeInfos;
-
- private ColumnarBatch columnarBatch;
-
- private Map idToColumnVectors;
-
- private WritableColumnVector[] columnVectors;
-
- // The capacity of vectorized batch.
- private int capacity;
-
- // If true, this class returns batches instead of rows.
- private boolean returnColumnarBatch;
-
- // The memory mode of the columnarBatch.
- private final MemoryMode memoryMode;
-
- /**
- * Batch of rows that we assemble and the current index we've returned. Every time this
- * batch is used up (batchIdx == numBatched), we populated the batch.
- */
- private int batchIdx = 0;
- private int numBatched = 0;
-
- public Spark33HoodieVectorizedParquetRecordReader(
- ZoneId convertTz,
- String datetimeRebaseMode,
- String datetimeRebaseTz,
- String int96RebaseMode,
- String int96RebaseTz,
- boolean useOffHeap,
- int capacity,
- Map> typeChangeInfos) {
- super(convertTz, datetimeRebaseMode, datetimeRebaseTz, int96RebaseMode, int96RebaseTz, useOffHeap, capacity);
- memoryMode = useOffHeap ? MemoryMode.OFF_HEAP : MemoryMode.ON_HEAP;
- this.typeChangeInfos = typeChangeInfos;
- this.capacity = capacity;
- }
-
- @Override
- public void initBatch(StructType partitionColumns, InternalRow partitionValues) {
- super.initBatch(partitionColumns, partitionValues);
- if (columnVectors == null) {
- columnVectors = new WritableColumnVector[sparkSchema.length() + partitionColumns.length()];
- }
- if (idToColumnVectors == null) {
- idToColumnVectors = new HashMap<>();
- typeChangeInfos.entrySet()
- .stream()
- .forEach(f -> {
- WritableColumnVector vector =
- memoryMode == MemoryMode.OFF_HEAP ? new OffHeapColumnVector(capacity, f.getValue().getLeft()) : new OnHeapColumnVector(capacity, f.getValue().getLeft());
- idToColumnVectors.put(f.getKey(), vector);
- });
- }
- }
-
- @Override
- public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext) throws IOException, InterruptedException, UnsupportedOperationException {
- super.initialize(inputSplit, taskAttemptContext);
- }
-
- @Override
- public void close() throws IOException {
- super.close();
- for (Map.Entry e : idToColumnVectors.entrySet()) {
- e.getValue().close();
- }
- idToColumnVectors = null;
- columnarBatch = null;
- columnVectors = null;
- }
-
- @Override
- public ColumnarBatch resultBatch() {
- ColumnarBatch currentColumnBatch = super.resultBatch();
- boolean changed = false;
- for (Map.Entry> entry : typeChangeInfos.entrySet()) {
- boolean rewrite = SparkInternalSchemaConverter
- .convertColumnVectorType((WritableColumnVector) currentColumnBatch.column(entry.getKey()),
- idToColumnVectors.get(entry.getKey()), currentColumnBatch.numRows());
- if (rewrite) {
- changed = true;
- columnVectors[entry.getKey()] = idToColumnVectors.get(entry.getKey());
- }
- }
- if (changed) {
- if (columnarBatch == null) {
- // fill other vector
- for (int i = 0; i < columnVectors.length; i++) {
- if (columnVectors[i] == null) {
- columnVectors[i] = (WritableColumnVector) currentColumnBatch.column(i);
- }
- }
- columnarBatch = new ColumnarBatch(columnVectors);
- }
- columnarBatch.setNumRows(currentColumnBatch.numRows());
- return columnarBatch;
- } else {
- return currentColumnBatch;
- }
- }
-
- @Override
- public boolean nextBatch() throws IOException {
- boolean result = super.nextBatch();
- if (idToColumnVectors != null) {
- idToColumnVectors.entrySet().stream().forEach(e -> e.getValue().reset());
- }
- numBatched = resultBatch().numRows();
- batchIdx = 0;
- return result;
- }
-
- @Override
- public void enableReturningBatches() {
- returnColumnarBatch = true;
- super.enableReturningBatches();
- }
-
- @Override
- public Object getCurrentValue() {
- if (typeChangeInfos == null || typeChangeInfos.isEmpty()) {
- return super.getCurrentValue();
- }
-
- if (returnColumnarBatch) {
- return columnarBatch == null ? super.getCurrentValue() : columnarBatch;
- }
-
- return columnarBatch == null ? super.getCurrentValue() : columnarBatch.getRow(batchIdx - 1);
- }
-
- @Override
- public boolean nextKeyValue() throws IOException {
- resultBatch();
-
- if (returnColumnarBatch) {
- return nextBatch();
- }
-
- if (batchIdx >= numBatched) {
- if (!nextBatch()) {
- return false;
- }
- }
- ++batchIdx;
- return true;
- }
-}
-
diff --git a/hudi-spark-datasource/hudi-spark3.3.x/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister b/hudi-spark-datasource/hudi-spark3.3.x/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
index 33ab03f55477b..c8dd99a95c27a 100644
--- a/hudi-spark-datasource/hudi-spark3.3.x/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
+++ b/hudi-spark-datasource/hudi-spark3.3.x/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
@@ -16,4 +16,4 @@
# limitations under the License.
-org.apache.hudi.Spark3DefaultSource
\ No newline at end of file
+org.apache.hudi.Spark32PlusDefaultSource
\ No newline at end of file
diff --git a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/hudi/Spark33HoodieFileScanRDD.scala b/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/hudi/Spark33HoodieFileScanRDD.scala
index c387134ca2655..b2ed3bce23321 100644
--- a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/hudi/Spark33HoodieFileScanRDD.scala
+++ b/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/hudi/Spark33HoodieFileScanRDD.scala
@@ -26,8 +26,9 @@ import org.apache.spark.sql.types.StructType
class Spark33HoodieFileScanRDD(@transient private val sparkSession: SparkSession,
read: PartitionedFile => Iterator[InternalRow],
- @transient filePartitions: Seq[FilePartition],
- readDataSchema: StructType, metadataColumns: Seq[AttributeReference] = Seq.empty)
+ @transient filePartitions: Seq[FilePartition],
+ readDataSchema: StructType,
+ metadataColumns: Seq[AttributeReference] = Seq.empty)
extends FileScanRDD(sparkSession, read, filePartitions, readDataSchema, metadataColumns)
with HoodieUnsafeRDD {
diff --git a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/hudi/Spark3DefaultSource.scala b/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/hudi/Spark3DefaultSource.scala
deleted file mode 100644
index 3bc3446d1f120..0000000000000
--- a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/hudi/Spark3DefaultSource.scala
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * 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.hudi
-
-import org.apache.spark.sql.sources.DataSourceRegister
-
-/**
- * NOTE: PLEASE READ CAREFULLY
- * All of Spark DataSourceV2 APIs are deliberately disabled to make sure
- * there are no regressions in performance
- * Please check out HUDI-4178 for more details
- */
-class Spark3DefaultSource extends DefaultSource with DataSourceRegister /* with TableProvider */ {
-
- override def shortName(): String = "hudi"
-
- /*
- def inferSchema: StructType = new StructType()
-
- override def inferSchema(options: CaseInsensitiveStringMap): StructType = inferSchema
-
- override def getTable(schema: StructType,
- partitioning: Array[Transform],
- properties: java.util.Map[String, String]): Table = {
- val options = new CaseInsensitiveStringMap(properties)
- val path = options.get("path")
- if (path == null) throw new HoodieException("'path' cannot be null, missing 'path' from table properties")
-
- HoodieInternalV2Table(SparkSession.active, path)
- }
- */
-}
diff --git a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/HoodieSpark33CatalogUtils.scala b/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/HoodieSpark33CatalogUtils.scala
new file mode 100644
index 0000000000000..f7785568ff489
--- /dev/null
+++ b/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/HoodieSpark33CatalogUtils.scala
@@ -0,0 +1,30 @@
+/*
+ * 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
+
+import org.apache.spark.sql.connector.expressions.{BucketTransform, NamedReference, Transform}
+
+object HoodieSpark33CatalogUtils extends HoodieSpark3CatalogUtils {
+
+ override def unapplyBucketTransform(t: Transform): Option[(Int, Seq[NamedReference], Seq[NamedReference])] =
+ t match {
+ case BucketTransform(numBuckets, refs, sortedRefs) => Some(numBuckets, refs, sortedRefs)
+ case _ => None
+ }
+
+}
diff --git a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_3Adapter.scala b/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_3Adapter.scala
index 1cbd6be31f01e..72bc1d19f47fc 100644
--- a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_3Adapter.scala
+++ b/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_3Adapter.scala
@@ -25,16 +25,18 @@ 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.execution.datasources.parquet.{ParquetFileFormat, Spark32PlusHoodieParquetFileFormat}
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}
+import org.apache.spark.sql.{HoodieCatalystExpressionUtils, HoodieCatalystPlansUtils, HoodieSpark33CatalogUtils, HoodieSpark33CatalystExpressionUtils, HoodieSpark33CatalystPlanUtils, HoodieSpark3CatalogUtils, SparkSession}
/**
* Implementation of [[SparkAdapter]] for Spark 3.3.x branch
*/
class Spark3_3Adapter extends BaseSpark3Adapter {
+ override def getCatalogUtils: HoodieSpark3CatalogUtils = HoodieSpark33CatalogUtils
+
override def getCatalystExpressionUtils: HoodieCatalystExpressionUtils = HoodieSpark33CatalystExpressionUtils
override def getCatalystPlanUtils: HoodieCatalystPlansUtils = HoodieSpark33CatalystPlanUtils
@@ -52,7 +54,7 @@ class Spark3_3Adapter extends BaseSpark3Adapter {
}
override def createHoodieParquetFileFormat(appendPartitionValues: Boolean): Option[ParquetFileFormat] = {
- Some(new Spark33HoodieParquetFileFormat(appendPartitionValues))
+ Some(new Spark32PlusHoodieParquetFileFormat(appendPartitionValues))
}
override def createHoodieFileScanRDD(sparkSession: SparkSession,
diff --git a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/connector/catalog/HoodieIdentifier.scala b/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/connector/catalog/HoodieIdentifier.scala
deleted file mode 100644
index 2649c56e5a8a4..0000000000000
--- a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/connector/catalog/HoodieIdentifier.scala
+++ /dev/null
@@ -1,43 +0,0 @@
-/*
- * 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.catalog
-
-import java.util
-import java.util.Objects
-
-/**
- * This class is to make scala-2.11 compilable.
- * Using Identifier.of(namespace, name) to get a IdentifierImpl will throw
- * compile exception( Static methods in interface require -target:jvm-1.8)
- */
-case class HoodieIdentifier(namespace: Array[String], name: String) extends Identifier {
-
- override def equals(o: Any): Boolean = {
- o match {
- case that: HoodieIdentifier => util.Arrays.equals(namespace.asInstanceOf[Array[Object]],
- that.namespace.asInstanceOf[Array[Object]]) && name == that.name
- case _ => false
- }
- }
-
- override def hashCode: Int = {
- val nh = namespace.toSeq.hashCode().asInstanceOf[Object]
- Objects.hash(nh, name)
- }
-}
diff --git a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33HoodieParquetFileFormat.scala b/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33HoodieParquetFileFormat.scala
deleted file mode 100644
index bab8ff4928847..0000000000000
--- a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33HoodieParquetFileFormat.scala
+++ /dev/null
@@ -1,505 +0,0 @@
-/*
- * 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.execution.datasources.parquet
-
-import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.fs.Path
-import org.apache.hadoop.mapred.FileSplit
-import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl
-import org.apache.hadoop.mapreduce.{JobID, TaskAttemptID, TaskID, TaskType}
-import org.apache.hudi.HoodieSparkUtils
-import org.apache.hudi.client.utils.SparkInternalSchemaConverter
-import org.apache.hudi.common.fs.FSUtils
-import org.apache.hudi.common.util.InternalSchemaCache
-import org.apache.hudi.common.util.StringUtils.isNullOrEmpty
-import org.apache.hudi.common.util.collection.Pair
-import org.apache.hudi.internal.schema.InternalSchema
-import org.apache.hudi.internal.schema.action.InternalSchemaMerger
-import org.apache.hudi.internal.schema.utils.{InternalSchemaUtils, SerDeHelper}
-import org.apache.parquet.filter2.compat.FilterCompat
-import org.apache.parquet.filter2.predicate.FilterApi
-import org.apache.parquet.format.converter.ParquetMetadataConverter.SKIP_ROW_GROUPS
-import org.apache.parquet.hadoop.{ParquetInputFormat, ParquetRecordReader}
-import org.apache.spark.TaskContext
-import org.apache.spark.sql.SparkSession
-import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection
-import org.apache.spark.sql.catalyst.expressions.{Cast, JoinedRow}
-import org.apache.spark.sql.catalyst.util.DateTimeUtils
-import org.apache.spark.sql.execution.datasources.parquet.Spark33HoodieParquetFileFormat._
-import org.apache.spark.sql.execution.datasources.{DataSourceUtils, PartitionedFile, RecordReaderIterator}
-import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.sources._
-import org.apache.spark.sql.types.{AtomicType, DataType, StructField, StructType}
-import org.apache.spark.util.SerializableConfiguration
-
-import java.net.URI
-
-/**
- * This class is an extension of [[ParquetFileFormat]] overriding Spark-specific behavior
- * that's not possible to customize in any other way
- *
- * NOTE: This is a version of [[AvroDeserializer]] impl from Spark 3.2.1 w/ w/ the following changes applied to it:
- *
- *
Avoiding appending partition values to the rows read from the data file
- *
Schema on-read
- *
- */
-class Spark33HoodieParquetFileFormat(private val shouldAppendPartitionValues: Boolean) extends ParquetFileFormat {
-
- override def buildReaderWithPartitionValues(sparkSession: SparkSession,
- dataSchema: StructType,
- partitionSchema: StructType,
- requiredSchema: StructType,
- filters: Seq[Filter],
- options: Map[String, String],
- hadoopConf: Configuration): PartitionedFile => Iterator[InternalRow] = {
- hadoopConf.set(ParquetInputFormat.READ_SUPPORT_CLASS, classOf[ParquetReadSupport].getName)
- hadoopConf.set(
- ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA,
- requiredSchema.json)
- hadoopConf.set(
- ParquetWriteSupport.SPARK_ROW_SCHEMA,
- requiredSchema.json)
- hadoopConf.set(
- SQLConf.SESSION_LOCAL_TIMEZONE.key,
- sparkSession.sessionState.conf.sessionLocalTimeZone)
- hadoopConf.setBoolean(
- SQLConf.NESTED_SCHEMA_PRUNING_ENABLED.key,
- sparkSession.sessionState.conf.nestedSchemaPruningEnabled)
- hadoopConf.setBoolean(
- SQLConf.CASE_SENSITIVE.key,
- sparkSession.sessionState.conf.caseSensitiveAnalysis)
-
- ParquetWriteSupport.setSchema(requiredSchema, hadoopConf)
-
- // Sets flags for `ParquetToSparkSchemaConverter`
- hadoopConf.setBoolean(
- SQLConf.PARQUET_BINARY_AS_STRING.key,
- sparkSession.sessionState.conf.isParquetBinaryAsString)
- hadoopConf.setBoolean(
- SQLConf.PARQUET_INT96_AS_TIMESTAMP.key,
- sparkSession.sessionState.conf.isParquetINT96AsTimestamp)
-
- val internalSchemaStr = hadoopConf.get(SparkInternalSchemaConverter.HOODIE_QUERY_SCHEMA)
- // For Spark DataSource v1, there's no Physical Plan projection/schema pruning w/in Spark itself,
- // therefore it's safe to do schema projection here
- if (!isNullOrEmpty(internalSchemaStr)) {
- val prunedInternalSchemaStr =
- pruneInternalSchema(internalSchemaStr, requiredSchema)
- hadoopConf.set(SparkInternalSchemaConverter.HOODIE_QUERY_SCHEMA, prunedInternalSchemaStr)
- }
-
- val broadcastedHadoopConf =
- sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf))
-
- // TODO: if you move this into the closure it reverts to the default values.
- // If true, enable using the custom RecordReader for parquet. This only works for
- // a subset of the types (no complex types).
- val resultSchema = StructType(partitionSchema.fields ++ requiredSchema.fields)
- val sqlConf = sparkSession.sessionState.conf
- val enableOffHeapColumnVector = sqlConf.offHeapColumnVectorEnabled
- val enableVectorizedReader: Boolean =
- sqlConf.parquetVectorizedReaderEnabled &&
- resultSchema.forall(_.dataType.isInstanceOf[AtomicType])
- val enableRecordFilter: Boolean = sqlConf.parquetRecordFilterEnabled
- val timestampConversion: Boolean = sqlConf.isParquetINT96TimestampConversion
- val capacity = sqlConf.parquetVectorizedReaderBatchSize
- val enableParquetFilterPushDown: Boolean = sqlConf.parquetFilterPushDown
- // Whole stage codegen (PhysicalRDD) is able to deal with batches directly
- val returningBatch = supportBatch(sparkSession, resultSchema)
- val pushDownDate = sqlConf.parquetFilterPushDownDate
- val pushDownTimestamp = sqlConf.parquetFilterPushDownTimestamp
- val pushDownDecimal = sqlConf.parquetFilterPushDownDecimal
- val pushDownStringStartWith = sqlConf.parquetFilterPushDownStringStartWith
- val pushDownInFilterThreshold = sqlConf.parquetFilterPushDownInFilterThreshold
- val isCaseSensitive = sqlConf.caseSensitiveAnalysis
- val parquetOptions = new ParquetOptions(options, sparkSession.sessionState.conf)
- val datetimeRebaseModeInRead = parquetOptions.datetimeRebaseModeInRead
- val int96RebaseModeInRead = parquetOptions.int96RebaseModeInRead
-
- (file: PartitionedFile) => {
- assert(!shouldAppendPartitionValues || file.partitionValues.numFields == partitionSchema.size)
-
- val filePath = new Path(new URI(file.filePath))
- val split = new FileSplit(filePath, file.start, file.length, Array.empty[String])
-
- val sharedConf = broadcastedHadoopConf.value.value
-
- // Fetch internal schema
- val internalSchemaStr = sharedConf.get(SparkInternalSchemaConverter.HOODIE_QUERY_SCHEMA)
- // Internal schema has to be pruned at this point
- val querySchemaOption = SerDeHelper.fromJson(internalSchemaStr)
-
- val shouldUseInternalSchema = !isNullOrEmpty(internalSchemaStr) && querySchemaOption.isPresent
-
- val tablePath = sharedConf.get(SparkInternalSchemaConverter.HOODIE_TABLE_PATH)
- val fileSchema = if (shouldUseInternalSchema) {
- val commitInstantTime = FSUtils.getCommitTime(filePath.getName).toLong;
- val validCommits = sharedConf.get(SparkInternalSchemaConverter.HOODIE_VALID_COMMITS_LIST)
- InternalSchemaCache.getInternalSchemaByVersionId(commitInstantTime, tablePath, sharedConf, if (validCommits == null) "" else validCommits)
- } else {
- null
- }
-
- lazy val footerFileMetaData =
- ParquetFooterReader.readFooter(sharedConf, filePath, SKIP_ROW_GROUPS).getFileMetaData
- // Try to push down filters when filter push-down is enabled.
- val pushed = if (enableParquetFilterPushDown) {
- val parquetSchema = footerFileMetaData.getSchema
- val parquetFilters = if (HoodieSparkUtils.gteqSpark3_2_1) {
- // NOTE: Below code could only be compiled against >= Spark 3.2.1,
- // and unfortunately won't compile against Spark 3.2.0
- // However this code is runtime-compatible w/ both Spark 3.2.0 and >= Spark 3.2.1
- val datetimeRebaseSpec =
- DataSourceUtils.datetimeRebaseSpec(footerFileMetaData.getKeyValueMetaData.get, datetimeRebaseModeInRead)
- new ParquetFilters(
- parquetSchema,
- pushDownDate,
- pushDownTimestamp,
- pushDownDecimal,
- pushDownStringStartWith,
- pushDownInFilterThreshold,
- isCaseSensitive,
- datetimeRebaseSpec)
- } else {
- // Spark 3.2.0
- val datetimeRebaseMode =
- Spark33DataSourceUtils.datetimeRebaseMode(footerFileMetaData.getKeyValueMetaData.get, datetimeRebaseModeInRead)
- createParquetFilters(
- parquetSchema,
- pushDownDate,
- pushDownTimestamp,
- pushDownDecimal,
- pushDownStringStartWith,
- pushDownInFilterThreshold,
- isCaseSensitive,
- datetimeRebaseMode)
- }
- filters.map(rebuildFilterFromParquet(_, fileSchema, querySchemaOption.orElse(null)))
- // Collects all converted Parquet filter predicates. Notice that not all predicates can be
- // converted (`ParquetFilters.createFilter` returns an `Option`). That's why a `flatMap`
- // is used here.
- .flatMap(parquetFilters.createFilter)
- .reduceOption(FilterApi.and)
- } else {
- None
- }
-
- // PARQUET_INT96_TIMESTAMP_CONVERSION says to apply timezone conversions to int96 timestamps'
- // *only* if the file was created by something other than "parquet-mr", so check the actual
- // writer here for this file. We have to do this per-file, as each file in the table may
- // have different writers.
- // Define isCreatedByParquetMr as function to avoid unnecessary parquet footer reads.
- def isCreatedByParquetMr: Boolean =
- footerFileMetaData.getCreatedBy().startsWith("parquet-mr")
-
- val convertTz =
- if (timestampConversion && !isCreatedByParquetMr) {
- Some(DateTimeUtils.getZoneId(sharedConf.get(SQLConf.SESSION_LOCAL_TIMEZONE.key)))
- } else {
- None
- }
-
- val attemptId = new TaskAttemptID(new TaskID(new JobID(), TaskType.MAP, 0), 0)
-
- // Clone new conf
- val hadoopAttemptConf = new Configuration(broadcastedHadoopConf.value.value)
- val typeChangeInfos: java.util.Map[Integer, Pair[DataType, DataType]] = if (shouldUseInternalSchema) {
- val mergedInternalSchema = new InternalSchemaMerger(fileSchema, querySchemaOption.get(), true, true).mergeSchema()
- val mergedSchema = SparkInternalSchemaConverter.constructSparkSchemaFromInternalSchema(mergedInternalSchema)
-
- hadoopAttemptConf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA, mergedSchema.json)
-
- SparkInternalSchemaConverter.collectTypeChangedCols(querySchemaOption.get(), mergedInternalSchema)
- } else {
- new java.util.HashMap()
- }
-
- val hadoopAttemptContext =
- new TaskAttemptContextImpl(hadoopAttemptConf, attemptId)
-
- // Try to push down filters when filter push-down is enabled.
- // Notice: This push-down is RowGroups level, not individual records.
- if (pushed.isDefined) {
- ParquetInputFormat.setFilterPredicate(hadoopAttemptContext.getConfiguration, pushed.get)
- }
- val taskContext = Option(TaskContext.get())
- if (enableVectorizedReader) {
- val vectorizedReader =
- if (shouldUseInternalSchema) {
- val int96RebaseSpec =
- DataSourceUtils.int96RebaseSpec(footerFileMetaData.getKeyValueMetaData.get, int96RebaseModeInRead)
- val datetimeRebaseSpec =
- DataSourceUtils.datetimeRebaseSpec(footerFileMetaData.getKeyValueMetaData.get, datetimeRebaseModeInRead)
- new Spark33HoodieVectorizedParquetRecordReader(
- convertTz.orNull,
- datetimeRebaseSpec.mode.toString,
- datetimeRebaseSpec.timeZone,
- int96RebaseSpec.mode.toString,
- int96RebaseSpec.timeZone,
- enableOffHeapColumnVector && taskContext.isDefined,
- capacity,
- typeChangeInfos)
- } else if (HoodieSparkUtils.gteqSpark3_2_1) {
- // NOTE: Below code could only be compiled against >= Spark 3.2.1,
- // and unfortunately won't compile against Spark 3.2.0
- // However this code is runtime-compatible w/ both Spark 3.2.0 and >= Spark 3.2.1
- val int96RebaseSpec =
- DataSourceUtils.int96RebaseSpec(footerFileMetaData.getKeyValueMetaData.get, int96RebaseModeInRead)
- val datetimeRebaseSpec =
- DataSourceUtils.datetimeRebaseSpec(footerFileMetaData.getKeyValueMetaData.get, datetimeRebaseModeInRead)
- new VectorizedParquetRecordReader(
- convertTz.orNull,
- datetimeRebaseSpec.mode.toString,
- datetimeRebaseSpec.timeZone,
- int96RebaseSpec.mode.toString,
- int96RebaseSpec.timeZone,
- enableOffHeapColumnVector && taskContext.isDefined,
- capacity)
- } else {
- // Spark 3.2.0
- val datetimeRebaseMode =
- Spark33DataSourceUtils.datetimeRebaseMode(footerFileMetaData.getKeyValueMetaData.get, datetimeRebaseModeInRead)
- val int96RebaseMode =
- Spark33DataSourceUtils.int96RebaseMode(footerFileMetaData.getKeyValueMetaData.get, int96RebaseModeInRead)
- createVectorizedParquetRecordReader(
- convertTz.orNull,
- datetimeRebaseMode.toString,
- int96RebaseMode.toString,
- enableOffHeapColumnVector && taskContext.isDefined,
- capacity)
- }
-
- // SPARK-37089: We cannot register a task completion listener to close this iterator here
- // because downstream exec nodes have already registered their listeners. Since listeners
- // are executed in reverse order of registration, a listener registered here would close the
- // iterator while downstream exec nodes are still running. When off-heap column vectors are
- // enabled, this can cause a use-after-free bug leading to a segfault.
- //
- // Instead, we use FileScanRDD's task completion listener to close this iterator.
- val iter = new RecordReaderIterator(vectorizedReader)
- try {
- vectorizedReader.initialize(split, hadoopAttemptContext)
-
- // NOTE: We're making appending of the partitioned values to the rows read from the
- // data file configurable
- if (shouldAppendPartitionValues) {
- logDebug(s"Appending $partitionSchema ${file.partitionValues}")
- vectorizedReader.initBatch(partitionSchema, file.partitionValues)
- } else {
- vectorizedReader.initBatch(StructType(Nil), InternalRow.empty)
- }
-
- if (returningBatch) {
- vectorizedReader.enableReturningBatches()
- }
-
- // UnsafeRowParquetRecordReader appends the columns internally to avoid another copy.
- iter.asInstanceOf[Iterator[InternalRow]]
- } catch {
- case e: Throwable =>
- // SPARK-23457: In case there is an exception in initialization, close the iterator to
- // avoid leaking resources.
- iter.close()
- throw e
- }
- } else {
- logDebug(s"Falling back to parquet-mr")
- val readSupport = if (HoodieSparkUtils.gteqSpark3_2_1) {
- // ParquetRecordReader returns InternalRow
- // NOTE: Below code could only be compiled against >= Spark 3.2.1,
- // and unfortunately won't compile against Spark 3.2.0
- // However this code is runtime-compatible w/ both Spark 3.2.0 and >= Spark 3.2.1
- val int96RebaseSpec =
- DataSourceUtils.int96RebaseSpec(footerFileMetaData.getKeyValueMetaData.get, int96RebaseModeInRead)
- val datetimeRebaseSpec =
- DataSourceUtils.datetimeRebaseSpec(footerFileMetaData.getKeyValueMetaData.get, datetimeRebaseModeInRead)
- new ParquetReadSupport(
- convertTz,
- enableVectorizedReader = false,
- datetimeRebaseSpec,
- int96RebaseSpec)
- } else {
- val datetimeRebaseMode =
- Spark33DataSourceUtils.datetimeRebaseMode(footerFileMetaData.getKeyValueMetaData.get, datetimeRebaseModeInRead)
- val int96RebaseMode =
- Spark33DataSourceUtils.int96RebaseMode(footerFileMetaData.getKeyValueMetaData.get, int96RebaseModeInRead)
- createParquetReadSupport(
- convertTz,
- /* enableVectorizedReader = */ false,
- datetimeRebaseMode,
- int96RebaseMode)
- }
-
- val reader = if (pushed.isDefined && enableRecordFilter) {
- val parquetFilter = FilterCompat.get(pushed.get, null)
- new ParquetRecordReader[InternalRow](readSupport, parquetFilter)
- } else {
- new ParquetRecordReader[InternalRow](readSupport)
- }
- val iter = new RecordReaderIterator[InternalRow](reader)
- try {
- reader.initialize(split, hadoopAttemptContext)
-
- val fullSchema = requiredSchema.toAttributes ++ partitionSchema.toAttributes
- val unsafeProjection = if (typeChangeInfos.isEmpty) {
- GenerateUnsafeProjection.generate(fullSchema, fullSchema)
- } else {
- // find type changed.
- val newFullSchema = new StructType(requiredSchema.fields.zipWithIndex.map { case (f, i) =>
- if (typeChangeInfos.containsKey(i)) {
- StructField(f.name, typeChangeInfos.get(i).getRight, f.nullable, f.metadata)
- } else f
- }).toAttributes ++ partitionSchema.toAttributes
- val castSchema = newFullSchema.zipWithIndex.map { case (attr, i) =>
- if (typeChangeInfos.containsKey(i)) {
- Cast(attr, typeChangeInfos.get(i).getLeft)
- } else attr
- }
- GenerateUnsafeProjection.generate(castSchema, newFullSchema)
- }
-
- // NOTE: We're making appending of the partitioned values to the rows read from the
- // data file configurable
- if (!shouldAppendPartitionValues || partitionSchema.length == 0) {
- // There is no partition columns
- iter.map(unsafeProjection)
- } else {
- val joinedRow = new JoinedRow()
- iter.map(d => unsafeProjection(joinedRow(d, file.partitionValues)))
- }
- } catch {
- case e: Throwable =>
- // SPARK-23457: In case there is an exception in initialization, close the iterator to
- // avoid leaking resources.
- iter.close()
- throw e
- }
- }
- }
- }
-
-}
-
-object Spark33HoodieParquetFileFormat {
-
- /**
- * NOTE: This method is specific to Spark 3.2.0
- */
- private def createParquetFilters(args: Any*): ParquetFilters = {
- // NOTE: ParquetFilters ctor args contain Scala enum, therefore we can't look it
- // up by arg types, and have to instead rely on the number of args based on individual class;
- // the ctor order is not guaranteed
- val ctor = classOf[ParquetFilters].getConstructors.maxBy(_.getParameterCount)
- ctor.newInstance(args.map(_.asInstanceOf[AnyRef]): _*)
- .asInstanceOf[ParquetFilters]
- }
-
- /**
- * NOTE: This method is specific to Spark 3.2.0
- */
- private def createParquetReadSupport(args: Any*): ParquetReadSupport = {
- // NOTE: ParquetReadSupport ctor args contain Scala enum, therefore we can't look it
- // up by arg types, and have to instead rely on the number of args based on individual class;
- // the ctor order is not guaranteed
- val ctor = classOf[ParquetReadSupport].getConstructors.maxBy(_.getParameterCount)
- ctor.newInstance(args.map(_.asInstanceOf[AnyRef]): _*)
- .asInstanceOf[ParquetReadSupport]
- }
-
- /**
- * NOTE: This method is specific to Spark 3.2.0
- */
- private def createVectorizedParquetRecordReader(args: Any*): VectorizedParquetRecordReader = {
- // NOTE: ParquetReadSupport ctor args contain Scala enum, therefore we can't look it
- // up by arg types, and have to instead rely on the number of args based on individual class;
- // the ctor order is not guaranteed
- val ctor = classOf[VectorizedParquetRecordReader].getConstructors.maxBy(_.getParameterCount)
- ctor.newInstance(args.map(_.asInstanceOf[AnyRef]): _*)
- .asInstanceOf[VectorizedParquetRecordReader]
- }
-
- def pruneInternalSchema(internalSchemaStr: String, requiredSchema: StructType): String = {
- val querySchemaOption = SerDeHelper.fromJson(internalSchemaStr)
- if (querySchemaOption.isPresent && requiredSchema.nonEmpty) {
- val prunedSchema = SparkInternalSchemaConverter.convertAndPruneStructTypeToInternalSchema(requiredSchema, querySchemaOption.get())
- SerDeHelper.toJson(prunedSchema)
- } else {
- internalSchemaStr
- }
- }
-
- private def rebuildFilterFromParquet(oldFilter: Filter, fileSchema: InternalSchema, querySchema: InternalSchema): Filter = {
- if (fileSchema == null || querySchema == null) {
- oldFilter
- } else {
- oldFilter match {
- case eq: EqualTo =>
- val newAttribute = InternalSchemaUtils.reBuildFilterName(eq.attribute, fileSchema, querySchema)
- if (newAttribute.isEmpty) AlwaysTrue else eq.copy(attribute = newAttribute)
- case eqs: EqualNullSafe =>
- val newAttribute = InternalSchemaUtils.reBuildFilterName(eqs.attribute, fileSchema, querySchema)
- if (newAttribute.isEmpty) AlwaysTrue else eqs.copy(attribute = newAttribute)
- case gt: GreaterThan =>
- val newAttribute = InternalSchemaUtils.reBuildFilterName(gt.attribute, fileSchema, querySchema)
- if (newAttribute.isEmpty) AlwaysTrue else gt.copy(attribute = newAttribute)
- case gtr: GreaterThanOrEqual =>
- val newAttribute = InternalSchemaUtils.reBuildFilterName(gtr.attribute, fileSchema, querySchema)
- if (newAttribute.isEmpty) AlwaysTrue else gtr.copy(attribute = newAttribute)
- case lt: LessThan =>
- val newAttribute = InternalSchemaUtils.reBuildFilterName(lt.attribute, fileSchema, querySchema)
- if (newAttribute.isEmpty) AlwaysTrue else lt.copy(attribute = newAttribute)
- case lte: LessThanOrEqual =>
- val newAttribute = InternalSchemaUtils.reBuildFilterName(lte.attribute, fileSchema, querySchema)
- if (newAttribute.isEmpty) AlwaysTrue else lte.copy(attribute = newAttribute)
- case i: In =>
- val newAttribute = InternalSchemaUtils.reBuildFilterName(i.attribute, fileSchema, querySchema)
- if (newAttribute.isEmpty) AlwaysTrue else i.copy(attribute = newAttribute)
- case isn: IsNull =>
- val newAttribute = InternalSchemaUtils.reBuildFilterName(isn.attribute, fileSchema, querySchema)
- if (newAttribute.isEmpty) AlwaysTrue else isn.copy(attribute = newAttribute)
- case isnn: IsNotNull =>
- val newAttribute = InternalSchemaUtils.reBuildFilterName(isnn.attribute, fileSchema, querySchema)
- if (newAttribute.isEmpty) AlwaysTrue else isnn.copy(attribute = newAttribute)
- case And(left, right) =>
- And(rebuildFilterFromParquet(left, fileSchema, querySchema), rebuildFilterFromParquet(right, fileSchema, querySchema))
- case Or(left, right) =>
- Or(rebuildFilterFromParquet(left, fileSchema, querySchema), rebuildFilterFromParquet(right, fileSchema, querySchema))
- case Not(child) =>
- Not(rebuildFilterFromParquet(child, fileSchema, querySchema))
- case ssw: StringStartsWith =>
- val newAttribute = InternalSchemaUtils.reBuildFilterName(ssw.attribute, fileSchema, querySchema)
- if (newAttribute.isEmpty) AlwaysTrue else ssw.copy(attribute = newAttribute)
- case ses: StringEndsWith =>
- val newAttribute = InternalSchemaUtils.reBuildFilterName(ses.attribute, fileSchema, querySchema)
- if (newAttribute.isEmpty) AlwaysTrue else ses.copy(attribute = newAttribute)
- case sc: StringContains =>
- val newAttribute = InternalSchemaUtils.reBuildFilterName(sc.attribute, fileSchema, querySchema)
- if (newAttribute.isEmpty) AlwaysTrue else sc.copy(attribute = newAttribute)
- case AlwaysTrue =>
- AlwaysTrue
- case AlwaysFalse =>
- AlwaysFalse
- case _ =>
- AlwaysTrue
- }
- }
- }
-}
-
diff --git a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/hudi/Spark33ResolveHudiAlterTableCommand.scala b/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/hudi/Spark33ResolveHudiAlterTableCommand.scala
index 06371afcfa229..84cf7c36d5e1f 100644
--- a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/hudi/Spark33ResolveHudiAlterTableCommand.scala
+++ b/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/hudi/Spark33ResolveHudiAlterTableCommand.scala
@@ -18,7 +18,6 @@
package org.apache.spark.sql.hudi
import org.apache.hudi.common.config.HoodieCommonConfig
-import org.apache.hudi.config.HoodieWriteConfig
import org.apache.hudi.internal.schema.action.TableChange.ColumnChangeID
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.catalyst.analysis.ResolvedTable
diff --git a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieSpark3Analysis.scala b/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieSpark3Analysis.scala
deleted file mode 100644
index 140b65b0aa6d0..0000000000000
--- a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieSpark3Analysis.scala
+++ /dev/null
@@ -1,222 +0,0 @@
-/*
- * 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.{DefaultSource, SparkAdapterSupport}
-import org.apache.hudi.common.table.HoodieTableMetaClient
-import org.apache.spark.sql.catalyst.TableIdentifier
-import org.apache.spark.sql.catalyst.analysis.{ResolvedTable, UnresolvedPartitionSpec}
-import org.apache.spark.sql.catalyst.catalog.{CatalogTable, HoodieCatalogTable}
-import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute}
-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.LogicalRelation
-import org.apache.spark.sql.execution.datasources.PreWriteCheck.failAnalysis
-import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, V2SessionCatalog}
-import org.apache.spark.sql.hudi.HoodieSqlCommonUtils.{castIfNeeded, getTableLocation, removeMetaFields, tableExistsInPath}
-import org.apache.spark.sql.hudi.catalog.{HoodieCatalog, HoodieInternalV2Table}
-import org.apache.spark.sql.hudi.command.{AlterHoodieTableDropPartitionCommand, ShowHoodieTablePartitionsCommand, TruncateHoodieTableCommand}
-import org.apache.spark.sql.hudi.{HoodieSqlCommonUtils, ProvidesHoodieConfig}
-import org.apache.spark.sql.types.StructType
-import org.apache.spark.sql.{AnalysisException, SQLContext, SparkSession}
-
-import scala.collection.JavaConverters.mapAsJavaMapConverter
-
-/**
- * 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
- */
-class HoodieDataSourceV2ToV1Fallback(sparkSession: SparkSession) extends Rule[LogicalPlan]
- with ProvidesHoodieConfig {
-
- override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsDown {
- case v2r @ DataSourceV2Relation(v2Table: HoodieInternalV2Table, _, _, _, _) =>
- val output = v2r.output
- val catalogTable = v2Table.catalogTable.map(_ => v2Table.v1Table)
- val relation = new DefaultSource().createRelation(new SQLContext(sparkSession),
- buildHoodieConfig(v2Table.hoodieCatalogTable), v2Table.hoodieCatalogTable.tableSchema)
-
- LogicalRelation(relation, output, catalogTable, isStreaming = false)
- }
-}
-
-class HoodieSpark3Analysis(sparkSession: SparkSession) extends Rule[LogicalPlan] {
- override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsDown {
- case s @ InsertIntoStatement(r @ DataSourceV2Relation(v2Table: HoodieInternalV2Table, _, _, _, _), partitionSpec, _, _, _, _)
- if s.query.resolved && needsSchemaAdjustment(s.query, v2Table.hoodieCatalogTable.table, partitionSpec, r.schema) =>
- val projection = resolveQueryColumnsByOrdinal(s.query, r.output)
- if (projection != s.query) {
- s.copy(query = projection)
- } else {
- s
- }
- }
-
- /**
- * Need to adjust schema based on the query and relation schema, for example,
- * if using insert into xx select 1, 2 here need to map to column names
- */
- private def needsSchemaAdjustment(query: LogicalPlan,
- table: CatalogTable,
- partitionSpec: Map[String, Option[String]],
- schema: StructType): Boolean = {
- val output = query.output
- val queryOutputWithoutMetaFields = removeMetaFields(output)
- val hoodieCatalogTable = HoodieCatalogTable(sparkSession, table)
-
- val partitionFields = hoodieCatalogTable.partitionFields
- val partitionSchema = hoodieCatalogTable.partitionSchema
- val staticPartitionValues = partitionSpec.filter(p => p._2.isDefined).mapValues(_.get)
-
- assert(staticPartitionValues.isEmpty ||
- staticPartitionValues.size == partitionSchema.size,
- s"Required partition columns is: ${partitionSchema.json}, Current static partitions " +
- s"is: ${staticPartitionValues.mkString("," + "")}")
-
- assert(staticPartitionValues.size + queryOutputWithoutMetaFields.size
- == hoodieCatalogTable.tableSchemaWithoutMetaFields.size,
- s"Required select columns count: ${hoodieCatalogTable.tableSchemaWithoutMetaFields.size}, " +
- s"Current select columns(including static partition column) count: " +
- s"${staticPartitionValues.size + queryOutputWithoutMetaFields.size},columns: " +
- s"(${(queryOutputWithoutMetaFields.map(_.name) ++ staticPartitionValues.keys).mkString(",")})")
-
- // static partition insert.
- if (staticPartitionValues.nonEmpty) {
- // drop partition fields in origin schema to align fields.
- schema.dropWhile(p => partitionFields.contains(p.name))
- }
-
- val existingSchemaOutput = output.take(schema.length)
- existingSchemaOutput.map(_.name) != schema.map(_.name) ||
- existingSchemaOutput.map(_.dataType) != schema.map(_.dataType)
- }
-
- private def resolveQueryColumnsByOrdinal(query: LogicalPlan,
- targetAttrs: Seq[Attribute]): LogicalPlan = {
- // always add a Cast. it will be removed in the optimizer if it is unnecessary.
- val project = query.output.zipWithIndex.map { case (attr, i) =>
- if (i < targetAttrs.length) {
- val targetAttr = targetAttrs(i)
- val castAttr = castIfNeeded(attr.withNullability(targetAttr.nullable), targetAttr.dataType, conf)
- Alias(castAttr, targetAttr.name)()
- } else {
- attr
- }
- }
- Project(project, query)
- }
-}
-
-/**
- * Rule for resolve hoodie's extended syntax or rewrite some logical plan.
- */
-case class HoodieSpark3ResolveReferences(sparkSession: SparkSession) extends Rule[LogicalPlan]
- with SparkAdapterSupport with ProvidesHoodieConfig {
-
- def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperatorsUp {
- // Fill schema for Create Table without specify schema info
- // CreateTable / CreateTableAsSelect was migrated to v2 in Spark 3.3.0
- // https://issues.apache.org/jira/browse/SPARK-36850
- case c @ CreateTable(tableCatalog, schema, partitioning, tableSpec, _)
- if sparkAdapter.isHoodieTable(tableSpec.properties.asJava) =>
-
- if (schema.isEmpty && partitioning.nonEmpty) {
- failAnalysis("It is not allowed to specify partition columns when the table schema is " +
- "not defined. When the table schema is not provided, schema and partition columns " +
- "will be inferred.")
- }
- val hoodieCatalog = tableCatalog match {
- case catalog: HoodieCatalog => catalog
- case _ => tableCatalog.asInstanceOf[V2SessionCatalog]
- }
-
- val tablePath = getTableLocation(tableSpec.properties,
- TableIdentifier(c.tableName.name(), c.tableName.namespace().lastOption)
- , sparkSession)
-
- val tableExistInCatalog = hoodieCatalog.tableExists(c.tableName)
- // Only when the table has not exist in catalog, we need to fill the schema info for creating table.
- if (!tableExistInCatalog && tableExistsInPath(tablePath, sparkSession.sessionState.newHadoopConf())) {
- val metaClient = HoodieTableMetaClient.builder()
- .setBasePath(tablePath)
- .setConf(sparkSession.sessionState.newHadoopConf())
- .build()
- val tableSchema = HoodieSqlCommonUtils.getTableSqlSchema(metaClient)
- if (tableSchema.isDefined && schema.isEmpty) {
- // Fill the schema with the schema from the table
- c.copy(tableSchema = tableSchema.get)
- } else if (tableSchema.isDefined && schema != tableSchema.get) {
- throw new AnalysisException(s"Specified schema in create table statement is not equal to the table schema." +
- s"You should not specify the schema for an existing table: ${c.tableName.name()} ")
- } else {
- c
- }
- } else {
- c
- }
- case p => p
- }
-}
-
-/**
- * Rule replacing resolved Spark's commands (not working for Hudi tables out-of-the-box) with
- * corresponding Hudi implementations
- */
-case class HoodieSpark3PostAnalysisRule(sparkSession: SparkSession) extends Rule[LogicalPlan] {
- override def apply(plan: LogicalPlan): LogicalPlan = {
- plan match {
- case ShowPartitions(ResolvedTable(_, id, HoodieV1OrV2Table(_), _), specOpt, _) =>
- ShowHoodieTablePartitionsCommand(
- id.asTableIdentifier, specOpt.map(s => s.asInstanceOf[UnresolvedPartitionSpec].spec))
-
- // Rewrite TruncateTableCommand to TruncateHoodieTableCommand
- case TruncateTable(ResolvedTable(_, id, HoodieV1OrV2Table(_), _)) =>
- TruncateHoodieTableCommand(id.asTableIdentifier, None)
-
- case TruncatePartition(ResolvedTable(_, id, HoodieV1OrV2Table(_), _), partitionSpec: UnresolvedPartitionSpec) =>
- TruncateHoodieTableCommand(id.asTableIdentifier, Some(partitionSpec.spec))
-
- case DropPartitions(ResolvedTable(_, id, HoodieV1OrV2Table(_), _), specs, ifExists, purge) =>
- AlterHoodieTableDropPartitionCommand(
- id.asTableIdentifier,
- specs.seq.map(f => f.asInstanceOf[UnresolvedPartitionSpec]).map(s => s.spec),
- ifExists,
- purge,
- retainData = true
- )
-
- case _ => plan
- }
- }
-}
-
-private[sql] object HoodieV1OrV2Table extends SparkAdapterSupport {
- def unapply(table: Table): Option[CatalogTable] = table match {
- case V1Table(catalogTable) if sparkAdapter.isHoodieTable(catalogTable) => Some(catalogTable)
- case v2: HoodieInternalV2Table => v2.catalogTable
- case _ => None
- }
-}
-
diff --git a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/hudi/catalog/HoodieCatalog.scala b/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/hudi/catalog/HoodieCatalog.scala
deleted file mode 100644
index b562c2f0a207f..0000000000000
--- a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/hudi/catalog/HoodieCatalog.scala
+++ /dev/null
@@ -1,367 +0,0 @@
-/*
- * 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.catalog
-
-import org.apache.hadoop.fs.Path
-import org.apache.hudi.exception.HoodieException
-import org.apache.hudi.sql.InsertMode
-import org.apache.hudi.sync.common.util.ConfigUtils
-import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions, SparkAdapterSupport}
-import org.apache.spark.sql.catalyst.TableIdentifier
-import org.apache.spark.sql.catalyst.analysis.{NoSuchTableException, TableAlreadyExistsException, UnresolvedAttribute}
-import org.apache.spark.sql.catalyst.catalog.HoodieCatalogTable.needFilterProps
-import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogTable, CatalogTableType, CatalogUtils, HoodieCatalogTable}
-import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.IdentifierHelper
-import org.apache.spark.sql.connector.catalog.TableChange.{AddColumn, ColumnChange, UpdateColumnComment, UpdateColumnType}
-import org.apache.spark.sql.connector.catalog._
-import org.apache.spark.sql.connector.expressions.{BucketTransform, FieldReference, IdentityTransform, Transform}
-import org.apache.spark.sql.execution.datasources.DataSource
-import org.apache.spark.sql.hudi.analysis.HoodieV1OrV2Table
-import org.apache.spark.sql.hudi.catalog.HoodieCatalog.convertTransforms
-import org.apache.spark.sql.hudi.command._
-import org.apache.spark.sql.hudi.{HoodieSqlCommonUtils, ProvidesHoodieConfig}
-import org.apache.spark.sql.types.{StructField, StructType}
-import org.apache.spark.sql.{Dataset, SaveMode, SparkSession, _}
-
-import java.net.URI
-import java.util
-import scala.collection.JavaConverters.{mapAsJavaMapConverter, mapAsScalaMapConverter}
-import scala.collection.mutable
-
-class HoodieCatalog extends DelegatingCatalogExtension
- with StagingTableCatalog
- with SparkAdapterSupport
- with ProvidesHoodieConfig {
-
- val spark: SparkSession = SparkSession.active
-
- override def stageCreate(ident: Identifier, schema: StructType, partitions: Array[Transform], properties: util.Map[String, String]): StagedTable = {
- if (sparkAdapter.isHoodieTable(properties)) {
- val locUriAndTableType = deduceTableLocationURIAndTableType(ident, properties)
- HoodieStagedTable(ident, locUriAndTableType, this, schema, partitions,
- properties, TableCreationMode.STAGE_CREATE)
- } else {
- BasicStagedTable(
- ident,
- super.createTable(ident, schema, partitions, properties),
- this)
- }
- }
-
- override def stageReplace(ident: Identifier, schema: StructType, partitions: Array[Transform], properties: util.Map[String, String]): StagedTable = {
- if (sparkAdapter.isHoodieTable(properties)) {
- val locUriAndTableType = deduceTableLocationURIAndTableType(ident, properties)
- HoodieStagedTable(ident, locUriAndTableType, this, schema, partitions,
- properties, TableCreationMode.STAGE_REPLACE)
- } else {
- super.dropTable(ident)
- BasicStagedTable(
- ident,
- super.createTable(ident, schema, partitions, properties),
- this)
- }
- }
-
- override def stageCreateOrReplace(ident: Identifier,
- schema: StructType,
- partitions: Array[Transform],
- properties: util.Map[String, String]): StagedTable = {
- if (sparkAdapter.isHoodieTable(properties)) {
- val locUriAndTableType = deduceTableLocationURIAndTableType(ident, properties)
- HoodieStagedTable(ident, locUriAndTableType, this, schema, partitions,
- properties, TableCreationMode.CREATE_OR_REPLACE)
- } else {
- try super.dropTable(ident) catch {
- case _: NoSuchTableException => // ignore the exception
- }
- BasicStagedTable(
- ident,
- super.createTable(ident, schema, partitions, properties),
- this)
- }
- }
-
- override def loadTable(ident: Identifier): Table = {
- super.loadTable(ident) match {
- case V1Table(catalogTable0) if sparkAdapter.isHoodieTable(catalogTable0) =>
- val catalogTable = catalogTable0.comment match {
- case Some(v) =>
- val newProps = catalogTable0.properties + (TableCatalog.PROP_COMMENT -> v)
- catalogTable0.copy(properties = newProps)
- case _ =>
- catalogTable0
- }
-
- val v2Table = HoodieInternalV2Table(
- spark = spark,
- path = catalogTable.location.toString,
- catalogTable = Some(catalogTable),
- tableIdentifier = Some(ident.toString))
-
- val schemaEvolutionEnabled: Boolean = spark.sessionState.conf.getConfString(DataSourceReadOptions.SCHEMA_EVOLUTION_ENABLED.key,
- DataSourceReadOptions.SCHEMA_EVOLUTION_ENABLED.defaultValue.toString).toBoolean
-
- // NOTE: PLEASE READ CAREFULLY
- //
- // Since Hudi relations don't currently implement DS V2 Read API, we by default 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
- if (schemaEvolutionEnabled) {
- v2Table
- } else {
- v2Table.v1TableWrapper
- }
-
- case t => t
- }
- }
-
- override def createTable(ident: Identifier,
- schema: StructType,
- partitions: Array[Transform],
- properties: util.Map[String, String]): Table = {
- if (sparkAdapter.isHoodieTable(properties)) {
- val locUriAndTableType = deduceTableLocationURIAndTableType(ident, properties)
- createHoodieTable(ident, schema, locUriAndTableType, partitions, properties,
- Map.empty, Option.empty, TableCreationMode.CREATE)
- } else {
- super.createTable(ident, schema, partitions, properties)
- }
- }
-
- override def tableExists(ident: Identifier): Boolean = super.tableExists(ident)
-
- override def dropTable(ident: Identifier): Boolean = {
- val table = loadTable(ident)
- table match {
- case HoodieV1OrV2Table(_) =>
- DropHoodieTableCommand(ident.asTableIdentifier, ifExists = true, isView = false, purge = false).run(spark)
- true
- case _ => super.dropTable(ident)
- }
- }
-
- override def purgeTable(ident: Identifier): Boolean = {
- val table = loadTable(ident)
- table match {
- case HoodieV1OrV2Table(_) =>
- DropHoodieTableCommand(ident.asTableIdentifier, ifExists = true, isView = false, purge = true).run(spark)
- true
- case _ => super.purgeTable(ident)
- }
- }
-
- @throws[NoSuchTableException]
- @throws[TableAlreadyExistsException]
- override def renameTable(oldIdent: Identifier, newIdent: Identifier): Unit = {
- loadTable(oldIdent) match {
- case HoodieV1OrV2Table(_) =>
- AlterHoodieTableRenameCommand(oldIdent.asTableIdentifier, newIdent.asTableIdentifier, false).run(spark)
- case _ => super.renameTable(oldIdent, newIdent)
- }
- }
-
- override def alterTable(ident: Identifier, changes: TableChange*): Table = {
- loadTable(ident) match {
- case HoodieV1OrV2Table(table) => {
- val tableIdent = TableIdentifier(ident.name(), ident.namespace().lastOption)
- changes.groupBy(c => c.getClass).foreach {
- case (t, newColumns) if t == classOf[AddColumn] =>
- AlterHoodieTableAddColumnsCommand(
- tableIdent,
- newColumns.asInstanceOf[Seq[AddColumn]].map { col =>
- StructField(
- col.fieldNames()(0),
- col.dataType(),
- col.isNullable)
- }).run(spark)
-
- case (t, columnChanges) if classOf[ColumnChange].isAssignableFrom(t) =>
- columnChanges.foreach {
- case dataType: UpdateColumnType =>
- val colName = UnresolvedAttribute(dataType.fieldNames()).name
- val newDataType = dataType.newDataType()
- val structField = StructField(colName, newDataType)
- AlterHoodieTableChangeColumnCommand(tableIdent, colName, structField).run(spark)
- case dataType: UpdateColumnComment =>
- val newComment = dataType.newComment()
- val colName = UnresolvedAttribute(dataType.fieldNames()).name
- val fieldOpt = table.schema.findNestedField(dataType.fieldNames(), includeCollections = true,
- spark.sessionState.conf.resolver).map(_._2)
- val field = fieldOpt.getOrElse {
- throw new AnalysisException(
- s"Couldn't find column $colName in:\n${table.schema.treeString}")
- }
- AlterHoodieTableChangeColumnCommand(tableIdent, colName, field.withComment(newComment)).run(spark)
- }
- case (t, _) =>
- throw new UnsupportedOperationException(s"not supported table change: ${t.getClass}")
- }
-
- loadTable(ident)
- }
- case _ => super.alterTable(ident, changes: _*)
- }
- }
-
- private def deduceTableLocationURIAndTableType(
- ident: Identifier, properties: util.Map[String, String]): (URI, CatalogTableType) = {
- val locOpt = if (isPathIdentifier(ident)) {
- Option(ident.name())
- } else {
- Option(properties.get("location"))
- }
- val tableType = if (locOpt.nonEmpty) {
- CatalogTableType.EXTERNAL
- } else {
- CatalogTableType.MANAGED
- }
- val locUriOpt = locOpt.map(CatalogUtils.stringToURI)
- val tableIdent = ident.asTableIdentifier
- val existingTableOpt = getExistingTableIfExists(tableIdent)
- val locURI = locUriOpt
- .orElse(existingTableOpt.flatMap(_.storage.locationUri))
- .getOrElse(spark.sessionState.catalog.defaultTablePath(tableIdent))
- (locURI, tableType)
- }
-
- def createHoodieTable(ident: Identifier,
- schema: StructType,
- locUriAndTableType: (URI, CatalogTableType),
- partitions: Array[Transform],
- allTableProperties: util.Map[String, String],
- writeOptions: Map[String, String],
- sourceQuery: Option[DataFrame],
- operation: TableCreationMode): Table = {
-
- val (partitionColumns, maybeBucketSpec) = convertTransforms(partitions)
- val newSchema = schema
- val newPartitionColumns = partitionColumns
- val newBucketSpec = maybeBucketSpec
-
- val storage = DataSource.buildStorageFormatFromOptions(writeOptions.--(needFilterProps))
- .copy(locationUri = Option(locUriAndTableType._1))
- val commentOpt = Option(allTableProperties.get("comment"))
-
- val tablePropertiesNew = new util.HashMap[String, String](allTableProperties)
- // put path to table properties.
- tablePropertiesNew.put("path", locUriAndTableType._1.getPath)
-
- val tableDesc = new CatalogTable(
- identifier = ident.asTableIdentifier,
- tableType = locUriAndTableType._2,
- storage = storage,
- schema = newSchema,
- provider = Option("hudi"),
- partitionColumnNames = newPartitionColumns,
- bucketSpec = newBucketSpec,
- properties = tablePropertiesNew.asScala.toMap.--(needFilterProps),
- comment = commentOpt)
-
- val hoodieCatalogTable = HoodieCatalogTable(spark, tableDesc)
-
- if (operation == TableCreationMode.STAGE_CREATE) {
- val tablePath = hoodieCatalogTable.tableLocation
- val hadoopConf = spark.sessionState.newHadoopConf()
- assert(HoodieSqlCommonUtils.isEmptyPath(tablePath, hadoopConf),
- s"Path '$tablePath' should be empty for CTAS")
- hoodieCatalogTable.initHoodieTable()
-
- val tblProperties = hoodieCatalogTable.catalogProperties
- val options = Map(
- DataSourceWriteOptions.HIVE_CREATE_MANAGED_TABLE.key -> (tableDesc.tableType == CatalogTableType.MANAGED).toString,
- DataSourceWriteOptions.HIVE_TABLE_SERDE_PROPERTIES.key -> ConfigUtils.configToString(tblProperties.asJava),
- DataSourceWriteOptions.HIVE_TABLE_PROPERTIES.key -> ConfigUtils.configToString(tableDesc.properties.asJava),
- DataSourceWriteOptions.SQL_INSERT_MODE.key -> InsertMode.NON_STRICT.value(),
- DataSourceWriteOptions.SQL_ENABLE_BULK_INSERT.key -> "true"
- )
- saveSourceDF(sourceQuery, tableDesc.properties ++ buildHoodieInsertConfig(hoodieCatalogTable, spark, isOverwrite = false, Map.empty, options))
- CreateHoodieTableCommand.createTableInCatalog(spark, hoodieCatalogTable, ignoreIfExists = false)
- } else if (sourceQuery.isEmpty) {
- saveSourceDF(sourceQuery, tableDesc.properties)
- new CreateHoodieTableCommand(tableDesc, false).run(spark)
- } else {
- saveSourceDF(sourceQuery, tableDesc.properties ++ buildHoodieInsertConfig(hoodieCatalogTable, spark, isOverwrite = false, Map.empty, Map.empty))
- new CreateHoodieTableCommand(tableDesc, false).run(spark)
- }
-
- loadTable(ident)
- }
-
- private def isPathIdentifier(ident: Identifier) = new Path(ident.name()).isAbsolute
-
- protected def isPathIdentifier(table: CatalogTable): Boolean = {
- isPathIdentifier(table.identifier)
- }
-
- protected def isPathIdentifier(tableIdentifier: TableIdentifier): Boolean = {
- isPathIdentifier(HoodieIdentifier(tableIdentifier.database.toArray, tableIdentifier.table))
- }
-
- private def getExistingTableIfExists(table: TableIdentifier): Option[CatalogTable] = {
- // If this is a path identifier, we cannot return an existing CatalogTable. The Create command
- // will check the file system itself
- val catalog = spark.sessionState.catalog
- // scalastyle:off
- if (isPathIdentifier(table)) return None
- // scalastyle:on
- val tableExists = catalog.tableExists(table)
- if (tableExists) {
- val oldTable = catalog.getTableMetadata(table)
- if (oldTable.tableType == CatalogTableType.VIEW) throw new HoodieException(
- s"$table is a view. You may not write data into a view.")
- if (!sparkAdapter.isHoodieTable(oldTable)) throw new HoodieException(s"$table is not a Hoodie table.")
- Some(oldTable)
- } else None
- }
-
- private def saveSourceDF(sourceQuery: Option[Dataset[_]],
- properties: Map[String, String]): Unit = {
- sourceQuery.map(df => {
- df.write.format("org.apache.hudi")
- .options(properties)
- .mode(SaveMode.Append)
- .save()
- df
- })
- }
-}
-
-object HoodieCatalog {
- def convertTransforms(partitions: Seq[Transform]): (Seq[String], Option[BucketSpec]) = {
- val identityCols = new mutable.ArrayBuffer[String]
- var bucketSpec = Option.empty[BucketSpec]
-
- partitions.map {
- case IdentityTransform(FieldReference(Seq(col))) =>
- identityCols += col
-
-
- case BucketTransform(numBuckets, Seq(FieldReference(Seq(col))), _) =>
- bucketSpec = Some(BucketSpec(numBuckets, col :: Nil, Nil))
-
- case _ =>
- throw new HoodieException(s"Partitioning by expressions is not supported.")
- }
-
- (identityCols, bucketSpec)
- }
-}
\ No newline at end of file
diff --git a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/hudi/catalog/HoodieStagedTable.scala b/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/hudi/catalog/HoodieStagedTable.scala
deleted file mode 100644
index e18f23ebde03f..0000000000000
--- a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/hudi/catalog/HoodieStagedTable.scala
+++ /dev/null
@@ -1,98 +0,0 @@
-/*
- * 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.catalog
-
-import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.fs.Path
-import org.apache.hudi.DataSourceWriteOptions.RECORDKEY_FIELD
-import org.apache.spark.sql.DataFrame
-import org.apache.spark.sql.catalyst.catalog.CatalogTableType
-import org.apache.spark.sql.connector.catalog.{Identifier, StagedTable, SupportsWrite, TableCapability}
-import org.apache.spark.sql.connector.expressions.Transform
-import org.apache.spark.sql.connector.write.{LogicalWriteInfo, V1Write, WriteBuilder}
-import org.apache.spark.sql.types.StructType
-
-import java.net.URI
-import java.util
-import scala.collection.JavaConverters.{mapAsScalaMapConverter, setAsJavaSetConverter}
-
-case class HoodieStagedTable(ident: Identifier,
- locUriAndTableType: (URI, CatalogTableType),
- catalog: HoodieCatalog,
- override val schema: StructType,
- partitions: Array[Transform],
- override val properties: util.Map[String, String],
- mode: TableCreationMode) extends StagedTable with SupportsWrite {
-
- private var sourceQuery: Option[DataFrame] = None
- private var writeOptions: Map[String, String] = Map.empty
-
- override def commitStagedChanges(): Unit = {
- val props = new util.HashMap[String, String]()
- val optionsThroughProperties = properties.asScala.collect {
- case (k, _) if k.startsWith("option.") => k.stripPrefix("option.")
- }.toSet
- val sqlWriteOptions = new util.HashMap[String, String]()
- properties.asScala.foreach { case (k, v) =>
- if (!k.startsWith("option.") && !optionsThroughProperties.contains(k)) {
- props.put(k, v)
- } else if (optionsThroughProperties.contains(k)) {
- sqlWriteOptions.put(k, v)
- }
- }
- if (writeOptions.isEmpty && !sqlWriteOptions.isEmpty) {
- writeOptions = sqlWriteOptions.asScala.toMap
- }
- props.putAll(properties)
- props.put("hoodie.table.name", ident.name())
- props.put(RECORDKEY_FIELD.key, properties.get("primaryKey"))
- catalog.createHoodieTable(
- ident, schema, locUriAndTableType, partitions, props, writeOptions, sourceQuery, mode)
- }
-
- override def name(): String = ident.name()
-
- override def abortStagedChanges(): Unit = {
- clearTablePath(locUriAndTableType._1.getPath, catalog.spark.sparkContext.hadoopConfiguration)
- }
-
- private def clearTablePath(tablePath: String, conf: Configuration): Unit = {
- val path = new Path(tablePath)
- val fs = path.getFileSystem(conf)
- fs.delete(path, true)
- }
-
- override def capabilities(): util.Set[TableCapability] = Set(TableCapability.V1_BATCH_WRITE).asJava
-
- override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = {
- writeOptions = info.options.asCaseSensitiveMap().asScala.toMap
- new HoodieV1WriteBuilder
- }
-
- /*
- * WriteBuilder for creating a Hoodie table.
- */
- private class HoodieV1WriteBuilder extends WriteBuilder {
- override def build(): V1Write = () => {
- (data: DataFrame, overwrite: Boolean) => {
- sourceQuery = Option(data)
- }
- }
- }
-
-}
diff --git a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/hudi/command/AlterTableCommand.scala b/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/hudi/command/AlterTableCommand.scala
deleted file mode 100644
index bca3e7050c792..0000000000000
--- a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/hudi/command/AlterTableCommand.scala
+++ /dev/null
@@ -1,347 +0,0 @@
-/*
- * 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.command
-
-import java.net.URI
-import java.nio.charset.StandardCharsets
-import java.util
-import java.util.concurrent.atomic.AtomicInteger
-import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.fs.Path
-import org.apache.hudi.DataSourceWriteOptions._
-import org.apache.hudi.client.utils.SparkInternalSchemaConverter
-import org.apache.hudi.common.model.{HoodieCommitMetadata, WriteOperationType}
-import org.apache.hudi.{DataSourceOptionsHelper, DataSourceUtils}
-import org.apache.hudi.common.table.timeline.{HoodieActiveTimeline, HoodieInstant}
-import org.apache.hudi.common.table.timeline.HoodieInstant.State
-import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver}
-import org.apache.hudi.common.util.{CommitUtils, Option}
-import org.apache.hudi.config.HoodieWriteConfig
-import org.apache.hudi.internal.schema.InternalSchema
-import org.apache.hudi.internal.schema.action.TableChange.ColumnChangeID
-import org.apache.hudi.internal.schema.action.TableChanges
-import org.apache.hudi.internal.schema.convert.AvroInternalSchemaConverter
-import org.apache.hudi.internal.schema.utils.{SchemaChangeUtils, SerDeHelper}
-import org.apache.hudi.internal.schema.io.FileBasedInternalSchemaStorageManager
-import org.apache.hudi.table.HoodieSparkTable
-import org.apache.spark.api.java.JavaSparkContext
-import org.apache.spark.internal.Logging
-import org.apache.spark.sql.{Row, SparkSession}
-import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogTableType}
-import org.apache.spark.sql.connector.catalog.{TableCatalog, TableChange}
-import org.apache.spark.sql.connector.catalog.TableChange.{AddColumn, DeleteColumn, RemoveProperty, SetProperty}
-import org.apache.spark.sql.types.StructType
-
-import scala.collection.JavaConverters._
-import scala.util.control.NonFatal
-
-case class AlterTableCommand(table: CatalogTable, changes: Seq[TableChange], changeType: ColumnChangeID) extends HoodieLeafRunnableCommand with Logging {
- override def run(sparkSession: SparkSession): Seq[Row] = {
- changeType match {
- case ColumnChangeID.ADD => applyAddAction(sparkSession)
- case ColumnChangeID.DELETE => applyDeleteAction(sparkSession)
- case ColumnChangeID.UPDATE => applyUpdateAction(sparkSession)
- case ColumnChangeID.PROPERTY_CHANGE if (changes.filter(_.isInstanceOf[SetProperty]).size == changes.size) =>
- applyPropertySet(sparkSession)
- case ColumnChangeID.PROPERTY_CHANGE if (changes.filter(_.isInstanceOf[RemoveProperty]).size == changes.size) =>
- applyPropertyUnset(sparkSession)
- case ColumnChangeID.REPLACE => applyReplaceAction(sparkSession)
- case other => throw new RuntimeException(s"find unsupported alter command type: ${other}")
- }
- Seq.empty[Row]
- }
-
- def applyReplaceAction(sparkSession: SparkSession): Unit = {
- // convert to delete first then add again
- val deleteChanges = changes.filter(p => p.isInstanceOf[DeleteColumn]).map(_.asInstanceOf[DeleteColumn])
- val addChanges = changes.filter(p => p.isInstanceOf[AddColumn]).map(_.asInstanceOf[AddColumn])
- val (oldSchema, historySchema) = getInternalSchemaAndHistorySchemaStr(sparkSession)
- val newSchema = applyAddAction2Schema(sparkSession, applyDeleteAction2Schema(sparkSession, oldSchema, deleteChanges), addChanges)
- val verifiedHistorySchema = if (historySchema == null || historySchema.isEmpty) {
- SerDeHelper.inheritSchemas(oldSchema, "")
- } else {
- historySchema
- }
- AlterTableCommand.commitWithSchema(newSchema, verifiedHistorySchema, table, sparkSession)
- logInfo("column replace finished")
- }
-
- def applyAddAction2Schema(sparkSession: SparkSession, oldSchema: InternalSchema, addChanges: Seq[AddColumn]): InternalSchema = {
- val addChange = TableChanges.ColumnAddChange.get(oldSchema)
- addChanges.foreach { addColumn =>
- val names = addColumn.fieldNames()
- val parentName = AlterTableCommand.getParentName(names)
- // add col change
- val colType = SparkInternalSchemaConverter.buildTypeFromStructType(addColumn.dataType(), true, new AtomicInteger(0))
- addChange.addColumns(parentName, names.last, colType, addColumn.comment())
- // add position change
- addColumn.position() match {
- case after: TableChange.After =>
- addChange.addPositionChange(names.mkString("."),
- if (parentName.isEmpty) after.column() else parentName + "." + after.column(), "after")
- case _: TableChange.First =>
- addChange.addPositionChange(names.mkString("."), "", "first")
- case _ =>
- }
- }
- SchemaChangeUtils.applyTableChanges2Schema(oldSchema, addChange)
- }
-
- def applyDeleteAction2Schema(sparkSession: SparkSession, oldSchema: InternalSchema, deleteChanges: Seq[DeleteColumn]): InternalSchema = {
- val deleteChange = TableChanges.ColumnDeleteChange.get(oldSchema)
- deleteChanges.foreach { c =>
- val originalColName = c.fieldNames().mkString(".")
- checkSchemaChange(Seq(originalColName), table)
- deleteChange.deleteColumn(originalColName)
- }
- SchemaChangeUtils.applyTableChanges2Schema(oldSchema, deleteChange).setSchemaId(oldSchema.getMaxColumnId)
- }
-
-
- def applyAddAction(sparkSession: SparkSession): Unit = {
- val (oldSchema, historySchema) = getInternalSchemaAndHistorySchemaStr(sparkSession)
- val newSchema = applyAddAction2Schema(sparkSession, oldSchema, changes.map(_.asInstanceOf[AddColumn]))
- val verifiedHistorySchema = if (historySchema == null || historySchema.isEmpty) {
- SerDeHelper.inheritSchemas(oldSchema, "")
- } else {
- historySchema
- }
- AlterTableCommand.commitWithSchema(newSchema, verifiedHistorySchema, table, sparkSession)
- logInfo("column add finished")
- }
-
- def applyDeleteAction(sparkSession: SparkSession): Unit = {
- val (oldSchema, historySchema) = getInternalSchemaAndHistorySchemaStr(sparkSession)
- val newSchema = applyDeleteAction2Schema(sparkSession, oldSchema, changes.map(_.asInstanceOf[DeleteColumn]))
- // delete action should not change the getMaxColumnId field.
- newSchema.setMaxColumnId(oldSchema.getMaxColumnId)
- val verifiedHistorySchema = if (historySchema == null || historySchema.isEmpty) {
- SerDeHelper.inheritSchemas(oldSchema, "")
- } else {
- historySchema
- }
- AlterTableCommand.commitWithSchema(newSchema, verifiedHistorySchema, table, sparkSession)
- logInfo("column delete finished")
- }
-
- def applyUpdateAction(sparkSession: SparkSession): Unit = {
- val (oldSchema, historySchema) = getInternalSchemaAndHistorySchemaStr(sparkSession)
- val updateChange = TableChanges.ColumnUpdateChange.get(oldSchema)
- changes.foreach { change =>
- change match {
- case updateType: TableChange.UpdateColumnType =>
- val newType = SparkInternalSchemaConverter.buildTypeFromStructType(updateType.newDataType(), true, new AtomicInteger(0))
- updateChange.updateColumnType(updateType.fieldNames().mkString("."), newType)
- case updateComment: TableChange.UpdateColumnComment =>
- updateChange.updateColumnComment(updateComment.fieldNames().mkString("."), updateComment.newComment())
- case updateName: TableChange.RenameColumn =>
- val originalColName = updateName.fieldNames().mkString(".")
- checkSchemaChange(Seq(originalColName), table)
- updateChange.renameColumn(originalColName, updateName.newName())
- case updateNullAbility: TableChange.UpdateColumnNullability =>
- updateChange.updateColumnNullability(updateNullAbility.fieldNames().mkString("."), updateNullAbility.nullable())
- case updatePosition: TableChange.UpdateColumnPosition =>
- val names = updatePosition.fieldNames()
- val parentName = AlterTableCommand.getParentName(names)
- updatePosition.position() match {
- case after: TableChange.After =>
- updateChange.addPositionChange(names.mkString("."),
- if (parentName.isEmpty) after.column() else parentName + "." + after.column(), "after")
- case _: TableChange.First =>
- updateChange.addPositionChange(names.mkString("."), "", "first")
- case _ =>
- }
- }
- }
- val newSchema = SchemaChangeUtils.applyTableChanges2Schema(oldSchema, updateChange)
- val verifiedHistorySchema = if (historySchema == null || historySchema.isEmpty) {
- SerDeHelper.inheritSchemas(oldSchema, "")
- } else {
- historySchema
- }
- AlterTableCommand.commitWithSchema(newSchema, verifiedHistorySchema, table, sparkSession)
- logInfo("column update finished")
- }
-
- // to do support unset default value to columns, and apply them to internalSchema
- def applyPropertyUnset(sparkSession: SparkSession): Unit = {
- val catalog = sparkSession.sessionState.catalog
- val propKeys = changes.map(_.asInstanceOf[RemoveProperty]).map(_.property())
- // ignore NonExist unset
- propKeys.foreach { k =>
- if (!table.properties.contains(k) && k != TableCatalog.PROP_COMMENT) {
- logWarning(s"find non exist unset property: ${k} , ignore it")
- }
- }
- val tableComment = if (propKeys.contains(TableCatalog.PROP_COMMENT)) None else table.comment
- val newProperties = table.properties.filter { case (k, _) => !propKeys.contains(k) }
- val newTable = table.copy(properties = newProperties, comment = tableComment)
- catalog.alterTable(newTable)
- logInfo("table properties change finished")
- }
-
- // to do support set default value to columns, and apply them to internalSchema
- def applyPropertySet(sparkSession: SparkSession): Unit = {
- val catalog = sparkSession.sessionState.catalog
- val properties = changes.map(_.asInstanceOf[SetProperty]).map(f => f.property -> f.value).toMap
- // This overrides old properties and update the comment parameter of CatalogTable
- // with the newly added/modified comment since CatalogTable also holds comment as its
- // direct property.
- val newTable = table.copy(
- properties = table.properties ++ properties,
- comment = properties.get(TableCatalog.PROP_COMMENT).orElse(table.comment))
- catalog.alterTable(newTable)
- logInfo("table properties change finished")
- }
-
- def getInternalSchemaAndHistorySchemaStr(sparkSession: SparkSession): (InternalSchema, String) = {
- val path = AlterTableCommand.getTableLocation(table, sparkSession)
- val hadoopConf = sparkSession.sessionState.newHadoopConf()
- val metaClient = HoodieTableMetaClient.builder().setBasePath(path)
- .setConf(hadoopConf).build()
- val schemaUtil = new TableSchemaResolver(metaClient)
-
- val schema = schemaUtil.getTableInternalSchemaFromCommitMetadata().orElse {
- AvroInternalSchemaConverter.convert(schemaUtil.getTableAvroSchema)
- }
-
- val historySchemaStr = schemaUtil.getTableHistorySchemaStrFromCommitMetadata.orElse("")
- (schema, historySchemaStr)
- }
-
- def checkSchemaChange(colNames: Seq[String], catalogTable: CatalogTable): Unit = {
- val primaryKeys = catalogTable.storage.properties.getOrElse("primaryKey", catalogTable.properties.getOrElse("primaryKey", "keyid")).split(",").map(_.trim)
- val preCombineKey = Seq(catalogTable.storage.properties.getOrElse("preCombineField", catalogTable.properties.getOrElse("preCombineField", "ts"))).map(_.trim)
- val partitionKey = catalogTable.partitionColumnNames.map(_.trim)
- val checkNames = primaryKeys ++ preCombineKey ++ partitionKey
- colNames.foreach { col =>
- if (checkNames.contains(col)) {
- throw new UnsupportedOperationException("cannot support apply changes for primaryKey/CombineKey/partitionKey")
- }
- }
- }
-}
-
-object AlterTableCommand extends Logging {
-
- /**
- * Generate an commit with new schema to change the table's schema.
- *
- * @param internalSchema new schema after change
- * @param historySchemaStr history schemas
- * @param table The hoodie table.
- * @param sparkSession The spark session.
- */
- def commitWithSchema(internalSchema: InternalSchema, historySchemaStr: String, table: CatalogTable, sparkSession: SparkSession): Unit = {
- val schema = AvroInternalSchemaConverter.convert(internalSchema, table.identifier.table)
- val path = getTableLocation(table, sparkSession)
- val jsc = new JavaSparkContext(sparkSession.sparkContext)
- val client = DataSourceUtils.createHoodieClient(jsc, schema.toString,
- path, table.identifier.table, parametersWithWriteDefaults(table.storage.properties).asJava)
-
- val hadoopConf = sparkSession.sessionState.newHadoopConf()
- val metaClient = HoodieTableMetaClient.builder().setBasePath(path).setConf(hadoopConf).build()
-
- val commitActionType = CommitUtils.getCommitActionType(WriteOperationType.ALTER_SCHEMA, metaClient.getTableType)
- val instantTime = HoodieActiveTimeline.createNewInstantTime
- client.startCommitWithTime(instantTime, commitActionType)
-
- val hoodieTable = HoodieSparkTable.create(client.getConfig, client.getEngineContext)
- val timeLine = hoodieTable.getActiveTimeline
- val requested = new HoodieInstant(State.REQUESTED, commitActionType, instantTime)
- val metadata = new HoodieCommitMetadata
- metadata.setOperationType(WriteOperationType.ALTER_SCHEMA)
- timeLine.transitionRequestedToInflight(requested, Option.of(metadata.toJsonString.getBytes(StandardCharsets.UTF_8)))
- val extraMeta = new util.HashMap[String, String]()
- extraMeta.put(SerDeHelper.LATEST_SCHEMA, SerDeHelper.toJson(internalSchema.setSchemaId(instantTime.toLong)))
- val schemaManager = new FileBasedInternalSchemaStorageManager(metaClient)
- schemaManager.persistHistorySchemaStr(instantTime, SerDeHelper.inheritSchemas(internalSchema, historySchemaStr))
- client.commit(instantTime, jsc.emptyRDD, Option.of(extraMeta))
- val existRoTable = sparkSession.catalog.tableExists(table.identifier.unquotedString + "_ro")
- val existRtTable = sparkSession.catalog.tableExists(table.identifier.unquotedString + "_rt")
- try {
- sparkSession.catalog.refreshTable(table.identifier.unquotedString)
- // try to refresh ro/rt table
- if (existRoTable) sparkSession.catalog.refreshTable(table.identifier.unquotedString + "_ro")
- if (existRoTable) sparkSession.catalog.refreshTable(table.identifier.unquotedString + "_rt")
- } catch {
- case NonFatal(e) =>
- log.error(s"Exception when attempting to refresh table ${table.identifier.quotedString}", e)
- }
- // try to sync to hive
- // drop partition field before call alter table
- val fullSparkSchema = SparkInternalSchemaConverter.constructSparkSchemaFromInternalSchema(internalSchema)
- val dataSparkSchema = new StructType(fullSparkSchema.fields.filter(p => !table.partitionColumnNames.exists(f => sparkSession.sessionState.conf.resolver(f, p.name))))
- alterTableDataSchema(sparkSession, table.identifier.database.getOrElse("default"), table.identifier.table, dataSparkSchema)
- if (existRoTable) alterTableDataSchema(sparkSession, table.identifier.database.getOrElse("default"), table.identifier.table + "_ro", dataSparkSchema)
- if (existRtTable) alterTableDataSchema(sparkSession, table.identifier.database.getOrElse("default"), table.identifier.table + "_rt", dataSparkSchema)
- }
-
- def alterTableDataSchema(sparkSession: SparkSession, db: String, tableName: String, dataSparkSchema: StructType): Unit = {
- sparkSession.sessionState.catalog
- .externalCatalog
- .alterTableDataSchema(db, tableName, dataSparkSchema)
- }
-
- def getTableLocation(table: CatalogTable, sparkSession: SparkSession): String = {
- val uri = if (table.tableType == CatalogTableType.MANAGED) {
- Some(sparkSession.sessionState.catalog.defaultTablePath(table.identifier))
- } else {
- table.storage.locationUri
- }
- val conf = sparkSession.sessionState.newHadoopConf()
- uri.map(makePathQualified(_, conf))
- .map(removePlaceHolder)
- .getOrElse(throw new IllegalArgumentException(s"Missing location for ${table.identifier}"))
- }
-
- private def removePlaceHolder(path: String): String = {
- if (path == null || path.length == 0) {
- path
- } else if (path.endsWith("-PLACEHOLDER")) {
- path.substring(0, path.length() - 16)
- } else {
- path
- }
- }
-
- def makePathQualified(path: URI, hadoopConf: Configuration): String = {
- val hadoopPath = new Path(path)
- val fs = hadoopPath.getFileSystem(hadoopConf)
- fs.makeQualified(hadoopPath).toUri.toString
- }
-
- def getParentName(names: Array[String]): String = {
- if (names.size > 1) {
- names.dropRight(1).mkString(".")
- } else ""
- }
-
- def parametersWithWriteDefaults(parameters: Map[String, String]): Map[String, String] = {
- Map(OPERATION.key -> OPERATION.defaultValue,
- TABLE_TYPE.key -> TABLE_TYPE.defaultValue,
- PRECOMBINE_FIELD.key -> PRECOMBINE_FIELD.defaultValue,
- HoodieWriteConfig.WRITE_PAYLOAD_CLASS_NAME.key -> HoodieWriteConfig.DEFAULT_WRITE_PAYLOAD_CLASS,
- INSERT_DROP_DUPS.key -> INSERT_DROP_DUPS.defaultValue,
- ASYNC_COMPACT_ENABLE.key -> ASYNC_COMPACT_ENABLE.defaultValue,
- INLINE_CLUSTERING_ENABLE.key -> INLINE_CLUSTERING_ENABLE.defaultValue,
- ASYNC_CLUSTERING_ENABLE.key -> ASYNC_CLUSTERING_ENABLE.defaultValue
- ) ++ DataSourceOptionsHelper.translateConfigurations(parameters)
- }
-}
-
diff --git a/hudi-utilities/pom.xml b/hudi-utilities/pom.xml
index a4690fd3f9f93..16d0d47d4f098 100644
--- a/hudi-utilities/pom.xml
+++ b/hudi-utilities/pom.xml
@@ -112,25 +112,11 @@
test
-
-
- org.apache.hudi
- hudi-client-common
- ${project.version}
-
-
- org.apache.hudi
- hudi-spark-client
- ${project.version}
-
-
- org.apache.hudi
- hudi-common
- ${project.version}
-
+
+
org.apache.hudi
- hudi-hive-sync
+ hudi-spark_${scala.binary.version}${project.version}
@@ -139,14 +125,22 @@
+
+
+
+
org.apache.hudi
- hudi-spark-common_${scala.binary.version}
+ hudi-common${project.version}
+
org.apache.hudi
- hudi-spark_${scala.binary.version}
+ hudi-hive-sync${project.version}
@@ -156,18 +150,6 @@
-
- org.apache.hudi
- ${hudi.spark.module}_${scala.binary.version}
- ${project.version}
-
-
-
- org.apache.hudi
- ${hudi.spark.common.module}
- ${project.version}
-
-
org.apache.kafka
diff --git a/packaging/hudi-integ-test-bundle/pom.xml b/packaging/hudi-integ-test-bundle/pom.xml
index 84aee680a6e9b..776f9f73d1e46 100644
--- a/packaging/hudi-integ-test-bundle/pom.xml
+++ b/packaging/hudi-integ-test-bundle/pom.xml
@@ -76,10 +76,11 @@
org.apache.hudi:hudi-client-commonorg.apache.hudi:hudi-spark-clientorg.apache.hudi:hudi-spark-common_${scala.binary.version}
- org.apache.hudi:hudi-utilities_${scala.binary.version}org.apache.hudi:hudi-spark_${scala.binary.version}org.apache.hudi:${hudi.spark.module}_${scala.binary.version}
- org.apache.hudi:${hudi.spark.common.module}
+ org.apache.hudi:${hudi.spark.common.modules.1}
+ org.apache.hudi:${hudi.spark.common.modules.2}
+ org.apache.hudi:hudi-utilities_${scala.binary.version}org.apache.hudi:hudi-hive-syncorg.apache.hudi:hudi-sync-commonorg.apache.hudi:hudi-hadoop-mr
@@ -438,45 +439,76 @@
2.7.4
+
+
org.apache.hudi
- hudi-common
+ hudi-spark_${scala.binary.version}${project.version}
+
+
org.apache.hudi
- hudi-common
+ hudi-hive-sync
+ ${project.version}
+
+
+ javax.servlet
+ servlet-api
+
+
+
+
+
+ org.apache.hudi
+ hudi-utilities_${scala.binary.version}${project.version}
- tests
- test-jarcompileorg.apache.hudi
- hudi-hive-sync
+ hudi-integ-test
+ ${project.version}
+
+
+
+
+
+ org.apache.hudi
+ hudi-client-common${project.version}teststest-jar
+ testorg.apache.hudi
- hudi-spark_${scala.binary.version}
+ hudi-spark-client${project.version}
+ tests
+ test-jar
+ testorg.apache.hudi
- ${hudi.spark.module}_${scala.binary.version}
+ hudi-common${project.version}
+ tests
+ test-jar
+ compileorg.apache.hudi
- ${hudi.spark.common.module}
+ hudi-hive-sync${project.version}
+ tests
+ test-jar
@@ -569,59 +601,6 @@
compile
-
- org.apache.hudi
- hudi-hive-sync
- ${project.version}
-
-
- javax.servlet
- servlet-api
-
-
-
-
-
- org.apache.hudi
- hudi-client-common
- ${project.version}
-
-
- org.apache.hudi
- hudi-spark-client
- ${project.version}
-
-
-
- org.apache.hudi
- hudi-utilities_${scala.binary.version}
- ${project.version}
- compile
-
-
-
- org.apache.hudi
- hudi-integ-test
- ${project.version}
-
-
-
- org.apache.hudi
- hudi-client-common
- ${project.version}
- tests
- test-jar
- test
-
-
- org.apache.hudi
- hudi-spark-client
- ${project.version}
- tests
- test-jar
- test
-
-
commons-codeccommons-codec
diff --git a/packaging/hudi-spark-bundle/pom.xml b/packaging/hudi-spark-bundle/pom.xml
index 63a2a451b9f23..4d474ae67edc3 100644
--- a/packaging/hudi-spark-bundle/pom.xml
+++ b/packaging/hudi-spark-bundle/pom.xml
@@ -75,7 +75,8 @@
org.apache.hudi:hudi-spark-common_${scala.binary.version}org.apache.hudi:hudi-spark_${scala.binary.version}org.apache.hudi:${hudi.spark.module}_${scala.binary.version}
- org.apache.hudi:${hudi.spark.common.module}
+ org.apache.hudi:${hudi.spark.common.modules.1}
+ org.apache.hudi:${hudi.spark.common.modules.2}org.apache.hudi:hudi-hive-syncorg.apache.hudi:hudi-sync-commonorg.apache.hudi:hudi-hadoop-mr
@@ -140,14 +141,14 @@
-
- javax.servlet.
- org.apache.hudi.javax.servlet.
- org.apache.spark.sql.avro.org.apache.hudi.org.apache.spark.sql.avro.
+
+ javax.servlet.
+ org.apache.hudi.javax.servlet.
+ com.yammer.metrics.org.apache.hudi.com.yammer.metrics.
@@ -319,52 +320,28 @@
-
-
- org.apache.hudi
- hudi-common
- ${project.version}
-
-
- org.apache.hudi
- hudi-client-common
- ${project.version}
-
+
+
org.apache.hudi
- hudi-spark-client
+ hudi-spark_${scala.binary.version}${project.version}
+
+
+
org.apache.hudihudi-hadoop-mr${project.version}
+
org.apache.hudihudi-hive-sync${project.version}
-
- org.apache.hudi
- hudi-spark-common_${scala.binary.version}
- ${project.version}
-
-
- org.apache.hudi
- hudi-spark_${scala.binary.version}
- ${project.version}
-
-
- org.apache.hudi
- ${hudi.spark.module}_${scala.binary.version}
- ${project.version}
-
-
- org.apache.hudi
- ${hudi.spark.common.module}
- ${project.version}
-
+
org.apache.hudihudi-timeline-service
diff --git a/packaging/hudi-utilities-bundle/pom.xml b/packaging/hudi-utilities-bundle/pom.xml
index 75f8ecdefd988..86b1eea2a1320 100644
--- a/packaging/hudi-utilities-bundle/pom.xml
+++ b/packaging/hudi-utilities-bundle/pom.xml
@@ -98,7 +98,8 @@
org.apache.hudi:hudi-spark-common_${scala.binary.version}org.apache.hudi:hudi-spark_${scala.binary.version}org.apache.hudi:${hudi.spark.module}_${scala.binary.version}
- org.apache.hudi:${hudi.spark.common.module}
+ org.apache.hudi:${hudi.spark.common.modules.1}
+ org.apache.hudi:${hudi.spark.common.modules.2}org.apache.hudi:hudi-hive-syncorg.apache.hudi:hudi-sync-commonorg.apache.hudi:hudi-hadoop-mr
@@ -342,22 +343,16 @@
-
-
- org.apache.hudi
- hudi-common
- ${project.version}
-
-
- org.apache.hudi
- hudi-client-common
- ${project.version}
-
+
+
org.apache.hudi
- hudi-spark-client
+ hudi-spark_${scala.binary.version}${project.version}
+
+
+
org.apache.hudihudi-hive-sync
@@ -369,26 +364,7 @@
-
- org.apache.hudi
- hudi-spark-common_${scala.binary.version}
- ${project.version}
-
-
- org.apache.hudi
- hudi-spark_${scala.binary.version}
- ${project.version}
-
-
- org.apache.hudi
- ${hudi.spark.module}_${scala.binary.version}
- ${project.version}
-
-
- org.apache.hudi
- ${hudi.spark.common.module}
- ${project.version}
-
+
org.apache.hudihudi-utilities_${scala.binary.version}
@@ -408,6 +384,12 @@
hive-service${hive.version}${utilities.bundle.hive.scope}
+
+
+ org.apache.hbase
+ *
+
+
@@ -429,6 +411,12 @@
hive-metastore${hive.version}${utilities.bundle.hive.scope}
+
+
+ org.apache.hbase
+ *
+
+
diff --git a/packaging/hudi-utilities-slim-bundle/pom.xml b/packaging/hudi-utilities-slim-bundle/pom.xml
index 392d7a8f7db32..50cbb8ea15b4d 100644
--- a/packaging/hudi-utilities-slim-bundle/pom.xml
+++ b/packaging/hudi-utilities-slim-bundle/pom.xml
@@ -325,22 +325,10 @@
hudi-utilities_${scala.binary.version}${project.version}
-
- org.apache.hudi
- hudi-spark-common_${scala.binary.version}
- org.apache.hudihudi-spark_${scala.binary.version}
-
- org.apache.hudi
- ${hudi.spark.module}_${scala.binary.version}
-
-
- org.apache.hudi
- ${hudi.spark.common.module}
-
diff --git a/pom.xml b/pom.xml
index b5c9116a97911..ff361e64e0704 100644
--- a/pom.xml
+++ b/pom.xml
@@ -149,12 +149,17 @@
3.2.13.3.0hudi-spark2
- hudi-spark2-common
+
+ hudi-spark2-common
+ 1.8.22.9.12.11.122.12.10${scala11.version}
+ 2.8.12.110.133.3.1
@@ -165,7 +170,6 @@
9.4.15.v201902153.1.0-incubating2.4.9
- 3.5.11.9.131.4.1993.1.2
@@ -377,9 +381,17 @@
org.sl4fj:slf4j-jcllog4j:log4jch.qos.logback:logback-classic
+
+ org.apache.hbase:hbase-common:*
+ org.apache.hbase:hbase-client:*
+ org.apache.hbase:hbase-server:*org.slf4j:slf4j-simple:*:*:test
+ org.apache.hbase:hbase-common:${hbase.version}
+ org.apache.hbase:hbase-client:${hbase.version}
+ org.apache.hbase:hbase-server:${hbase.version}
@@ -612,6 +624,19 @@
+
+
+ org.scala-lang
+ scala-library
+ ${scala.version}
+
+
+
+ org.scala-lang.modules
+ scala-collection-compat_${scala.binary.version}
+ ${scala.collection-compat.version}
+
+
+
+
spark2
@@ -1930,7 +1980,8 @@
-
+
+
spark3
@@ -1959,6 +2010,7 @@
hudi-spark-datasource/hudi-spark3.3.xhudi-spark-datasource/hudi-spark3-common
+ hudi-spark-datasource/hudi-spark3.2plus-common
@@ -1976,7 +2028,7 @@
${scala12.version}2.12hudi-spark3.1.x
- hudi-spark3-common
+ hudi-spark3-common${scalatest.spark3.version}${kafka.spark3.version}4.8-1
@@ -2008,7 +2060,9 @@
${scala12.version}2.12hudi-spark3.2.x
- hudi-spark3-common
+
+ hudi-spark3-common
+ hudi-spark3.2plus-common${scalatest.spark3.version}${kafka.spark3.version}1.12.2
@@ -2026,6 +2080,7 @@
hudi-spark-datasource/hudi-spark3.2.xhudi-spark-datasource/hudi-spark3-common
+ hudi-spark-datasource/hudi-spark3.2plus-common
@@ -2043,7 +2098,9 @@
${scala12.version}2.12hudi-spark3.3.x
- hudi-spark3-common
+
+ hudi-spark3-common
+ hudi-spark3.2plus-common${scalatest.spark3.version}${kafka.spark3.version}1.12.2
@@ -2062,6 +2119,7 @@
hudi-spark-datasource/hudi-spark3.3.xhudi-spark-datasource/hudi-spark3-common
+ hudi-spark-datasource/hudi-spark3.2plus-common