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-lang scala-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.hudi hudi-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-lang scala-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-lang scala-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.hudi hudi-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-lang scala-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.hudi hudi-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: + * + *
    + *
  1. Number of the buckets
  2. + *
  3. Seq of references (to be bucketed by)
  4. + *
  5. Seq of sorted references
  6. + *
+ */ + 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.hudi hudi-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.hudi hudi-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: - *
      - *
    1. Avoiding appending partition values to the rows read from the data file
    2. - *
    3. Schema on-read
    4. - *
    - */ -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-common org.apache.hudi:hudi-spark-client org.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-sync org.apache.hudi:hudi-sync-common org.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-jar compile org.apache.hudi - hudi-hive-sync + hudi-integ-test + ${project.version} + + + + + + org.apache.hudi + hudi-client-common ${project.version} tests test-jar + test org.apache.hudi - hudi-spark_${scala.binary.version} + hudi-spark-client ${project.version} + tests + test-jar + test org.apache.hudi - ${hudi.spark.module}_${scala.binary.version} + hudi-common ${project.version} + tests + test-jar + compile org.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-codec commons-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-sync org.apache.hudi:hudi-sync-common org.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.hudi hudi-hadoop-mr ${project.version} + org.apache.hudi hudi-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.hudi hudi-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-sync org.apache.hudi:hudi-sync-common org.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.hudi hudi-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.hudi hudi-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.hudi hudi-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.1 3.3.0 hudi-spark2 - hudi-spark2-common + + hudi-spark2-common + 1.8.2 2.9.1 2.11.12 2.12.10 ${scala11.version} + 2.8.1 2.11 0.13 3.3.1 @@ -165,7 +170,6 @@ 9.4.15.v20190215 3.1.0-incubating 2.4.9 - 3.5.1 1.9.13 1.4.199 3.1.2 @@ -377,9 +381,17 @@ org.sl4fj:slf4j-jcl log4j:log4j ch.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.x hudi-spark-datasource/hudi-spark3-common + hudi-spark-datasource/hudi-spark3.2plus-common @@ -1976,7 +2028,7 @@ ${scala12.version} 2.12 hudi-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.12 hudi-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.x hudi-spark-datasource/hudi-spark3-common + hudi-spark-datasource/hudi-spark3.2plus-common @@ -2043,7 +2098,9 @@ ${scala12.version} 2.12 hudi-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.x hudi-spark-datasource/hudi-spark3-common + hudi-spark-datasource/hudi-spark3.2plus-common