diff --git a/.github/workflows/bot.yml b/.github/workflows/bot.yml
index 59eada9cc647f..306b6ea1fd5de 100644
--- a/.github/workflows/bot.yml
+++ b/.github/workflows/bot.yml
@@ -63,6 +63,10 @@ jobs:
sparkProfile: "spark3.1"
sparkModules: "hudi-spark-datasource/hudi-spark3.1.x"
+ - scalaProfile: "scala-2.12"
+ sparkProfile: "spark3.0"
+ sparkModules: "hudi-spark-datasource/hudi-spark3.0.x"
+
- scalaProfile: "scala-2.12"
sparkProfile: "spark3.2"
sparkModules: "hudi-spark-datasource/hudi-spark3.2.x"
@@ -159,6 +163,9 @@ jobs:
- flinkProfile: 'flink1.13'
sparkProfile: 'spark3.1'
sparkRuntime: 'spark3.1.3'
+ - flinkProfile: 'flink1.14'
+ sparkProfile: 'spark3.0'
+ sparkRuntime: 'spark3.0.2'
steps:
- uses: actions/checkout@v2
- name: Set up JDK 8
diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkAdapterSupport.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkAdapterSupport.scala
index 9fe67f9918d01..0e29b36c18dcc 100644
--- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkAdapterSupport.scala
+++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkAdapterSupport.scala
@@ -37,8 +37,10 @@ object SparkAdapterSupport {
"org.apache.spark.sql.adapter.Spark3_3Adapter"
} else if (HoodieSparkUtils.isSpark3_2) {
"org.apache.spark.sql.adapter.Spark3_2Adapter"
- } else if (HoodieSparkUtils.isSpark3_0 || HoodieSparkUtils.isSpark3_1) {
+ } else if (HoodieSparkUtils.isSpark3_1) {
"org.apache.spark.sql.adapter.Spark3_1Adapter"
+ } else if (HoodieSparkUtils.isSpark3_0) {
+ "org.apache.spark.sql.adapter.Spark3_0Adapter"
} else {
"org.apache.spark.sql.adapter.Spark2Adapter"
}
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 1470133df1743..d575e69bc1d32 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
@@ -84,6 +84,8 @@ object HoodieAnalysis extends SparkAdapterSupport {
"org.apache.spark.sql.hudi.Spark32ResolveHudiAlterTableCommand"
} else if (HoodieSparkUtils.gteqSpark3_1) {
"org.apache.spark.sql.hudi.Spark31ResolveHudiAlterTableCommand"
+ } else if (HoodieSparkUtils.gteqSpark3_0) {
+ "org.apache.spark.sql.hudi.Spark30ResolveHudiAlterTableCommand"
} else {
throw new IllegalStateException("Unsupported Spark version")
}
@@ -126,15 +128,18 @@ object HoodieAnalysis extends SparkAdapterSupport {
// Default rules
)
- if (HoodieSparkUtils.gteqSpark3_1) {
+ if (HoodieSparkUtils.gteqSpark3_0) {
val nestedSchemaPruningClass =
if (HoodieSparkUtils.gteqSpark3_3) {
"org.apache.spark.sql.execution.datasources.Spark33NestedSchemaPruning"
} else if (HoodieSparkUtils.gteqSpark3_2) {
"org.apache.spark.sql.execution.datasources.Spark32NestedSchemaPruning"
- } else {
+ } else if (HoodieSparkUtils.gteqSpark3_1) {
// spark 3.1
"org.apache.spark.sql.execution.datasources.Spark31NestedSchemaPruning"
+ } else {
+ // spark 3.0
+ "org.apache.spark.sql.execution.datasources.Spark30NestedSchemaPruning"
}
val nestedSchemaPruningRule = ReflectionUtils.loadClass(nestedSchemaPruningClass).asInstanceOf[Rule[LogicalPlan]]
diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/avro/TestAvroSerDe.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/avro/TestAvroSerDe.scala
index 069d56f282d27..bf68e34fb648c 100644
--- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/avro/TestAvroSerDe.scala
+++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/avro/TestAvroSerDe.scala
@@ -20,6 +20,7 @@ package org.apache.spark.sql.avro
import org.apache.avro.generic.GenericData
import org.apache.hudi.SparkAdapterSupport
import org.apache.hudi.avro.model.{HoodieMetadataColumnStats, IntWrapper}
+import org.apache.spark.internal.Logging
import org.apache.spark.sql.avro.SchemaConverters.SchemaType
import org.junit.jupiter.api.Assertions.assertEquals
import org.junit.jupiter.api.Test
diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestMergeIntoTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestMergeIntoTable.scala
index 574ddd99b0fb4..247e4171198b3 100644
--- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestMergeIntoTable.scala
+++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestMergeIntoTable.scala
@@ -114,58 +114,65 @@ class TestMergeIntoTable extends HoodieSparkSqlTestBase with ScalaAssertionSuppo
})
}
- test("Test MergeInto with more than once update actions") {
- withRecordType()(withTempDir {tmp =>
- val targetTable = generateTableName
- spark.sql(
- s"""
- |create table ${targetTable} (
- | id int,
- | name string,
- | data int,
- | country string,
- | ts bigint
- |) using hudi
- |tblproperties (
- | type = 'cow',
- | primaryKey = 'id',
- | preCombineField = 'ts'
- | )
- |partitioned by (country)
- |location '${tmp.getCanonicalPath}/$targetTable'
- |""".stripMargin)
- spark.sql(
- s"""
- |merge into ${targetTable} as target
- |using (
- |select 1 as id, 'lb' as name, 6 as data, 'shu' as country, 1646643193 as ts
- |) source
- |on source.id = target.id
- |when matched then
- |update set *
- |when not matched then
- |insert *
- |""".stripMargin)
- spark.sql(
- s"""
- |merge into ${targetTable} as target
- |using (
- |select 1 as id, 'lb' as name, 5 as data, 'shu' as country, 1646643196 as ts
- |) source
- |on source.id = target.id
- |when matched and source.data > target.data then
- |update set target.data = source.data, target.ts = source.ts
- |when matched and source.data = 5 then
- |update set target.data = source.data, target.ts = source.ts
- |when not matched then
- |insert *
- |""".stripMargin)
-
- checkAnswer(s"select id, name, data, country, ts from $targetTable")(
- Seq(1, "lb", 5, "shu", 1646643196L)
- )
+ /**
+ * In Spark 3.0.x, UPDATE and DELETE can appear at most once in MATCHED clauses in a MERGE INTO statement.
+ * Refer to: `org.apache.spark.sql.catalyst.parser.AstBuilder#visitMergeIntoTable`
+ *
+ */
+ test("Test MergeInto with more than once update actions for spark >= 3.1.x") {
+
+ if (HoodieSparkUtils.gteqSpark3_1) {
+ withRecordType()(withTempDir { tmp =>
+ val targetTable = generateTableName
+ spark.sql(
+ s"""
+ |create table ${targetTable} (
+ | id int,
+ | name string,
+ | data int,
+ | country string,
+ | ts bigint
+ |) using hudi
+ |tblproperties (
+ | type = 'cow',
+ | primaryKey = 'id',
+ | preCombineField = 'ts'
+ | )
+ |partitioned by (country)
+ |location '${tmp.getCanonicalPath}/$targetTable'
+ |""".stripMargin)
+ spark.sql(
+ s"""
+ |merge into ${targetTable} as target
+ |using (
+ |select 1 as id, 'lb' as name, 6 as data, 'shu' as country, 1646643193 as ts
+ |) source
+ |on source.id = target.id
+ |when matched then
+ |update set *
+ |when not matched then
+ |insert *
+ |""".stripMargin)
+ spark.sql(
+ s"""
+ |merge into ${targetTable} as target
+ |using (
+ |select 1 as id, 'lb' as name, 5 as data, 'shu' as country, 1646643196 as ts
+ |) source
+ |on source.id = target.id
+ |when matched and source.data > target.data then
+ |update set target.data = source.data, target.ts = source.ts
+ |when matched and source.data = 5 then
+ |update set target.data = source.data, target.ts = source.ts
+ |when not matched then
+ |insert *
+ |""".stripMargin)
- })
+ checkAnswer(s"select id, name, data, country, ts from $targetTable")(
+ Seq(1, "lb", 5, "shu", 1646643196L)
+ )
+ })
+ }
}
test("Test MergeInto with ignored record") {
diff --git a/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/HoodieSpark3CatalystExpressionUtils.scala b/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/HoodieSpark3CatalystExpressionUtils.scala
index b8ddb0a799bed..cf63383ef35a7 100644
--- a/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/HoodieSpark3CatalystExpressionUtils.scala
+++ b/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/HoodieSpark3CatalystExpressionUtils.scala
@@ -17,16 +17,21 @@
package org.apache.spark.sql
-import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeSet, Expression, Predicate, PredicateHelper}
-import org.apache.spark.sql.execution.datasources.DataSourceStrategy
+import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeSet, Expression}
+abstract class HoodieSpark3CatalystExpressionUtils extends HoodieCatalystExpressionUtils {
-trait HoodieSpark3CatalystExpressionUtils extends HoodieCatalystExpressionUtils
- with PredicateHelper {
+ /**
+ * The attribute name may differ from the one in the schema if the query analyzer
+ * is case insensitive. We should change attribute names to match the ones in the schema,
+ * so we do not need to worry about case sensitivity anymore
+ */
+ def normalizeExprs(exprs: Seq[Expression], attributes: Seq[Attribute]): Seq[Expression]
- override def normalizeExprs(exprs: Seq[Expression], attributes: Seq[Attribute]): Seq[Expression] =
- DataSourceStrategy.normalizeExprs(exprs, attributes)
-
- override def extractPredicatesWithinOutputSet(condition: Expression,
- outputSet: AttributeSet): Option[Expression] =
- super[PredicateHelper].extractPredicatesWithinOutputSet(condition, outputSet)
+ /**
+ * Returns a filter that its reference is a subset of `outputSet` and it contains the maximum
+ * constraints from `condition`. This is used for predicate push-down
+ * When there is no such filter, `None` is returned.
+ */
+ def extractPredicatesWithinOutputSet(condition: Expression,
+ outputSet: AttributeSet): Option[Expression]
}
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 67912a7a22663..763374b36a836 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
@@ -25,22 +25,15 @@ import org.apache.hudi.spark3.internal.ReflectUtil
import org.apache.hudi.{AvroConversionUtils, DefaultSource, Spark3RowSerDe}
import org.apache.spark.internal.Logging
import org.apache.spark.sql.avro.{HoodieAvroSchemaConverters, HoodieSparkAvroSchemaConverters}
-import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases
-import org.apache.spark.sql.catalyst.catalog.CatalogTable
import org.apache.spark.sql.catalyst.encoders.RowEncoder
import org.apache.spark.sql.catalyst.expressions.{Expression, InterpretedPredicate, Predicate}
-import org.apache.spark.sql.catalyst.planning.PhysicalOperation
-import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.catalyst.util.DateFormatter
-import org.apache.spark.sql.connector.catalog.V2TableWithV1Fallback
import org.apache.spark.sql.execution.datasources._
-import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
import org.apache.spark.sql.hudi.SparkAdapter
import org.apache.spark.sql.sources.BaseRelation
-import org.apache.spark.sql.types.StructType
import org.apache.spark.sql.{HoodieSpark3CatalogUtils, SQLContext, SparkSession}
+import org.apache.spark.sql.types.StructType
import org.apache.spark.storage.StorageLevel
-import org.apache.spark.storage.StorageLevel._
import java.time.ZoneId
import java.util.TimeZone
@@ -63,20 +56,6 @@ abstract class BaseSpark3Adapter extends SparkAdapter with Logging {
new Spark3RowSerDe(encoder)
}
- override def resolveHoodieTable(plan: LogicalPlan): Option[CatalogTable] = {
- super.resolveHoodieTable(plan).orElse {
- EliminateSubqueryAliases(plan) match {
- // First, we need to weed out unresolved plans
- case plan if !plan.resolved => None
- // NOTE: When resolving Hudi table we allow [[Filter]]s and [[Project]]s be applied
- // on top of it
- case PhysicalOperation(_, _, DataSourceV2Relation(v2: V2TableWithV1Fallback, _, _, _, _)) if isHoodieTable(v2.v1Table) =>
- Some(v2.v1Table)
- case _ => None
- }
- }
- }
-
override def getAvroSchemaConverters: HoodieAvroSchemaConverters = HoodieSparkAvroSchemaConverters
override def getSparkParsePartitionUtil: SparkParsePartitionUtil = Spark3ParsePartitionUtil
@@ -108,23 +87,5 @@ abstract class BaseSpark3Adapter extends SparkAdapter with Logging {
DefaultSource.createRelation(sqlContext, metaClient, dataSchema, globPaths, parameters.asScala.toMap)
}
- /**
- * Converts instance of [[StorageLevel]] to a corresponding string
- */
- override def convertStorageLevelToString(level: StorageLevel): String = level match {
- case NONE => "NONE"
- case DISK_ONLY => "DISK_ONLY"
- case DISK_ONLY_2 => "DISK_ONLY_2"
- case DISK_ONLY_3 => "DISK_ONLY_3"
- case MEMORY_ONLY => "MEMORY_ONLY"
- case MEMORY_ONLY_2 => "MEMORY_ONLY_2"
- case MEMORY_ONLY_SER => "MEMORY_ONLY_SER"
- case MEMORY_ONLY_SER_2 => "MEMORY_ONLY_SER_2"
- case MEMORY_AND_DISK => "MEMORY_AND_DISK"
- case MEMORY_AND_DISK_2 => "MEMORY_AND_DISK_2"
- case MEMORY_AND_DISK_SER => "MEMORY_AND_DISK_SER"
- case MEMORY_AND_DISK_SER_2 => "MEMORY_AND_DISK_SER_2"
- case OFF_HEAP => "OFF_HEAP"
- case _ => throw new IllegalArgumentException(s"Invalid StorageLevel: $level")
- }
+ override def convertStorageLevelToString(level: StorageLevel): String
}
diff --git a/hudi-spark-datasource/hudi-spark3.0.x/pom.xml b/hudi-spark-datasource/hudi-spark3.0.x/pom.xml
new file mode 100644
index 0000000000000..432b37a2d1ecd
--- /dev/null
+++ b/hudi-spark-datasource/hudi-spark3.0.x/pom.xml
@@ -0,0 +1,289 @@
+
+
+
+
+ hudi-spark-datasource
+ org.apache.hudi
+ 0.14.0-SNAPSHOT
+
+ 4.0.0
+
+ hudi-spark3.0.x_2.12
+ 0.14.0-SNAPSHOT
+
+ hudi-spark3.0.x_2.12
+ jar
+
+
+ ${project.parent.parent.basedir}
+
+
+
+
+
+ src/main/resources
+
+
+
+
+
+ net.alchim31.maven
+ scala-maven-plugin
+ ${scala-maven-plugin.version}
+
+
+ -nobootcp
+ -target:jvm-1.8
+
+ 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
+
+
+
+
+
+ org.apache.maven.plugins
+ maven-compiler-plugin
+
+
+ compile
+
+ compile
+
+
+
+
+
+ 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
+ ${spark30.version}
+
+
+ slf4j-log4j12
+ org.slf4j
+
+
+ log4j
+ log4j
+
+
+ true
+
+
+
+ com.fasterxml.jackson.core
+ jackson-databind
+ ${fasterxml.spark3.version}
+
+
+ com.fasterxml.jackson.core
+ jackson-annotations
+ ${fasterxml.spark3.version}
+
+
+ com.fasterxml.jackson.core
+ jackson-core
+ ${fasterxml.spark3.version}
+
+
+
+ org.apache.hudi
+ hudi-spark-client
+ ${project.version}
+
+
+ org.apache.spark
+ *
+
+
+
+
+
+ org.apache.hudi
+ hudi-spark-common_${scala.binary.version}
+ ${project.version}
+
+
+ org.apache.spark
+ *
+
+
+
+
+
+ org.apache.hudi
+ hudi-spark3-common
+ ${project.version}
+
+
+ org.apache.spark
+ *
+
+
+
+
+
+
+ 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.spark
+ *
+
+
+
+
+
+ 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.apache.spark
+ *
+
+
+
+
+
+ org.junit.jupiter
+ junit-jupiter-api
+ test
+
+
+ org.junit.jupiter
+ junit-jupiter-params
+ test
+
+
+
+
diff --git a/hudi-spark-datasource/hudi-spark3.0.x/src/main/java/org/apache/spark/sql/execution/datasources/parquet/Spark30HoodieVectorizedParquetRecordReader.java b/hudi-spark-datasource/hudi-spark3.0.x/src/main/java/org/apache/spark/sql/execution/datasources/parquet/Spark30HoodieVectorizedParquetRecordReader.java
new file mode 100644
index 0000000000000..e1607373c1ea6
--- /dev/null
+++ b/hudi-spark-datasource/hudi-spark3.0.x/src/main/java/org/apache/spark/sql/execution/datasources/parquet/Spark30HoodieVectorizedParquetRecordReader.java
@@ -0,0 +1,187 @@
+/*
+ * 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 Spark30HoodieVectorizedParquetRecordReader 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 Spark30HoodieVectorizedParquetRecordReader(
+ ZoneId convertTz,
+ String datetimeRebaseMode,
+ boolean useOffHeap,
+ int capacity,
+ Map> typeChangeInfos) {
+ super(convertTz, datetimeRebaseMode, 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.0.x/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister b/hudi-spark-datasource/hudi-spark3.0.x/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
new file mode 100644
index 0000000000000..4043b10b3b7d1
--- /dev/null
+++ b/hudi-spark-datasource/hudi-spark3.0.x/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
@@ -0,0 +1,19 @@
+
+# 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.
+
+
+org.apache.hudi.Spark30DefaultSource
\ No newline at end of file
diff --git a/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/hudi/Spark30DefaultSource.scala b/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/hudi/Spark30DefaultSource.scala
new file mode 100644
index 0000000000000..d1a2766e487ae
--- /dev/null
+++ b/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/hudi/Spark30DefaultSource.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.hudi
+
+import org.apache.spark.sql.sources.DataSourceRegister
+
+class Spark30DefaultSource extends DefaultSource with DataSourceRegister {
+ override def shortName(): String = "hudi"
+}
diff --git a/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/hudi/Spark30HoodieFileScanRDD.scala b/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/hudi/Spark30HoodieFileScanRDD.scala
new file mode 100644
index 0000000000000..c9700541e6415
--- /dev/null
+++ b/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/hudi/Spark30HoodieFileScanRDD.scala
@@ -0,0 +1,35 @@
+/*
+ * 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.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 Spark30HoodieFileScanRDD(@transient private val sparkSession: SparkSession,
+ read: PartitionedFile => Iterator[InternalRow],
+ @transient filePartitions: Seq[FilePartition])
+ extends FileScanRDD(sparkSession, read, filePartitions)
+ with HoodieUnsafeRDD {
+
+ override final def collect(): Array[InternalRow] = super[HoodieUnsafeRDD].collect()
+}
diff --git a/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/HoodieSpark30CatalogUtils.scala b/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/HoodieSpark30CatalogUtils.scala
new file mode 100644
index 0000000000000..f8863f52297c9
--- /dev/null
+++ b/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/HoodieSpark30CatalogUtils.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 HoodieSpark30CatalogUtils 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.0.x/src/main/scala/org/apache/spark/sql/HoodieSpark30CatalystExpressionUtils.scala b/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/HoodieSpark30CatalystExpressionUtils.scala
new file mode 100644
index 0000000000000..ef3e8fdb6d16b
--- /dev/null
+++ b/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/HoodieSpark30CatalystExpressionUtils.scala
@@ -0,0 +1,152 @@
+/*
+ * 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.HoodieSparkTypeUtils.isCastPreservingOrdering
+import org.apache.spark.sql.catalyst.expressions.{AnsiCast, Attribute, AttributeReference, AttributeSet, BitwiseOr, Cast, DateAdd, DateDiff, DateFormatClass, DateSub, Divide, Exp, Expm1, Expression, FromUTCTimestamp, FromUnixTime, Log, Log10, Log1p, Log2, Lower, Multiply, ParseToDate, ParseToTimestamp, PredicateHelper, ShiftLeft, ShiftRight, ToUTCTimestamp, ToUnixTimestamp, Upper}
+import org.apache.spark.sql.types.DataType
+
+object HoodieSpark30CatalystExpressionUtils extends HoodieSpark3CatalystExpressionUtils {
+
+ override def matchCast(expr: Expression): Option[(Expression, DataType, Option[String])] =
+ expr match {
+ case Cast(child, dataType, timeZoneId) => Some((child, dataType, timeZoneId))
+ case _ => None
+ }
+
+ override def normalizeExprs(exprs: Seq[Expression], attributes: Seq[Attribute]): Seq[Expression] = {
+ exprs.map {
+ _.transform {
+ case a: AttributeReference =>
+ a.withName(attributes.find(_.semanticEquals(a)).getOrElse(a).name)
+ }
+ }
+ }
+
+ // NOTE: This method has been borrowed from Spark 3.1
+ override def extractPredicatesWithinOutputSet(condition: Expression,
+ outputSet: AttributeSet): Option[Expression] = condition match {
+ case org.apache.spark.sql.catalyst.expressions.And(left, right) =>
+ val leftResultOptional = extractPredicatesWithinOutputSet(left, outputSet)
+ val rightResultOptional = extractPredicatesWithinOutputSet(right, outputSet)
+ (leftResultOptional, rightResultOptional) match {
+ case (Some(leftResult), Some(rightResult)) => Some(org.apache.spark.sql.catalyst.expressions.And(leftResult, rightResult))
+ case (Some(leftResult), None) => Some(leftResult)
+ case (None, Some(rightResult)) => Some(rightResult)
+ case _ => None
+ }
+
+ // The Or predicate is convertible when both of its children can be pushed down.
+ // That is to say, if one/both of the children can be partially pushed down, the Or
+ // predicate can be partially pushed down as well.
+ //
+ // Here is an example used to explain the reason.
+ // Let's say we have
+ // condition: (a1 AND a2) OR (b1 AND b2),
+ // outputSet: AttributeSet(a1, b1)
+ // a1 and b1 is convertible, while a2 and b2 is not.
+ // The predicate can be converted as
+ // (a1 OR b1) AND (a1 OR b2) AND (a2 OR b1) AND (a2 OR b2)
+ // As per the logical in And predicate, we can push down (a1 OR b1).
+ case org.apache.spark.sql.catalyst.expressions.Or(left, right) =>
+ for {
+ lhs <- extractPredicatesWithinOutputSet(left, outputSet)
+ rhs <- extractPredicatesWithinOutputSet(right, outputSet)
+ } yield org.apache.spark.sql.catalyst.expressions.Or(lhs, rhs)
+
+ // Here we assume all the `Not` operators is already below all the `And` and `Or` operators
+ // after the optimization rule `BooleanSimplification`, so that we don't need to handle the
+ // `Not` operators here.
+ case other =>
+ if (other.references.subsetOf(outputSet)) {
+ Some(other)
+ } else {
+ None
+ }
+ }
+
+ override def tryMatchAttributeOrderingPreservingTransformation(expr: Expression): Option[AttributeReference] = {
+ expr match {
+ case OrderPreservingTransformation(attrRef) => Some(attrRef)
+ case _ => None
+ }
+ }
+
+ def canUpCast(fromType: DataType, toType: DataType): Boolean =
+ Cast.canUpCast(fromType, toType)
+
+ override def unapplyCastExpression(expr: Expression): Option[(Expression, DataType, Option[String], Boolean)] =
+ expr match {
+ case Cast(castedExpr, dataType, timeZoneId) => Some((castedExpr, dataType, timeZoneId, false))
+ case AnsiCast(castedExpr, dataType, timeZoneId) => Some((castedExpr, dataType, timeZoneId, true))
+ case _ => None
+ }
+
+ private object OrderPreservingTransformation {
+ def unapply(expr: Expression): Option[AttributeReference] = {
+ expr match {
+ // Date/Time Expressions
+ case DateFormatClass(OrderPreservingTransformation(attrRef), _, _) => Some(attrRef)
+ case DateAdd(OrderPreservingTransformation(attrRef), _) => Some(attrRef)
+ case DateSub(OrderPreservingTransformation(attrRef), _) => Some(attrRef)
+ case DateDiff(OrderPreservingTransformation(attrRef), _) => Some(attrRef)
+ case DateDiff(_, OrderPreservingTransformation(attrRef)) => Some(attrRef)
+ case FromUnixTime(OrderPreservingTransformation(attrRef), _, _) => Some(attrRef)
+ case FromUTCTimestamp(OrderPreservingTransformation(attrRef), _) => Some(attrRef)
+ case ParseToDate(OrderPreservingTransformation(attrRef), _, _) => Some(attrRef)
+ case ParseToTimestamp(OrderPreservingTransformation(attrRef), _, _) => Some(attrRef)
+ case ToUnixTimestamp(OrderPreservingTransformation(attrRef), _, _) => Some(attrRef)
+ case ToUTCTimestamp(OrderPreservingTransformation(attrRef), _) => Some(attrRef)
+
+ // String Expressions
+ case Lower(OrderPreservingTransformation(attrRef)) => Some(attrRef)
+ case Upper(OrderPreservingTransformation(attrRef)) => Some(attrRef)
+ case org.apache.spark.sql.catalyst.expressions.Left(OrderPreservingTransformation(attrRef), _, _) => Some(attrRef)
+
+ // Math Expressions
+ // Binary
+ case org.apache.spark.sql.catalyst.expressions.Add(OrderPreservingTransformation(attrRef), _) => Some(attrRef)
+ case org.apache.spark.sql.catalyst.expressions.Add(_, OrderPreservingTransformation(attrRef)) => Some(attrRef)
+ case Multiply(OrderPreservingTransformation(attrRef), _) => Some(attrRef)
+ case Multiply(_, OrderPreservingTransformation(attrRef)) => Some(attrRef)
+ case Divide(OrderPreservingTransformation(attrRef), _) => Some(attrRef)
+ case BitwiseOr(OrderPreservingTransformation(attrRef), _) => Some(attrRef)
+ case BitwiseOr(_, OrderPreservingTransformation(attrRef)) => Some(attrRef)
+ // Unary
+ case Exp(OrderPreservingTransformation(attrRef)) => Some(attrRef)
+ case Expm1(OrderPreservingTransformation(attrRef)) => Some(attrRef)
+ case Log(OrderPreservingTransformation(attrRef)) => Some(attrRef)
+ case Log10(OrderPreservingTransformation(attrRef)) => Some(attrRef)
+ case Log1p(OrderPreservingTransformation(attrRef)) => Some(attrRef)
+ case Log2(OrderPreservingTransformation(attrRef)) => Some(attrRef)
+ case ShiftLeft(OrderPreservingTransformation(attrRef), _) => Some(attrRef)
+ case ShiftRight(OrderPreservingTransformation(attrRef), _) => Some(attrRef)
+
+ // Other
+ case cast @ Cast(OrderPreservingTransformation(attrRef), _, _)
+ if isCastPreservingOrdering(cast.child.dataType, cast.dataType) => Some(attrRef)
+
+ // Identity transformation
+ case attrRef: AttributeReference => Some(attrRef)
+ // No match
+ case _ => None
+ }
+ }
+ }
+}
diff --git a/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/HoodieSpark30CatalystPlanUtils.scala b/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/HoodieSpark30CatalystPlanUtils.scala
new file mode 100644
index 0000000000000..c24f5b0d4d9ca
--- /dev/null
+++ b/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/HoodieSpark30CatalystPlanUtils.scala
@@ -0,0 +1,52 @@
+/*
+ * 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.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.analysis.ResolvedTable
+import org.apache.spark.sql.catalyst.expressions.{AttributeSet, ProjectionOverSchema}
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.connector.catalog.{Identifier, Table, TableCatalog}
+import org.apache.spark.sql.execution.command.AlterTableRecoverPartitionsCommand
+import org.apache.spark.sql.types.StructType
+
+object HoodieSpark30CatalystPlanUtils extends HoodieSpark3CatalystPlanUtils {
+
+ def unapplyResolvedTable(plan: LogicalPlan): Option[(TableCatalog, Identifier, Table)] =
+ plan match {
+ case ResolvedTable(catalog, identifier, table) => Some((catalog, identifier, table))
+ case _ => None
+ }
+
+ override def projectOverSchema(schema: StructType, output: AttributeSet): ProjectionOverSchema = ProjectionOverSchema(schema)
+
+ override def isRepairTable(plan: LogicalPlan): Boolean = {
+ plan.isInstanceOf[AlterTableRecoverPartitionsCommand]
+ }
+
+ override def getRepairTableChildren(plan: LogicalPlan): Option[(TableIdentifier, Boolean, Boolean, String)] = {
+ plan match {
+ // For Spark >= 3.2.x, AlterTableRecoverPartitionsCommand was renamed RepairTableCommand, and added two new
+ // parameters: enableAddPartitions and enableDropPartitions. By setting them to true and false, can restore
+ // AlterTableRecoverPartitionsCommand's behavior
+ case c: AlterTableRecoverPartitionsCommand =>
+ Some((c.tableName, true, false, c.cmd))
+ }
+ }
+}
diff --git a/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_0Adapter.scala b/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_0Adapter.scala
new file mode 100644
index 0000000000000..b1a0c7dac0388
--- /dev/null
+++ b/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_0Adapter.scala
@@ -0,0 +1,117 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.adapter
+
+import org.apache.avro.Schema
+import org.apache.hudi.Spark30HoodieFileScanRDD
+import org.apache.spark.sql.avro.{HoodieAvroDeserializer, HoodieAvroSerializer, HoodieSpark3_0AvroDeserializer, HoodieSpark3_0AvroSerializer}
+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._
+import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Spark30HoodieParquetFileFormat}
+import org.apache.spark.sql.execution.datasources.{FilePartition, FileScanRDD, PartitionedFile}
+import org.apache.spark.sql.hudi.SparkAdapter
+import org.apache.spark.sql.parser.{HoodieExtendedParserInterface, HoodieSpark3_0ExtendedSqlParser}
+import org.apache.spark.sql.types.{DataType, Metadata, MetadataBuilder, StructType}
+import org.apache.spark.sql.vectorized.ColumnarUtils
+import org.apache.spark.sql._
+import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases
+import org.apache.spark.sql.catalyst.catalog.CatalogTable
+import org.apache.spark.sql.catalyst.planning.PhysicalOperation
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.storage.StorageLevel._
+
+/**
+ * Implementation of [[SparkAdapter]] for Spark 3.0.x
+ */
+class Spark3_0Adapter extends BaseSpark3Adapter {
+
+ override def resolveHoodieTable(plan: LogicalPlan): Option[CatalogTable] = {
+ super.resolveHoodieTable(plan).orElse {
+ EliminateSubqueryAliases(plan) match {
+ // First, we need to weed out unresolved plans
+ case plan if !plan.resolved => None
+ // NOTE: When resolving Hudi table we allow [[Filter]]s and [[Project]]s be applied
+ // on top of it
+ case PhysicalOperation(_, _, DataSourceV2Relation(table: CatalogTable, _, _, _, _)) if isHoodieTable(table) =>
+ Some(table)
+ case _ => None
+ }
+ }
+ }
+
+ override def isColumnarBatchRow(r: InternalRow): Boolean = ColumnarUtils.isColumnarBatchRow(r)
+
+ def createCatalystMetadataForMetaField: Metadata =
+ // NOTE: Since [[METADATA_COL_ATTR_KEY]] flag is not available in Spark 2.x,
+ // we simply produce an empty [[Metadata]] instance
+ new MetadataBuilder().build()
+
+ override def getCatalogUtils: HoodieSpark3CatalogUtils = HoodieSpark30CatalogUtils
+
+ override def getCatalystPlanUtils: HoodieCatalystPlansUtils = HoodieSpark30CatalystPlanUtils
+
+ override def getCatalystExpressionUtils: HoodieCatalystExpressionUtils = HoodieSpark30CatalystExpressionUtils
+
+ override def createAvroSerializer(rootCatalystType: DataType, rootAvroType: Schema, nullable: Boolean): HoodieAvroSerializer =
+ new HoodieSpark3_0AvroSerializer(rootCatalystType, rootAvroType, nullable)
+
+ override def createAvroDeserializer(rootAvroType: Schema, rootCatalystType: DataType): HoodieAvroDeserializer =
+ new HoodieSpark3_0AvroDeserializer(rootAvroType, rootCatalystType)
+
+ override def createExtendedSparkParser(spark: SparkSession, delegate: ParserInterface): HoodieExtendedParserInterface =
+ new HoodieSpark3_0ExtendedSqlParser(spark, delegate)
+
+ override def createHoodieParquetFileFormat(appendPartitionValues: Boolean): Option[ParquetFileFormat] = {
+ Some(new Spark30HoodieParquetFileFormat(appendPartitionValues))
+ }
+
+ override def createHoodieFileScanRDD(sparkSession: SparkSession,
+ readFunction: PartitionedFile => Iterator[InternalRow],
+ filePartitions: Seq[FilePartition],
+ readDataSchema: StructType,
+ metadataColumns: Seq[AttributeReference] = Seq.empty): FileScanRDD = {
+ new Spark30HoodieFileScanRDD(sparkSession, readFunction, filePartitions)
+ }
+
+ override def extractDeleteCondition(deleteFromTable: Command): Expression = {
+ deleteFromTable.asInstanceOf[DeleteFromTable].condition.getOrElse(null)
+ }
+
+ /**
+ * Converts instance of [[StorageLevel]] to a corresponding string
+ */
+ override def convertStorageLevelToString(level: StorageLevel): String = level match {
+ case NONE => "NONE"
+ case DISK_ONLY => "DISK_ONLY"
+ case DISK_ONLY_2 => "DISK_ONLY_2"
+ case MEMORY_ONLY => "MEMORY_ONLY"
+ case MEMORY_ONLY_2 => "MEMORY_ONLY_2"
+ case MEMORY_ONLY_SER => "MEMORY_ONLY_SER"
+ case MEMORY_ONLY_SER_2 => "MEMORY_ONLY_SER_2"
+ case MEMORY_AND_DISK => "MEMORY_AND_DISK"
+ case MEMORY_AND_DISK_2 => "MEMORY_AND_DISK_2"
+ case MEMORY_AND_DISK_SER => "MEMORY_AND_DISK_SER"
+ case MEMORY_AND_DISK_SER_2 => "MEMORY_AND_DISK_SER_2"
+ case OFF_HEAP => "OFF_HEAP"
+ case _ => throw new IllegalArgumentException(s"Invalid StorageLevel: $level")
+ }
+}
diff --git a/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/avro/AvroDeserializer.scala b/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/avro/AvroDeserializer.scala
new file mode 100644
index 0000000000000..3433f3a408514
--- /dev/null
+++ b/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/avro/AvroDeserializer.scala
@@ -0,0 +1,492 @@
+/*
+ * 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.avro
+
+import org.apache.avro.Conversions.DecimalConversion
+import org.apache.avro.LogicalTypes.{TimestampMicros, TimestampMillis}
+import org.apache.avro.Schema.Type._
+import org.apache.avro.generic._
+import org.apache.avro.util.Utf8
+import org.apache.avro.{LogicalTypes, Schema, SchemaBuilder}
+import org.apache.spark.sql.avro.AvroDeserializer.{createDateRebaseFuncInRead, createTimestampRebaseFuncInRead}
+import org.apache.spark.sql.catalyst.expressions.{SpecificInternalRow, UnsafeArrayData}
+import org.apache.spark.sql.catalyst.util.DateTimeConstants.MILLIS_PER_DAY
+import org.apache.spark.sql.catalyst.util._
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.execution.datasources.DataSourceUtils
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.internal.SQLConf.LegacyBehaviorPolicy
+import org.apache.spark.sql.types._
+import org.apache.spark.unsafe.types.UTF8String
+
+import java.math.BigDecimal
+import java.nio.ByteBuffer
+import scala.collection.JavaConverters._
+import scala.collection.mutable.ArrayBuffer
+
+/**
+ * A deserializer to deserialize data in avro format to data in catalyst format.
+ *
+ * NOTE: This code is borrowed from Spark 3.1.2
+ * This code is borrowed, so that we can better control compatibility w/in Spark minor
+ * branches (3.2.x, 3.1.x, etc)
+ *
+ * PLEASE REFRAIN MAKING ANY CHANGES TO THIS CODE UNLESS ABSOLUTELY NECESSARY
+ */
+private[sql] class AvroDeserializer(rootAvroType: Schema,
+ rootCatalystType: DataType,
+ datetimeRebaseMode: LegacyBehaviorPolicy.Value) {
+
+ def this(rootAvroType: Schema, rootCatalystType: DataType) = {
+ this(
+ rootAvroType,
+ rootCatalystType,
+ LegacyBehaviorPolicy.withName(SQLConf.get.getConf(SQLConf.LEGACY_AVRO_REBASE_MODE_IN_READ)))
+ }
+
+ private lazy val decimalConversions = new DecimalConversion()
+
+ private val dateRebaseFunc = createDateRebaseFuncInRead(
+ datetimeRebaseMode, "Avro")
+
+ private val timestampRebaseFunc = createTimestampRebaseFuncInRead(
+ datetimeRebaseMode, "Avro")
+
+ private val converter: Any => Option[Any] = rootCatalystType match {
+ // A shortcut for empty schema.
+ case st: StructType if st.isEmpty =>
+ (data: Any) => Some(InternalRow.empty)
+
+ case st: StructType =>
+ val resultRow = new SpecificInternalRow(st.map(_.dataType))
+ val fieldUpdater = new RowUpdater(resultRow)
+ val writer = getRecordWriter(rootAvroType, st, Nil, applyFilters = _ => false)
+ (data: Any) => {
+ val record = data.asInstanceOf[GenericRecord]
+ val skipRow = writer(fieldUpdater, record)
+ if (skipRow) None else Some(resultRow)
+ }
+
+ case _ =>
+ val tmpRow = new SpecificInternalRow(Seq(rootCatalystType))
+ val fieldUpdater = new RowUpdater(tmpRow)
+ val writer = newWriter(rootAvroType, rootCatalystType, Nil)
+ (data: Any) => {
+ writer(fieldUpdater, 0, data)
+ Some(tmpRow.get(0, rootCatalystType))
+ }
+ }
+
+ def deserialize(data: Any): Option[Any] = converter(data)
+
+ /**
+ * Creates a writer to write avro values to Catalyst values at the given ordinal with the given
+ * updater.
+ */
+ private def newWriter(avroType: Schema,
+ catalystType: DataType,
+ path: List[String]): (CatalystDataUpdater, Int, Any) => Unit =
+ (avroType.getType, catalystType) match {
+ case (NULL, NullType) => (updater, ordinal, _) =>
+ updater.setNullAt(ordinal)
+
+ // TODO: we can avoid boxing if future version of avro provide primitive accessors.
+ case (BOOLEAN, BooleanType) => (updater, ordinal, value) =>
+ updater.setBoolean(ordinal, value.asInstanceOf[Boolean])
+
+ case (INT, IntegerType) => (updater, ordinal, value) =>
+ updater.setInt(ordinal, value.asInstanceOf[Int])
+
+ case (INT, DateType) => (updater, ordinal, value) =>
+ updater.setInt(ordinal, dateRebaseFunc(value.asInstanceOf[Int]))
+
+ case (LONG, LongType) => (updater, ordinal, value) =>
+ updater.setLong(ordinal, value.asInstanceOf[Long])
+
+ case (LONG, TimestampType) => avroType.getLogicalType match {
+ // For backward compatibility, if the Avro type is Long and it is not logical type
+ // (the `null` case), the value is processed as timestamp type with millisecond precision.
+ case null | _: TimestampMillis => (updater, ordinal, value) =>
+ val millis = value.asInstanceOf[Long]
+ val micros = AvroUtils.millisToMicros(millis)
+ updater.setLong(ordinal, timestampRebaseFunc(micros))
+ case _: TimestampMicros => (updater, ordinal, value) =>
+ val micros = value.asInstanceOf[Long]
+ updater.setLong(ordinal, timestampRebaseFunc(micros))
+ case other => throw new IncompatibleSchemaException(
+ s"Cannot convert Avro logical type ${other} to Catalyst Timestamp type.")
+ }
+
+ // Before we upgrade Avro to 1.8 for logical type support, spark-avro converts Long to Date.
+ // For backward compatibility, we still keep this conversion.
+ case (LONG, DateType) => (updater, ordinal, value) =>
+ updater.setInt(ordinal, (value.asInstanceOf[Long] / MILLIS_PER_DAY).toInt)
+
+ case (FLOAT, FloatType) => (updater, ordinal, value) =>
+ updater.setFloat(ordinal, value.asInstanceOf[Float])
+
+ case (DOUBLE, DoubleType) => (updater, ordinal, value) =>
+ updater.setDouble(ordinal, value.asInstanceOf[Double])
+
+ case (STRING, StringType) => (updater, ordinal, value) =>
+ val str = value match {
+ case s: String => UTF8String.fromString(s)
+ case s: Utf8 =>
+ val bytes = new Array[Byte](s.getByteLength)
+ System.arraycopy(s.getBytes, 0, bytes, 0, s.getByteLength)
+ UTF8String.fromBytes(bytes)
+ }
+ updater.set(ordinal, str)
+
+ case (ENUM, StringType) => (updater, ordinal, value) =>
+ updater.set(ordinal, UTF8String.fromString(value.toString))
+
+ case (FIXED, BinaryType) => (updater, ordinal, value) =>
+ updater.set(ordinal, value.asInstanceOf[GenericFixed].bytes().clone())
+
+ case (BYTES, BinaryType) => (updater, ordinal, value) =>
+ val bytes = value match {
+ case b: ByteBuffer =>
+ val bytes = new Array[Byte](b.remaining)
+ b.get(bytes)
+ // Do not forget to reset the position
+ b.rewind()
+ bytes
+ case b: Array[Byte] => b
+ case other => throw new RuntimeException(s"$other is not a valid avro binary.")
+ }
+ updater.set(ordinal, bytes)
+
+ case (FIXED, _: DecimalType) => (updater, ordinal, value) =>
+ val d = avroType.getLogicalType.asInstanceOf[LogicalTypes.Decimal]
+ val bigDecimal = decimalConversions.fromFixed(value.asInstanceOf[GenericFixed], avroType, d)
+ val decimal = createDecimal(bigDecimal, d.getPrecision, d.getScale)
+ updater.setDecimal(ordinal, decimal)
+
+ case (BYTES, _: DecimalType) => (updater, ordinal, value) =>
+ val d = avroType.getLogicalType.asInstanceOf[LogicalTypes.Decimal]
+ val bigDecimal = decimalConversions.fromBytes(value.asInstanceOf[ByteBuffer], avroType, d)
+ val decimal = createDecimal(bigDecimal, d.getPrecision, d.getScale)
+ updater.setDecimal(ordinal, decimal)
+
+ case (RECORD, st: StructType) =>
+ // Avro datasource doesn't accept filters with nested attributes. See SPARK-32328.
+ // We can always return `false` from `applyFilters` for nested records.
+ val writeRecord = getRecordWriter(avroType, st, path, applyFilters = _ => false)
+ (updater, ordinal, value) =>
+ val row = new SpecificInternalRow(st)
+ writeRecord(new RowUpdater(row), value.asInstanceOf[GenericRecord])
+ updater.set(ordinal, row)
+
+ case (ARRAY, ArrayType(elementType, containsNull)) =>
+ val elementWriter = newWriter(avroType.getElementType, elementType, path)
+ (updater, ordinal, value) =>
+ val collection = value.asInstanceOf[java.util.Collection[Any]]
+ val result = createArrayData(elementType, collection.size())
+ val elementUpdater = new ArrayDataUpdater(result)
+
+ var i = 0
+ val iter = collection.iterator()
+ while (iter.hasNext) {
+ val element = iter.next()
+ if (element == null) {
+ if (!containsNull) {
+ throw new RuntimeException(s"Array value at path ${path.mkString(".")} is not " +
+ "allowed to be null")
+ } else {
+ elementUpdater.setNullAt(i)
+ }
+ } else {
+ elementWriter(elementUpdater, i, element)
+ }
+ i += 1
+ }
+
+ updater.set(ordinal, result)
+
+ case (MAP, MapType(keyType, valueType, valueContainsNull)) if keyType == StringType =>
+ val keyWriter = newWriter(SchemaBuilder.builder().stringType(), StringType, path)
+ val valueWriter = newWriter(avroType.getValueType, valueType, path)
+ (updater, ordinal, value) =>
+ val map = value.asInstanceOf[java.util.Map[AnyRef, AnyRef]]
+ val keyArray = createArrayData(keyType, map.size())
+ val keyUpdater = new ArrayDataUpdater(keyArray)
+ val valueArray = createArrayData(valueType, map.size())
+ val valueUpdater = new ArrayDataUpdater(valueArray)
+ val iter = map.entrySet().iterator()
+ var i = 0
+ while (iter.hasNext) {
+ val entry = iter.next()
+ assert(entry.getKey != null)
+ keyWriter(keyUpdater, i, entry.getKey)
+ if (entry.getValue == null) {
+ if (!valueContainsNull) {
+ throw new RuntimeException(s"Map value at path ${path.mkString(".")} is not " +
+ "allowed to be null")
+ } else {
+ valueUpdater.setNullAt(i)
+ }
+ } else {
+ valueWriter(valueUpdater, i, entry.getValue)
+ }
+ i += 1
+ }
+
+ // The Avro map will never have null or duplicated map keys, it's safe to create a
+ // ArrayBasedMapData directly here.
+ updater.set(ordinal, new ArrayBasedMapData(keyArray, valueArray))
+
+ case (UNION, _) =>
+ val allTypes = avroType.getTypes.asScala
+ val nonNullTypes = allTypes.filter(_.getType != NULL)
+ val nonNullAvroType = Schema.createUnion(nonNullTypes.asJava)
+ if (nonNullTypes.nonEmpty) {
+ if (nonNullTypes.length == 1) {
+ newWriter(nonNullTypes.head, catalystType, path)
+ } else {
+ nonNullTypes.map(_.getType).toSeq match {
+ case Seq(a, b) if Set(a, b) == Set(INT, LONG) && catalystType == LongType =>
+ (updater, ordinal, value) =>
+ value match {
+ case null => updater.setNullAt(ordinal)
+ case l: java.lang.Long => updater.setLong(ordinal, l)
+ case i: java.lang.Integer => updater.setLong(ordinal, i.longValue())
+ }
+
+ case Seq(a, b) if Set(a, b) == Set(FLOAT, DOUBLE) && catalystType == DoubleType =>
+ (updater, ordinal, value) =>
+ value match {
+ case null => updater.setNullAt(ordinal)
+ case d: java.lang.Double => updater.setDouble(ordinal, d)
+ case f: java.lang.Float => updater.setDouble(ordinal, f.doubleValue())
+ }
+
+ case _ =>
+ catalystType match {
+ case st: StructType if st.length == nonNullTypes.size =>
+ val fieldWriters = nonNullTypes.zip(st.fields).map {
+ case (schema, field) => newWriter(schema, field.dataType, path :+ field.name)
+ }.toArray
+ (updater, ordinal, value) => {
+ val row = new SpecificInternalRow(st)
+ val fieldUpdater = new RowUpdater(row)
+ val i = GenericData.get().resolveUnion(nonNullAvroType, value)
+ fieldWriters(i)(fieldUpdater, i, value)
+ updater.set(ordinal, row)
+ }
+
+ case _ =>
+ throw new IncompatibleSchemaException(
+ s"Cannot convert Avro to catalyst because schema at path " +
+ s"${path.mkString(".")} is not compatible " +
+ s"(avroType = $avroType, sqlType = $catalystType).\n" +
+ s"Source Avro schema: $rootAvroType.\n" +
+ s"Target Catalyst type: $rootCatalystType")
+ }
+ }
+ }
+ } else {
+ (updater, ordinal, value) => updater.setNullAt(ordinal)
+ }
+
+ case _ =>
+ throw new IncompatibleSchemaException(
+ s"Cannot convert Avro to catalyst because schema at path ${path.mkString(".")} " +
+ s"is not compatible (avroType = $avroType, sqlType = $catalystType).\n" +
+ s"Source Avro schema: $rootAvroType.\n" +
+ s"Target Catalyst type: $rootCatalystType")
+ }
+
+ // TODO: move the following method in Decimal object on creating Decimal from BigDecimal?
+ private def createDecimal(decimal: BigDecimal, precision: Int, scale: Int): Decimal = {
+ if (precision <= Decimal.MAX_LONG_DIGITS) {
+ // Constructs a `Decimal` with an unscaled `Long` value if possible.
+ Decimal(decimal.unscaledValue().longValue(), precision, scale)
+ } else {
+ // Otherwise, resorts to an unscaled `BigInteger` instead.
+ Decimal(decimal, precision, scale)
+ }
+ }
+
+ private def getRecordWriter(avroType: Schema,
+ sqlType: StructType,
+ path: List[String],
+ applyFilters: Int => Boolean): (CatalystDataUpdater, GenericRecord) => Boolean = {
+ val validFieldIndexes = ArrayBuffer.empty[Int]
+ val fieldWriters = ArrayBuffer.empty[(CatalystDataUpdater, Any) => Unit]
+
+ val avroSchemaHelper = new AvroUtils.AvroSchemaHelper(avroType)
+ val length = sqlType.length
+ var i = 0
+ while (i < length) {
+ val sqlField = sqlType.fields(i)
+ avroSchemaHelper.getFieldByName(sqlField.name) match {
+ case Some(avroField) =>
+ validFieldIndexes += avroField.pos()
+
+ val baseWriter = newWriter(avroField.schema(), sqlField.dataType, path :+ sqlField.name)
+ val ordinal = i
+ val fieldWriter = (fieldUpdater: CatalystDataUpdater, value: Any) => {
+ if (value == null) {
+ fieldUpdater.setNullAt(ordinal)
+ } else {
+ baseWriter(fieldUpdater, ordinal, value)
+ }
+ }
+ fieldWriters += fieldWriter
+ case None if !sqlField.nullable =>
+ val fieldStr = s"${path.mkString(".")}.${sqlField.name}"
+ throw new IncompatibleSchemaException(
+ s"""
+ |Cannot find non-nullable field $fieldStr in Avro schema.
+ |Source Avro schema: $rootAvroType.
+ |Target Catalyst type: $rootCatalystType.
+ """.stripMargin)
+ case _ => // nothing to do
+ }
+ i += 1
+ }
+
+ (fieldUpdater, record) => {
+ var i = 0
+ var skipRow = false
+ while (i < validFieldIndexes.length && !skipRow) {
+ fieldWriters(i)(fieldUpdater, record.get(validFieldIndexes(i)))
+ skipRow = applyFilters(i)
+ i += 1
+ }
+ skipRow
+ }
+ }
+
+ private def createArrayData(elementType: DataType, length: Int): ArrayData = elementType match {
+ case BooleanType => UnsafeArrayData.fromPrimitiveArray(new Array[Boolean](length))
+ case ByteType => UnsafeArrayData.fromPrimitiveArray(new Array[Byte](length))
+ case ShortType => UnsafeArrayData.fromPrimitiveArray(new Array[Short](length))
+ case IntegerType => UnsafeArrayData.fromPrimitiveArray(new Array[Int](length))
+ case LongType => UnsafeArrayData.fromPrimitiveArray(new Array[Long](length))
+ case FloatType => UnsafeArrayData.fromPrimitiveArray(new Array[Float](length))
+ case DoubleType => UnsafeArrayData.fromPrimitiveArray(new Array[Double](length))
+ case _ => new GenericArrayData(new Array[Any](length))
+ }
+
+ /**
+ * A base interface for updating values inside catalyst data structure like `InternalRow` and
+ * `ArrayData`.
+ */
+ sealed trait CatalystDataUpdater {
+ def set(ordinal: Int, value: Any): Unit
+
+ def setNullAt(ordinal: Int): Unit = set(ordinal, null)
+
+ def setBoolean(ordinal: Int, value: Boolean): Unit = set(ordinal, value)
+
+ def setByte(ordinal: Int, value: Byte): Unit = set(ordinal, value)
+
+ def setShort(ordinal: Int, value: Short): Unit = set(ordinal, value)
+
+ def setInt(ordinal: Int, value: Int): Unit = set(ordinal, value)
+
+ def setLong(ordinal: Int, value: Long): Unit = set(ordinal, value)
+
+ def setDouble(ordinal: Int, value: Double): Unit = set(ordinal, value)
+
+ def setFloat(ordinal: Int, value: Float): Unit = set(ordinal, value)
+
+ def setDecimal(ordinal: Int, value: Decimal): Unit = set(ordinal, value)
+ }
+
+ final class RowUpdater(row: InternalRow) extends CatalystDataUpdater {
+ override def set(ordinal: Int, value: Any): Unit = row.update(ordinal, value)
+
+ override def setNullAt(ordinal: Int): Unit = row.setNullAt(ordinal)
+
+ override def setBoolean(ordinal: Int, value: Boolean): Unit = row.setBoolean(ordinal, value)
+
+ override def setByte(ordinal: Int, value: Byte): Unit = row.setByte(ordinal, value)
+
+ override def setShort(ordinal: Int, value: Short): Unit = row.setShort(ordinal, value)
+
+ override def setInt(ordinal: Int, value: Int): Unit = row.setInt(ordinal, value)
+
+ override def setLong(ordinal: Int, value: Long): Unit = row.setLong(ordinal, value)
+
+ override def setDouble(ordinal: Int, value: Double): Unit = row.setDouble(ordinal, value)
+
+ override def setFloat(ordinal: Int, value: Float): Unit = row.setFloat(ordinal, value)
+
+ override def setDecimal(ordinal: Int, value: Decimal): Unit =
+ row.setDecimal(ordinal, value, value.precision)
+ }
+
+ final class ArrayDataUpdater(array: ArrayData) extends CatalystDataUpdater {
+ override def set(ordinal: Int, value: Any): Unit = array.update(ordinal, value)
+
+ override def setNullAt(ordinal: Int): Unit = array.setNullAt(ordinal)
+
+ override def setBoolean(ordinal: Int, value: Boolean): Unit = array.setBoolean(ordinal, value)
+
+ override def setByte(ordinal: Int, value: Byte): Unit = array.setByte(ordinal, value)
+
+ override def setShort(ordinal: Int, value: Short): Unit = array.setShort(ordinal, value)
+
+ override def setInt(ordinal: Int, value: Int): Unit = array.setInt(ordinal, value)
+
+ override def setLong(ordinal: Int, value: Long): Unit = array.setLong(ordinal, value)
+
+ override def setDouble(ordinal: Int, value: Double): Unit = array.setDouble(ordinal, value)
+
+ override def setFloat(ordinal: Int, value: Float): Unit = array.setFloat(ordinal, value)
+
+ override def setDecimal(ordinal: Int, value: Decimal): Unit = array.update(ordinal, value)
+ }
+}
+
+object AvroDeserializer {
+
+ // NOTE: Following methods have been renamed in Spark 3.1.3 [1] making [[AvroDeserializer]] implementation
+ // (which relies on it) be only compatible with the exact same version of [[DataSourceUtils]].
+ // To make sure this implementation is compatible w/ all Spark versions w/in Spark 3.1.x branch,
+ // we're preemptively cloned those methods to make sure Hudi is compatible w/ Spark 3.1.2 as well as
+ // w/ Spark >= 3.1.3
+ //
+ // [1] https://github.com/apache/spark/pull/34978
+
+ def createDateRebaseFuncInRead(rebaseMode: LegacyBehaviorPolicy.Value,
+ format: String): Int => Int = rebaseMode match {
+ case LegacyBehaviorPolicy.EXCEPTION => days: Int =>
+ if (days < RebaseDateTime.lastSwitchJulianDay) {
+ throw DataSourceUtils.newRebaseExceptionInRead(format)
+ }
+ days
+ case LegacyBehaviorPolicy.LEGACY => RebaseDateTime.rebaseJulianToGregorianDays
+ case LegacyBehaviorPolicy.CORRECTED => identity[Int]
+ }
+
+ def createTimestampRebaseFuncInRead(rebaseMode: LegacyBehaviorPolicy.Value,
+ format: String): Long => Long = rebaseMode match {
+ case LegacyBehaviorPolicy.EXCEPTION => micros: Long =>
+ if (micros < RebaseDateTime.lastSwitchJulianTs) {
+ throw DataSourceUtils.newRebaseExceptionInRead(format)
+ }
+ micros
+ case LegacyBehaviorPolicy.LEGACY => RebaseDateTime.rebaseJulianToGregorianMicros
+ case LegacyBehaviorPolicy.CORRECTED => identity[Long]
+ }
+}
+
diff --git a/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/avro/AvroSerializer.scala b/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/avro/AvroSerializer.scala
new file mode 100644
index 0000000000000..fef0e35ee7b64
--- /dev/null
+++ b/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/avro/AvroSerializer.scala
@@ -0,0 +1,390 @@
+/*
+ * 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.avro
+
+import org.apache.avro.Conversions.DecimalConversion
+import org.apache.avro.LogicalTypes.{TimestampMicros, TimestampMillis}
+import org.apache.avro.{LogicalTypes, Schema}
+import org.apache.avro.Schema.Type
+import org.apache.avro.Schema.Type._
+import org.apache.avro.generic.GenericData.{EnumSymbol, Fixed, Record}
+import org.apache.avro.util.Utf8
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.avro.AvroSerializer.{createDateRebaseFuncInWrite, createTimestampRebaseFuncInWrite}
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{SpecializedGetters, SpecificInternalRow}
+import org.apache.spark.sql.catalyst.util.RebaseDateTime
+import org.apache.spark.sql.execution.datasources.DataSourceUtils
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.internal.SQLConf.LegacyBehaviorPolicy
+import org.apache.spark.sql.types._
+
+import java.nio.ByteBuffer
+import scala.collection.JavaConverters._
+
+/**
+ * A serializer to serialize data in catalyst format to data in avro format.
+ *
+ * NOTE: This code is borrowed from Spark 3.1.2
+ * This code is borrowed, so that we can better control compatibility w/in Spark minor
+ * branches (3.2.x, 3.1.x, etc)
+ *
+ * PLEASE REFRAIN MAKING ANY CHANGES TO THIS CODE UNLESS ABSOLUTELY NECESSARY
+ */
+private[sql] class AvroSerializer(
+ rootCatalystType: DataType,
+ rootAvroType: Schema,
+ nullable: Boolean,
+ datetimeRebaseMode: LegacyBehaviorPolicy.Value) extends Logging {
+
+ def this(rootCatalystType: DataType, rootAvroType: Schema, nullable: Boolean) = {
+ this(rootCatalystType, rootAvroType, nullable,
+ LegacyBehaviorPolicy.withName(SQLConf.get.getConf(
+ SQLConf.LEGACY_AVRO_REBASE_MODE_IN_WRITE)))
+ }
+
+ def serialize(catalystData: Any): Any = {
+ converter.apply(catalystData)
+ }
+
+ private val dateRebaseFunc = createDateRebaseFuncInWrite(
+ datetimeRebaseMode, "Avro")
+
+ private val timestampRebaseFunc = createTimestampRebaseFuncInWrite(
+ datetimeRebaseMode, "Avro")
+
+ private val converter: Any => Any = {
+ val actualAvroType = resolveNullableType(rootAvroType, nullable)
+ val baseConverter = rootCatalystType match {
+ case st: StructType =>
+ newStructConverter(st, actualAvroType).asInstanceOf[Any => Any]
+ case _ =>
+ val tmpRow = new SpecificInternalRow(Seq(rootCatalystType))
+ val converter = newConverter(rootCatalystType, actualAvroType)
+ (data: Any) =>
+ tmpRow.update(0, data)
+ converter.apply(tmpRow, 0)
+ }
+ if (nullable) {
+ (data: Any) =>
+ if (data == null) {
+ null
+ } else {
+ baseConverter.apply(data)
+ }
+ } else {
+ baseConverter
+ }
+ }
+
+ private type Converter = (SpecializedGetters, Int) => Any
+
+ private lazy val decimalConversions = new DecimalConversion()
+
+ private def newConverter(catalystType: DataType, avroType: Schema): Converter = {
+ (catalystType, avroType.getType) match {
+ case (NullType, NULL) =>
+ (getter, ordinal) => null
+ case (BooleanType, BOOLEAN) =>
+ (getter, ordinal) => getter.getBoolean(ordinal)
+ case (ByteType, INT) =>
+ (getter, ordinal) => getter.getByte(ordinal).toInt
+ case (ShortType, INT) =>
+ (getter, ordinal) => getter.getShort(ordinal).toInt
+ case (IntegerType, INT) =>
+ (getter, ordinal) => getter.getInt(ordinal)
+ case (LongType, LONG) =>
+ (getter, ordinal) => getter.getLong(ordinal)
+ case (FloatType, FLOAT) =>
+ (getter, ordinal) => getter.getFloat(ordinal)
+ case (DoubleType, DOUBLE) =>
+ (getter, ordinal) => getter.getDouble(ordinal)
+ case (d: DecimalType, FIXED)
+ if avroType.getLogicalType == LogicalTypes.decimal(d.precision, d.scale) =>
+ (getter, ordinal) =>
+ val decimal = getter.getDecimal(ordinal, d.precision, d.scale)
+ decimalConversions.toFixed(decimal.toJavaBigDecimal, avroType,
+ LogicalTypes.decimal(d.precision, d.scale))
+
+ case (d: DecimalType, BYTES)
+ if avroType.getLogicalType == LogicalTypes.decimal(d.precision, d.scale) =>
+ (getter, ordinal) =>
+ val decimal = getter.getDecimal(ordinal, d.precision, d.scale)
+ decimalConversions.toBytes(decimal.toJavaBigDecimal, avroType,
+ LogicalTypes.decimal(d.precision, d.scale))
+
+ case (StringType, ENUM) =>
+ val enumSymbols: Set[String] = avroType.getEnumSymbols.asScala.toSet
+ (getter, ordinal) =>
+ val data = getter.getUTF8String(ordinal).toString
+ if (!enumSymbols.contains(data)) {
+ throw new IncompatibleSchemaException(
+ "Cannot write \"" + data + "\" since it's not defined in enum \"" +
+ enumSymbols.mkString("\", \"") + "\"")
+ }
+ new EnumSymbol(avroType, data)
+
+ case (StringType, STRING) =>
+ (getter, ordinal) => new Utf8(getter.getUTF8String(ordinal).getBytes)
+
+ case (BinaryType, FIXED) =>
+ val size = avroType.getFixedSize()
+ (getter, ordinal) =>
+ val data: Array[Byte] = getter.getBinary(ordinal)
+ if (data.length != size) {
+ throw new IncompatibleSchemaException(
+ s"Cannot write ${data.length} ${if (data.length > 1) "bytes" else "byte"} of " +
+ "binary data into FIXED Type with size of " +
+ s"$size ${if (size > 1) "bytes" else "byte"}")
+ }
+ new Fixed(avroType, data)
+
+ case (BinaryType, BYTES) =>
+ (getter, ordinal) => ByteBuffer.wrap(getter.getBinary(ordinal))
+
+ case (DateType, INT) =>
+ (getter, ordinal) => dateRebaseFunc(getter.getInt(ordinal))
+
+ case (TimestampType, LONG) => avroType.getLogicalType match {
+ // For backward compatibility, if the Avro type is Long and it is not logical type
+ // (the `null` case), output the timestamp value as with millisecond precision.
+ case null | _: TimestampMillis => (getter, ordinal) =>
+ AvroUtils.microsToMillis(timestampRebaseFunc(getter.getLong(ordinal)))
+ case _: TimestampMicros => (getter, ordinal) =>
+ timestampRebaseFunc(getter.getLong(ordinal))
+ case other => throw new IncompatibleSchemaException(
+ s"Cannot convert Catalyst Timestamp type to Avro logical type ${other}")
+ }
+
+ case (ArrayType(et, containsNull), ARRAY) =>
+ val elementConverter = newConverter(
+ et, resolveNullableType(avroType.getElementType, containsNull))
+ (getter, ordinal) => {
+ val arrayData = getter.getArray(ordinal)
+ val len = arrayData.numElements()
+ val result = new Array[Any](len)
+ var i = 0
+ while (i < len) {
+ if (containsNull && arrayData.isNullAt(i)) {
+ result(i) = null
+ } else {
+ result(i) = elementConverter(arrayData, i)
+ }
+ i += 1
+ }
+ // avro writer is expecting a Java Collection, so we convert it into
+ // `ArrayList` backed by the specified array without data copying.
+ java.util.Arrays.asList(result: _*)
+ }
+
+ case (st: StructType, RECORD) =>
+ val structConverter = newStructConverter(st, avroType)
+ val numFields = st.length
+ (getter, ordinal) => structConverter(getter.getStruct(ordinal, numFields))
+
+ case (st: StructType, UNION) =>
+ val unionConverter = newUnionConverter(st, avroType)
+ val numFields = st.length
+ (getter, ordinal) => unionConverter(getter.getStruct(ordinal, numFields))
+
+ case (MapType(kt, vt, valueContainsNull), MAP) if kt == StringType =>
+ val valueConverter = newConverter(
+ vt, resolveNullableType(avroType.getValueType, valueContainsNull))
+ (getter, ordinal) =>
+ val mapData = getter.getMap(ordinal)
+ val len = mapData.numElements()
+ val result = new java.util.HashMap[String, Any](len)
+ val keyArray = mapData.keyArray()
+ val valueArray = mapData.valueArray()
+ var i = 0
+ while (i < len) {
+ val key = keyArray.getUTF8String(i).toString
+ if (valueContainsNull && valueArray.isNullAt(i)) {
+ result.put(key, null)
+ } else {
+ result.put(key, valueConverter(valueArray, i))
+ }
+ i += 1
+ }
+ result
+
+ case other =>
+ throw new IncompatibleSchemaException(s"Cannot convert Catalyst type $catalystType to " +
+ s"Avro type $avroType.")
+ }
+ }
+
+ private def newStructConverter(catalystStruct: StructType, avroStruct: Schema): InternalRow => Record = {
+ if (avroStruct.getType != RECORD || avroStruct.getFields.size() != catalystStruct.length) {
+ throw new IncompatibleSchemaException(s"Cannot convert Catalyst type $catalystStruct to " +
+ s"Avro type $avroStruct.")
+ }
+ val avroSchemaHelper = new AvroUtils.AvroSchemaHelper(avroStruct)
+
+ val (avroIndices: Array[Int], fieldConverters: Array[Converter]) =
+ catalystStruct.map { catalystField =>
+ val avroField = avroSchemaHelper.getFieldByName(catalystField.name) match {
+ case Some(f) => f
+ case None => throw new IncompatibleSchemaException(
+ s"Cannot find ${catalystField.name} in Avro schema")
+ }
+ val converter = newConverter(catalystField.dataType, resolveNullableType(
+ avroField.schema(), catalystField.nullable))
+ (avroField.pos(), converter)
+ }.toArray.unzip
+
+ val numFields = catalystStruct.length
+ row: InternalRow =>
+ val result = new Record(avroStruct)
+ var i = 0
+ while (i < numFields) {
+ if (row.isNullAt(i)) {
+ result.put(avroIndices(i), null)
+ } else {
+ result.put(avroIndices(i), fieldConverters(i).apply(row, i))
+ }
+ i += 1
+ }
+ result
+ }
+
+ private def newUnionConverter(catalystStruct: StructType, avroUnion: Schema): InternalRow => Any = {
+ if (avroUnion.getType != UNION || !canMapUnion(catalystStruct, avroUnion)) {
+ throw new IncompatibleSchemaException(s"Cannot convert Catalyst type $catalystStruct to " +
+ s"Avro type $avroUnion.")
+ }
+ val nullable = avroUnion.getTypes.size() > 0 && avroUnion.getTypes.get(0).getType == Type.NULL
+ val avroInnerTypes = if (nullable) {
+ avroUnion.getTypes.asScala.tail
+ } else {
+ avroUnion.getTypes.asScala
+ }
+ val fieldConverters = catalystStruct.zip(avroInnerTypes).map {
+ case (f1, f2) => newConverter(f1.dataType, f2)
+ }
+ val numFields = catalystStruct.length
+ (row: InternalRow) =>
+ var i = 0
+ var result: Any = null
+ while (i < numFields) {
+ if (!row.isNullAt(i)) {
+ if (result != null) {
+ throw new IncompatibleSchemaException(s"Cannot convert Catalyst record $catalystStruct to " +
+ s"Avro union $avroUnion. Record has more than one optional values set")
+ }
+ result = fieldConverters(i).apply(row, i)
+ }
+ i += 1
+ }
+ if (!nullable && result == null) {
+ throw new IncompatibleSchemaException(s"Cannot convert Catalyst record $catalystStruct to " +
+ s"Avro union $avroUnion. Record has no values set, while should have exactly one")
+ }
+ result
+ }
+
+ private def canMapUnion(catalystStruct: StructType, avroStruct: Schema): Boolean = {
+ (avroStruct.getTypes.size() > 0 &&
+ avroStruct.getTypes.get(0).getType == Type.NULL &&
+ avroStruct.getTypes.size() - 1 == catalystStruct.length) || avroStruct.getTypes.size() == catalystStruct.length
+ }
+
+ /**
+ * Resolve a possibly nullable Avro Type.
+ *
+ * An Avro type is nullable when it is a [[UNION]] of two types: one null type and another
+ * non-null type. This method will check the nullability of the input Avro type and return the
+ * non-null type within when it is nullable. Otherwise it will return the input Avro type
+ * unchanged. It will throw an [[UnsupportedAvroTypeException]] when the input Avro type is an
+ * unsupported nullable type.
+ *
+ * It will also log a warning message if the nullability for Avro and catalyst types are
+ * different.
+ */
+ private def resolveNullableType(avroType: Schema, nullable: Boolean): Schema = {
+ val (avroNullable, resolvedAvroType) = resolveAvroType(avroType)
+ warnNullabilityDifference(avroNullable, nullable)
+ resolvedAvroType
+ }
+
+ /**
+ * Check the nullability of the input Avro type and resolve it when it is nullable. The first
+ * return value is a [[Boolean]] indicating if the input Avro type is nullable. The second
+ * return value is the possibly resolved type.
+ */
+ private def resolveAvroType(avroType: Schema): (Boolean, Schema) = {
+ if (avroType.getType == Type.UNION) {
+ val fields = avroType.getTypes.asScala
+ val actualType = fields.filter(_.getType != Type.NULL)
+ if (fields.length == 2 && actualType.length == 1) {
+ (true, actualType.head)
+ } else {
+ // This is just a normal union, not used to designate nullability
+ (false, avroType)
+ }
+ } else {
+ (false, avroType)
+ }
+ }
+
+ /**
+ * log a warning message if the nullability for Avro and catalyst types are different.
+ */
+ private def warnNullabilityDifference(avroNullable: Boolean, catalystNullable: Boolean): Unit = {
+ if (avroNullable && !catalystNullable) {
+ logWarning("Writing Avro files with nullable Avro schema and non-nullable catalyst schema.")
+ }
+ if (!avroNullable && catalystNullable) {
+ logWarning("Writing Avro files with non-nullable Avro schema and nullable catalyst " +
+ "schema will throw runtime exception if there is a record with null value.")
+ }
+ }
+}
+
+object AvroSerializer {
+
+ // NOTE: Following methods have been renamed in Spark 3.1.3 [1] making [[AvroDeserializer]] implementation
+ // (which relies on it) be only compatible with the exact same version of [[DataSourceUtils]].
+ // To make sure this implementation is compatible w/ all Spark versions w/in Spark 3.1.x branch,
+ // we're preemptively cloned those methods to make sure Hudi is compatible w/ Spark 3.1.2 as well as
+ // w/ Spark >= 3.1.3
+ //
+ // [1] https://github.com/apache/spark/pull/34978
+
+ def createDateRebaseFuncInWrite(rebaseMode: LegacyBehaviorPolicy.Value,
+ format: String): Int => Int = rebaseMode match {
+ case LegacyBehaviorPolicy.EXCEPTION => days: Int =>
+ if (days < RebaseDateTime.lastSwitchGregorianDay) {
+ throw DataSourceUtils.newRebaseExceptionInWrite(format)
+ }
+ days
+ case LegacyBehaviorPolicy.LEGACY => RebaseDateTime.rebaseGregorianToJulianDays
+ case LegacyBehaviorPolicy.CORRECTED => identity[Int]
+ }
+
+ def createTimestampRebaseFuncInWrite(rebaseMode: LegacyBehaviorPolicy.Value,
+ format: String): Long => Long = rebaseMode match {
+ case LegacyBehaviorPolicy.EXCEPTION => micros: Long =>
+ if (micros < RebaseDateTime.lastSwitchGregorianTs) {
+ throw DataSourceUtils.newRebaseExceptionInWrite(format)
+ }
+ micros
+ case LegacyBehaviorPolicy.LEGACY => RebaseDateTime.rebaseGregorianToJulianMicros
+ case LegacyBehaviorPolicy.CORRECTED => identity[Long]
+ }
+}
diff --git a/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/avro/AvroUtils.scala b/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/avro/AvroUtils.scala
new file mode 100644
index 0000000000000..77485a1143c98
--- /dev/null
+++ b/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/avro/AvroUtils.scala
@@ -0,0 +1,115 @@
+/*
+ * 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.avro
+
+import org.apache.avro.Schema
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.catalyst.util.DateTimeConstants
+import org.apache.spark.sql.internal.SQLConf
+
+import java.util.Locale
+import scala.collection.JavaConverters._
+
+/**
+ * NOTE: This code is borrowed from Spark 3.1.3
+ * This code is borrowed, so that we can better control compatibility w/in Spark minor
+ * branches (3.2.x, 3.1.x, etc)
+ *
+ * PLEASE REFRAIN MAKING ANY CHANGES TO THIS CODE UNLESS ABSOLUTELY NECESSARY
+ */
+private[avro] object AvroUtils extends Logging {
+
+ /**
+ * Wraps an Avro Schema object so that field lookups are faster.
+ *
+ * @param avroSchema The schema in which to search for fields. Must be of type RECORD.
+ */
+ class AvroSchemaHelper(avroSchema: Schema) {
+ if (avroSchema.getType != Schema.Type.RECORD) {
+ throw new IncompatibleSchemaException(
+ s"Attempting to treat ${avroSchema.getName} as a RECORD, but it was: ${avroSchema.getType}")
+ }
+
+ private[this] val fieldMap = avroSchema.getFields.asScala
+ .groupBy(_.name.toLowerCase(Locale.ROOT))
+ .mapValues(_.toSeq) // toSeq needed for scala 2.13
+
+ /**
+ * Extract a single field from the contained avro schema which has the desired field name,
+ * performing the matching with proper case sensitivity according to SQLConf.resolver.
+ *
+ * @param name The name of the field to search for.
+ * @return `Some(match)` if a matching Avro field is found, otherwise `None`.
+ */
+ def getFieldByName(name: String): Option[Schema.Field] = {
+
+ // get candidates, ignoring case of field name
+ val candidates = fieldMap.get(name.toLowerCase(Locale.ROOT))
+ .getOrElse(Seq.empty[Schema.Field])
+
+ // search candidates, taking into account case sensitivity settings
+ candidates.filter(f => SQLConf.get.resolver(f.name(), name)) match {
+ case Seq(avroField) => Some(avroField)
+ case Seq() => None
+ case matches => throw new IncompatibleSchemaException(
+ s"Searching for '$name' in Avro schema gave ${matches.size} matches. Candidates: " +
+ matches.map(_.name()).mkString("[", ", ", "]")
+ )
+ }
+ }
+ }
+
+ /**
+ * Extract a single field from `avroSchema` which has the desired field name,
+ * performing the matching with proper case sensitivity according to [[SQLConf.resolver]].
+ *
+ * @param avroSchema The schema in which to search for the field. Must be of type RECORD.
+ * @param name The name of the field to search for.
+ * @return `Some(match)` if a matching Avro field is found, otherwise `None`.
+ * @throws IncompatibleSchemaException if `avroSchema` is not a RECORD or contains multiple
+ * fields matching `name` (i.e., case-insensitive matching
+ * is used and `avroSchema` has two or more fields that have
+ * the same name with difference case).
+ */
+ private[avro] def getAvroFieldByName(
+ avroSchema: Schema,
+ name: String): Option[Schema.Field] = {
+ if (avroSchema.getType != Schema.Type.RECORD) {
+ throw new IncompatibleSchemaException(
+ s"Attempting to treat ${avroSchema.getName} as a RECORD, but it was: ${avroSchema.getType}")
+ }
+ avroSchema.getFields.asScala.filter(f => SQLConf.get.resolver(f.name(), name)).toSeq match {
+ case Seq(avroField) => Some(avroField)
+ case Seq() => None
+ case matches => throw new IncompatibleSchemaException(
+ s"Searching for '$name' in Avro schema gave ${matches.size} matches. Candidates: " +
+ matches.map(_.name()).mkString("[", ", ", "]")
+ )
+ }
+ }
+
+ def microsToMillis(micros: Long): Long = {
+ // When the timestamp is negative i.e before 1970, we need to adjust the millseconds portion.
+ // Example - 1965-01-01 10:11:12.123456 is represented as (-157700927876544) in micro precision.
+ // In millis precision the above needs to be represented as (-157700927877).
+ Math.floorDiv(micros, DateTimeConstants.MICROS_PER_MILLIS)
+ }
+
+ def millisToMicros(millis: Long): Long = {
+ Math.multiplyExact(millis, DateTimeConstants.MICROS_PER_MILLIS)
+ }
+}
diff --git a/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/avro/HoodieSpark3_0AvroDeserializer.scala b/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/avro/HoodieSpark3_0AvroDeserializer.scala
new file mode 100644
index 0000000000000..f299e306fe37b
--- /dev/null
+++ b/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/avro/HoodieSpark3_0AvroDeserializer.scala
@@ -0,0 +1,34 @@
+/*
+ * 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.avro
+
+import org.apache.avro.Schema
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.internal.SQLConf.LegacyBehaviorPolicy
+import org.apache.spark.sql.types.DataType
+
+class HoodieSpark3_0AvroDeserializer(rootAvroType: Schema, rootCatalystType: DataType)
+ extends HoodieAvroDeserializer {
+
+ private val avroDeserializer = {
+ val avroRebaseModeInRead = LegacyBehaviorPolicy.withName(SQLConf.get.getConf(SQLConf.LEGACY_AVRO_REBASE_MODE_IN_READ))
+ new AvroDeserializer(rootAvroType, rootCatalystType, avroRebaseModeInRead)
+ }
+
+ def deserialize(data: Any): Option[Any] = avroDeserializer.deserialize(data)
+}
diff --git a/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/avro/HoodieSpark3_0AvroSerializer.scala b/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/avro/HoodieSpark3_0AvroSerializer.scala
new file mode 100644
index 0000000000000..adae3e99dd4c9
--- /dev/null
+++ b/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/avro/HoodieSpark3_0AvroSerializer.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.avro
+
+import org.apache.avro.Schema
+import org.apache.spark.sql.types.DataType
+
+class HoodieSpark3_0AvroSerializer(rootCatalystType: DataType, rootAvroType: Schema, nullable: Boolean)
+ extends HoodieAvroSerializer {
+
+ val avroSerializer = new AvroSerializer(rootCatalystType, rootAvroType, nullable)
+
+ override def serialize(catalystData: Any): Any = avroSerializer.serialize(catalystData)
+}
diff --git a/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/HStatments.scala b/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/HStatments.scala
new file mode 100644
index 0000000000000..f78891fa61f5d
--- /dev/null
+++ b/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/HStatments.scala
@@ -0,0 +1,66 @@
+/*
+ * 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.connector.catalog.TableChange.ColumnPosition
+import org.apache.spark.sql.types.DataType
+
+/**
+ * ALTER TABLE ... ADD COLUMNS command, as parsed from SQL.
+ */
+case class HoodieAlterTableAddColumnsStatement(
+ tableName: Seq[String],
+ columnsToAdd: Seq[QualifiedColType]) extends ParsedStatement
+
+/**
+ * ALTER TABLE ... CHANGE COLUMN command, as parsed from SQL.
+ */
+case class HoodieAlterTableAlterColumnStatement(
+ tableName: Seq[String],
+ column: Seq[String],
+ dataType: Option[DataType],
+ nullable: Option[Boolean],
+ comment: Option[String],
+ position: Option[ColumnPosition]) extends ParsedStatement
+
+
+/**
+ * ALTER TABLE ... RENAME COLUMN command, as parsed from SQL.
+ */
+case class HoodieAlterTableRenameColumnStatement(
+ tableName: Seq[String],
+ column: Seq[String],
+ newName: String) extends ParsedStatement
+
+/**
+ * ALTER TABLE ... DROP COLUMNS command, as parsed from SQL.
+ */
+case class HoodieAlterTableDropColumnsStatement(
+ tableName: Seq[String], columnsToDrop: Seq[Seq[String]]) extends ParsedStatement
+
+/**
+ * ALTER TABLE ... SET TBLPROPERTIES command, as parsed from SQL.
+ */
+case class HoodieAlterTableSetPropertiesStatement(
+ tableName: Seq[String], properties: Map[String, String]) extends ParsedStatement
+
+/**
+ * ALTER TABLE ... UNSET TBLPROPERTIES command, as parsed from SQL.
+ */
+case class HoodieAlterTableUnsetPropertiesStatement(
+ tableName: Seq[String], propertyKeys: Seq[String], ifExists: Boolean) extends ParsedStatement
diff --git a/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/execution/datasources/Spark30NestedSchemaPruning.scala b/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/execution/datasources/Spark30NestedSchemaPruning.scala
new file mode 100644
index 0000000000000..a8c8e860b9d65
--- /dev/null
+++ b/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/execution/datasources/Spark30NestedSchemaPruning.scala
@@ -0,0 +1,272 @@
+/*
+ * 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
+
+import org.apache.hudi.{HoodieBaseRelation, SparkAdapterSupport}
+import org.apache.spark.sql.HoodieSpark3CatalystPlanUtils
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeReference, AttributeSet, Expression, NamedExpression, ProjectionOverSchema}
+import org.apache.spark.sql.catalyst.planning.PhysicalOperation
+import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, Project}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.sources.BaseRelation
+import org.apache.spark.sql.types.{ArrayType, DataType, MapType, StructField, StructType}
+
+/**
+ * Prunes unnecessary physical columns given a [[PhysicalOperation]] over a data source relation.
+ * By "physical column", we mean a column as defined in the data source format like Parquet format
+ * or ORC format. For example, in Spark SQL, a root-level Parquet column corresponds to a SQL
+ * column, and a nested Parquet column corresponds to a [[StructField]].
+ *
+ * NOTE: This class is borrowed from Spark 3.2.1, with modifications adapting it to handle [[HoodieBaseRelation]],
+ * instead of [[HadoopFsRelation]]
+ */
+class Spark30NestedSchemaPruning extends Rule[LogicalPlan] {
+ import org.apache.spark.sql.catalyst.expressions.SchemaPruning._
+ override def apply(plan: LogicalPlan): LogicalPlan =
+ if (SQLConf.get.nestedSchemaPruningEnabled) {
+ apply0(plan)
+ } else {
+ plan
+ }
+
+ private def apply0(plan: LogicalPlan): LogicalPlan =
+ plan transformDown {
+ case op @ PhysicalOperation(projects, filters,
+ // NOTE: This is modified to accommodate for Hudi's custom relations, given that original
+ // [[NestedSchemaPruning]] rule is tightly coupled w/ [[HadoopFsRelation]]
+ // TODO generalize to any file-based relation
+ l @ LogicalRelation(relation: HoodieBaseRelation, _, _, _))
+ if relation.canPruneRelationSchema =>
+
+ prunePhysicalColumns(l.output, projects, filters, relation.dataSchema,
+ prunedDataSchema => {
+ val prunedRelation =
+ relation.updatePrunedDataSchema(prunedSchema = prunedDataSchema)
+ buildPrunedRelation(l, prunedRelation)
+ }).getOrElse(op)
+ }
+
+ /**
+ * This method returns optional logical plan. `None` is returned if no nested field is required or
+ * all nested fields are required.
+ */
+ private def prunePhysicalColumns(output: Seq[AttributeReference],
+ projects: Seq[NamedExpression],
+ filters: Seq[Expression],
+ dataSchema: StructType,
+ outputRelationBuilder: StructType => LogicalRelation): Option[LogicalPlan] = {
+ val (normalizedProjects, normalizedFilters) =
+ normalizeAttributeRefNames(output, projects, filters)
+ val requestedRootFields = identifyRootFields(normalizedProjects, normalizedFilters)
+
+ // If requestedRootFields includes a nested field, continue. Otherwise,
+ // return op
+ if (requestedRootFields.exists { root: RootField => !root.derivedFromAtt }) {
+ val prunedDataSchema = pruneDataSchema(dataSchema, requestedRootFields)
+
+ // If the data schema is different from the pruned data schema, continue. Otherwise,
+ // return op. We effect this comparison by counting the number of "leaf" fields in
+ // each schemata, assuming the fields in prunedDataSchema are a subset of the fields
+ // in dataSchema.
+ if (countLeaves(dataSchema) > countLeaves(prunedDataSchema)) {
+ val planUtils = SparkAdapterSupport.sparkAdapter.getCatalystPlanUtils.asInstanceOf[HoodieSpark3CatalystPlanUtils]
+
+ val prunedRelation = outputRelationBuilder(prunedDataSchema)
+ val projectionOverSchema = planUtils.projectOverSchema(prunedDataSchema, AttributeSet(output))
+
+ Some(buildNewProjection(projects, normalizedProjects, normalizedFilters,
+ prunedRelation, projectionOverSchema))
+ } else {
+ None
+ }
+ } else {
+ None
+ }
+ }
+
+ /**
+ * Normalizes the names of the attribute references in the given projects and filters to reflect
+ * the names in the given logical relation. This makes it possible to compare attributes and
+ * fields by name. Returns a tuple with the normalized projects and filters, respectively.
+ */
+ private def normalizeAttributeRefNames(output: Seq[AttributeReference],
+ projects: Seq[NamedExpression],
+ filters: Seq[Expression]): (Seq[NamedExpression], Seq[Expression]) = {
+ val normalizedAttNameMap = output.map(att => (att.exprId, att.name)).toMap
+ val normalizedProjects = projects.map(_.transform {
+ case att: AttributeReference if normalizedAttNameMap.contains(att.exprId) =>
+ att.withName(normalizedAttNameMap(att.exprId))
+ }).map { case expr: NamedExpression => expr }
+ val normalizedFilters = filters.map(_.transform {
+ case att: AttributeReference if normalizedAttNameMap.contains(att.exprId) =>
+ att.withName(normalizedAttNameMap(att.exprId))
+ })
+ (normalizedProjects, normalizedFilters)
+ }
+
+ /**
+ * Builds the new output [[Project]] Spark SQL operator that has the `leafNode`.
+ */
+ private def buildNewProjection(projects: Seq[NamedExpression],
+ normalizedProjects: Seq[NamedExpression],
+ filters: Seq[Expression],
+ prunedRelation: LogicalRelation,
+ projectionOverSchema: ProjectionOverSchema): Project = {
+ // Construct a new target for our projection by rewriting and
+ // including the original filters where available
+ val projectionChild =
+ if (filters.nonEmpty) {
+ val projectedFilters = filters.map(_.transformDown {
+ case projectionOverSchema(expr) => expr
+ })
+ val newFilterCondition = projectedFilters.reduce(And)
+ Filter(newFilterCondition, prunedRelation)
+ } else {
+ prunedRelation
+ }
+
+ // Construct the new projections of our Project by
+ // rewriting the original projections
+ val newProjects = normalizedProjects.map(_.transformDown {
+ case projectionOverSchema(expr) => expr
+ }).map { case expr: NamedExpression => expr }
+
+ if (log.isDebugEnabled) {
+ logDebug(s"New projects:\n${newProjects.map(_.treeString).mkString("\n")}")
+ }
+
+ Project(restoreOriginalOutputNames(newProjects, projects.map(_.name)), projectionChild)
+ }
+
+ /**
+ * Builds a pruned logical relation from the output of the output relation and the schema of the
+ * pruned base relation.
+ */
+ private def buildPrunedRelation(outputRelation: LogicalRelation,
+ prunedBaseRelation: BaseRelation): LogicalRelation = {
+ val prunedOutput = getPrunedOutput(outputRelation.output, prunedBaseRelation.schema)
+ outputRelation.copy(relation = prunedBaseRelation, output = prunedOutput)
+ }
+
+ // Prune the given output to make it consistent with `requiredSchema`.
+ private def getPrunedOutput(output: Seq[AttributeReference],
+ requiredSchema: StructType): Seq[AttributeReference] = {
+ // We need to replace the expression ids of the pruned relation output attributes
+ // with the expression ids of the original relation output attributes so that
+ // references to the original relation's output are not broken
+ val outputIdMap = output.map(att => (att.name, att.exprId)).toMap
+ requiredSchema
+ .toAttributes
+ .map {
+ case att if outputIdMap.contains(att.name) =>
+ att.withExprId(outputIdMap(att.name))
+ case att => att
+ }
+ }
+
+ /**
+ * Counts the "leaf" fields of the given dataType. Informally, this is the
+ * number of fields of non-complex data type in the tree representation of
+ * [[DataType]].
+ */
+ private def countLeaves(dataType: DataType): Int = {
+ dataType match {
+ case array: ArrayType => countLeaves(array.elementType)
+ case map: MapType => countLeaves(map.keyType) + countLeaves(map.valueType)
+ case struct: StructType =>
+ struct.map(field => countLeaves(field.dataType)).sum
+ case _ => 1
+ }
+ }
+
+ private def restoreOriginalOutputNames(
+ projectList: Seq[NamedExpression],
+ originalNames: Seq[String]): Seq[NamedExpression] = {
+ projectList.zip(originalNames).map {
+ case (attr: Attribute, name) => attr.withName(name)
+ case (alias: Alias, name) => if (name == alias.name) {
+ alias
+ } else {
+ AttributeReference(name, alias.dataType, alias.nullable, alias.metadata)(alias.exprId, alias.qualifier)
+ }
+ case (other, _) => other
+ }
+ }
+
+ // NOTE: `pruneDataSchema` and `sortLeftFieldsByRight` functions are copied from Spark 3.1.2,
+ // as these functions in `SchemaPruning` have bugs in Spark 3.0.2 (see SPARK-35096,
+ // https://github.com/apache/spark/commit/2bbe0a4151f2af00f1105489d5757be28ff278d6)
+ /**
+ * Prunes the nested schema by the requested fields. For example, if the schema is:
+ * `id int, s struct`, and given requested field "s.a", the inner field "b"
+ * is pruned in the returned schema: `id int, s struct`.
+ * Note that:
+ * 1. The schema field ordering at original schema is still preserved in pruned schema.
+ * 2. The top-level fields are not pruned here.
+ */
+ private def pruneDataSchema(
+ dataSchema: StructType,
+ requestedRootFields: Seq[RootField]): StructType = {
+ val resolver = SQLConf.get.resolver
+ // Merge the requested root fields into a single schema. Note the ordering of the fields
+ // in the resulting schema may differ from their ordering in the logical relation's
+ // original schema
+ val mergedSchema = requestedRootFields
+ .map { root: RootField => StructType(Array(root.field)) }
+ .reduceLeft(_ merge _)
+ val mergedDataSchema =
+ StructType(dataSchema.map(d => mergedSchema.find(m => resolver(m.name, d.name)).getOrElse(d)))
+ // Sort the fields of mergedDataSchema according to their order in dataSchema,
+ // recursively. This makes mergedDataSchema a pruned schema of dataSchema
+ sortLeftFieldsByRight(mergedDataSchema, dataSchema).asInstanceOf[StructType]
+ }
+
+ /**
+ * Sorts the fields and descendant fields of structs in left according to their order in
+ * right. This function assumes that the fields of left are a subset of the fields of
+ * right, recursively. That is, left is a "subschema" of right, ignoring order of
+ * fields.
+ */
+ private def sortLeftFieldsByRight(left: DataType, right: DataType): DataType =
+ (left, right) match {
+ case (ArrayType(leftElementType, containsNull), ArrayType(rightElementType, _)) =>
+ ArrayType(
+ sortLeftFieldsByRight(leftElementType, rightElementType),
+ containsNull)
+ case (MapType(leftKeyType, leftValueType, containsNull),
+ MapType(rightKeyType, rightValueType, _)) =>
+ MapType(
+ sortLeftFieldsByRight(leftKeyType, rightKeyType),
+ sortLeftFieldsByRight(leftValueType, rightValueType),
+ containsNull)
+ case (leftStruct: StructType, rightStruct: StructType) =>
+ val resolver = SQLConf.get.resolver
+ val filteredRightFieldNames = rightStruct.fieldNames
+ .filter(name => leftStruct.fieldNames.exists(resolver(_, name)))
+ val sortedLeftFields = filteredRightFieldNames.map { fieldName =>
+ val resolvedLeftStruct = leftStruct.find(p => resolver(p.name, fieldName)).get
+ val leftFieldType = resolvedLeftStruct.dataType
+ val rightFieldType = rightStruct(fieldName).dataType
+ val sortedLeftFieldType = sortLeftFieldsByRight(leftFieldType, rightFieldType)
+ StructField(fieldName, sortedLeftFieldType, nullable = resolvedLeftStruct.nullable)
+ }
+ StructType(sortedLeftFields)
+ case _ => left
+ }
+}
diff --git a/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark30HoodieParquetFileFormat.scala b/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark30HoodieParquetFileFormat.scala
new file mode 100644
index 0000000000000..4c33ac896770f
--- /dev/null
+++ b/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark30HoodieParquetFileFormat.scala
@@ -0,0 +1,414 @@
+/*
+ * 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.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.{ParquetFileReader, ParquetInputFormat, ParquetRecordReader}
+import org.apache.spark.TaskContext
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.avro.AvroDeserializer
+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.Spark30HoodieParquetFileFormat.{createParquetFilters, pruneInternalSchema, rebuildFilterFromParquet}
+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.1.2 w/ w/ the following changes applied to it:
+ *
+ * - Avoiding appending partition values to the rows read from the data file
+ * - Schema on-read
+ *
+ */
+class Spark30HoodieParquetFileFormat(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 timeZoneId = Option(sqlConf.sessionLocalTimeZone)
+
+ (file: PartitionedFile) => {
+ assert(!shouldAppendPartitionValues || file.partitionValues.numFields == partitionSchema.size)
+
+ val filePath = new Path(new URI(file.filePath))
+ val split =
+ new org.apache.parquet.hadoop.ParquetInputSplit(
+ filePath,
+ file.start,
+ file.start + file.length,
+ file.length,
+ Array.empty,
+ null)
+
+ 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)
+
+ var 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 =
+ ParquetFileReader.readFooter(sharedConf, filePath, SKIP_ROW_GROUPS).getFileMetaData
+ val datetimeRebaseMode = DataSourceUtils.datetimeRebaseMode(
+ footerFileMetaData.getKeyValueMetaData.get,
+ SQLConf.get.getConf(SQLConf.LEGACY_PARQUET_REBASE_MODE_IN_READ))
+ // Try to push down filters when filter push-down is enabled.
+ val pushed = if (enableParquetFilterPushDown) {
+ val parquetSchema = footerFileMetaData.getSchema
+ val parquetFilters = if (HoodieSparkUtils.gteqSpark3_1_3) {
+ createParquetFilters(
+ parquetSchema,
+ pushDownDate,
+ pushDownTimestamp,
+ pushDownDecimal,
+ pushDownStringStartWith,
+ pushDownInFilterThreshold,
+ isCaseSensitive,
+ datetimeRebaseMode)
+ } else {
+ createParquetFilters(
+ parquetSchema,
+ pushDownDate,
+ pushDownTimestamp,
+ pushDownDecimal,
+ pushDownStringStartWith,
+ pushDownInFilterThreshold,
+ isCaseSensitive)
+ }
+ 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 {
+ val (implicitTypeChangeInfo, sparkRequestSchema) = HoodieParquetFileFormatHelper.buildImplicitSchemaChangeInfo(hadoopAttemptConf, footerFileMetaData, requiredSchema)
+ if (!implicitTypeChangeInfo.isEmpty) {
+ shouldUseInternalSchema = true
+ hadoopAttemptConf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA, sparkRequestSchema.json)
+ }
+ implicitTypeChangeInfo
+ }
+
+ 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) {
+ new Spark30HoodieVectorizedParquetRecordReader(
+ convertTz.orNull,
+ datetimeRebaseMode.toString,
+ enableOffHeapColumnVector && taskContext.isDefined,
+ capacity,
+ typeChangeInfos)
+ } else {
+ new VectorizedParquetRecordReader(
+ convertTz.orNull,
+ datetimeRebaseMode.toString,
+ enableOffHeapColumnVector && taskContext.isDefined,
+ capacity)
+ }
+
+ val iter = new RecordReaderIterator(vectorizedReader)
+ // SPARK-23457 Register a task completion listener before `initialization`.
+ taskContext.foreach(_.addTaskCompletionListener[Unit](_ => iter.close()))
+ 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]]
+ } else {
+ logDebug(s"Falling back to parquet-mr")
+ // ParquetRecordReader returns InternalRow
+ val readSupport = new ParquetReadSupport(
+ convertTz,
+ enableVectorizedReader = false,
+ datetimeRebaseMode)
+ 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)
+ // SPARK-23457 Register a task completion listener before `initialization`.
+ taskContext.foreach(_.addTaskCompletionListener[Unit](_ => iter.close()))
+ 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)) {
+ val srcType = typeChangeInfos.get(i).getRight
+ val dstType = typeChangeInfos.get(i).getLeft
+ val needTimeZone = Cast.needsTimeZone(srcType, dstType)
+ Cast(attr, dstType, if (needTimeZone) timeZoneId else None)
+ } 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)))
+ }
+ }
+ }
+ }
+}
+
+object Spark30HoodieParquetFileFormat {
+
+ 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 createParquetFilters(args: Any*): ParquetFilters = {
+ // ParquetFilters bears a single ctor (in Spark 3.1)
+ val ctor = classOf[ParquetFilters].getConstructors.head
+ ctor.newInstance(args.map(_.asInstanceOf[AnyRef]): _*)
+ .asInstanceOf[ParquetFilters]
+ }
+
+ 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.0.x/src/main/scala/org/apache/spark/sql/hudi/Spark30ResolveHudiAlterTableCommand.scala b/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/hudi/Spark30ResolveHudiAlterTableCommand.scala
new file mode 100644
index 0000000000000..593432c59431b
--- /dev/null
+++ b/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/hudi/Spark30ResolveHudiAlterTableCommand.scala
@@ -0,0 +1,297 @@
+/*
+ * 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
+
+import org.apache.hudi.common.config.HoodieCommonConfig
+import org.apache.hudi.internal.schema.action.TableChange.ColumnChangeID
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.catalog.CatalogTable
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.catalog.TableChange._
+import org.apache.spark.sql.connector.catalog.TableChange
+import org.apache.spark.sql.hudi.command.Spark30AlterTableCommand
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{ArrayType, DataType, MapType, NullType, StructType}
+import org.apache.spark.sql.{AnalysisException, SparkSession}
+
+import java.util.Locale
+import scala.collection.mutable
+
+/**
+ * Rule to mostly resolve, normalize and rewrite column names based on case sensitivity
+ * for alter table column commands.
+ * TODO: we should remove this file when we support datasourceV2 for hoodie on spark3.0.x
+ */
+case class Spark30ResolveHudiAlterTableCommand(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) =>
+ if (isHoodieTable(table) && schemaEvolutionEnabled){
+ cols.foreach(c => failNullType(c.dataType))
+ val changes = cols.map { col =>
+ TableChange.addColumn(
+ col.name.toArray,
+ col.dataType,
+ col.nullable,
+ col.comment.orNull,
+ col.position.orNull)
+ }
+ val newChanges = normalizeChanges(changes, table.schema)
+ Spark30AlterTableCommand(table, newChanges, ColumnChangeID.ADD)
+ } else {
+ // throw back to spark
+ AlterTableAddColumnsStatement(add.tableName, add.columnsToAdd)
+ }
+ case a @ HoodieAlterTableAlterColumnStatement(asTable(table), _, _, _, _, _) =>
+ if (isHoodieTable(table) && schemaEvolutionEnabled){
+ a.dataType.foreach(failNullType)
+ val colName = a.column.toArray
+ val typeChange = a.dataType.map { newDataType =>
+ TableChange.updateColumnType(colName, newDataType)
+ }
+ val nullabilityChange = a.nullable.map { nullable =>
+ TableChange.updateColumnNullability(colName, nullable)
+ }
+ val commentChange = a.comment.map { newComment =>
+ TableChange.updateColumnComment(colName, newComment)
+ }
+ val positionChange = a.position.map { newPosition =>
+ TableChange.updateColumnPosition(colName, newPosition)
+ }
+ Spark30AlterTableCommand(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)
+ }
+ case rename @ HoodieAlterTableRenameColumnStatement(asTable(table), col, newName) =>
+ if (isHoodieTable(table) && schemaEvolutionEnabled){
+ val changes = Seq(TableChange.renameColumn(col.toArray, newName))
+ Spark30AlterTableCommand(table, normalizeChanges(changes, table.schema), ColumnChangeID.UPDATE)
+ } else {
+ // throw back to spark
+ AlterTableRenameColumnStatement(rename.tableName, rename.column, rename.newName)
+ }
+ case drop @ HoodieAlterTableDropColumnsStatement(asTable(table), cols) =>
+ if (isHoodieTable(table) && schemaEvolutionEnabled) {
+ val changes = cols.map(col => TableChange.deleteColumn(col.toArray))
+ Spark30AlterTableCommand(table, normalizeChanges(changes, table.schema), ColumnChangeID.DELETE)
+ } else {
+ // throw back to spark
+ AlterTableDropColumnsStatement(drop.tableName, drop.columnsToDrop)
+ }
+ case set @ HoodieAlterTableSetPropertiesStatement(asTable(table), props) =>
+ if (isHoodieTable(table) && schemaEvolutionEnabled) {
+ val changes = props.map { case (key, value) =>
+ TableChange.setProperty(key, value)
+ }.toSeq
+ Spark30AlterTableCommand(table, normalizeChanges(changes, table.schema), ColumnChangeID.PROPERTY_CHANGE)
+ } else {
+ // throw back to spark
+ AlterTableSetPropertiesStatement(set.tableName, set.properties)
+ }
+ case unset @ HoodieAlterTableUnsetPropertiesStatement(asTable(table), keys, _) =>
+ if (isHoodieTable(table) && schemaEvolutionEnabled) {
+ val changes = keys.map(key => TableChange.removeProperty(key))
+ Spark30AlterTableCommand(table, normalizeChanges(changes, table.schema), ColumnChangeID.PROPERTY_CHANGE)
+ } else {
+ // throw back to spark
+ AlterTableUnsetPropertiesStatement(unset.tableName, unset.propertyKeys, unset.ifExists)
+ }
+ }
+
+ private def schemaEvolutionEnabled(): Boolean =
+ sparkSession.sessionState.conf.getConfString(HoodieCommonConfig.SCHEMA_EVOLUTION_ENABLE.key,
+ HoodieCommonConfig.SCHEMA_EVOLUTION_ENABLE.defaultValue.toString).toBoolean
+
+ private def isHoodieTable(table: CatalogTable): Boolean = table.provider.map(_.toLowerCase(Locale.ROOT)).orNull == "hudi"
+
+ def normalizeChanges(changes: Seq[TableChange], schema: StructType): Seq[TableChange] = {
+ val colsToAdd = mutable.Map.empty[Seq[String], Seq[String]]
+ changes.flatMap {
+ case add: AddColumn =>
+ def addColumn(parentSchema: StructType, parentName: String, normalizedParentName: Seq[String]): TableChange = {
+ val fieldsAdded = colsToAdd.getOrElse(normalizedParentName, Nil)
+ val pos = findColumnPosition(add.position(), parentName, parentSchema, fieldsAdded)
+ val field = add.fieldNames().last
+ colsToAdd(normalizedParentName) = fieldsAdded :+ field
+ TableChange.addColumn(
+ (normalizedParentName :+ field).toArray,
+ add.dataType(),
+ add.isNullable,
+ add.comment,
+ pos)
+ }
+ val parent = add.fieldNames().init
+ if (parent.nonEmpty) {
+ // Adding a nested field, need to normalize the parent column and position
+ val target = schema.findNestedField(parent, includeCollections = true, SQLConf.get.resolver)
+ if (target.isEmpty) {
+ // Leave unresolved. Throws error in CheckAnalysis
+ Some(add)
+ } else {
+ val (normalizedName, sf) = target.get
+ sf.dataType match {
+ case struct: StructType =>
+ Some(addColumn(struct, parent.quoted, normalizedName :+ sf.name))
+ case other =>
+ Some(add)
+ }
+ }
+ } else {
+ // Adding to the root. Just need to normalize position
+ Some(addColumn(schema, "root", Nil))
+ }
+
+ case typeChange: UpdateColumnType =>
+ // Hive style syntax provides the column type, even if it may not have changed
+ val fieldOpt = schema.findNestedField(
+ typeChange.fieldNames(), includeCollections = true, SQLConf.get.resolver)
+
+ if (fieldOpt.isEmpty) {
+ // We couldn't resolve the field. Leave it to CheckAnalysis
+ Some(typeChange)
+ } else {
+ val (fieldNames, field) = fieldOpt.get
+ if (field.dataType == typeChange.newDataType()) {
+ // The user didn't want the field to change, so remove this change
+ None
+ } else {
+ Some(TableChange.updateColumnType(
+ (fieldNames :+ field.name).toArray, typeChange.newDataType()))
+ }
+ }
+ case n: UpdateColumnNullability =>
+ // Need to resolve column
+ resolveFieldNames(
+ schema,
+ n.fieldNames(),
+ TableChange.updateColumnNullability(_, n.nullable())).orElse(Some(n))
+
+ case position: UpdateColumnPosition =>
+ position.position() match {
+ case after: After =>
+ // Need to resolve column as well as position reference
+ val fieldOpt = schema.findNestedField(
+ position.fieldNames(), includeCollections = true, SQLConf.get.resolver)
+
+ if (fieldOpt.isEmpty) {
+ Some(position)
+ } else {
+ val (normalizedPath, field) = fieldOpt.get
+ val targetCol = schema.findNestedField(
+ normalizedPath :+ after.column(), includeCollections = true, SQLConf.get.resolver)
+ if (targetCol.isEmpty) {
+ // Leave unchanged to CheckAnalysis
+ Some(position)
+ } else {
+ Some(TableChange.updateColumnPosition(
+ (normalizedPath :+ field.name).toArray,
+ ColumnPosition.after(targetCol.get._2.name)))
+ }
+ }
+ case _ =>
+ // Need to resolve column
+ resolveFieldNames(
+ schema,
+ position.fieldNames(),
+ TableChange.updateColumnPosition(_, position.position())).orElse(Some(position))
+ }
+
+ case comment: UpdateColumnComment =>
+ resolveFieldNames(
+ schema,
+ comment.fieldNames(),
+ TableChange.updateColumnComment(_, comment.newComment())).orElse(Some(comment))
+
+ case rename: RenameColumn =>
+ resolveFieldNames(
+ schema,
+ rename.fieldNames(),
+ TableChange.renameColumn(_, rename.newName())).orElse(Some(rename))
+
+ case delete: DeleteColumn =>
+ resolveFieldNames(schema, delete.fieldNames(), TableChange.deleteColumn)
+ .orElse(Some(delete))
+
+ case column: ColumnChange =>
+ // This is informational for future developers
+ throw new UnsupportedOperationException(
+ "Please add an implementation for a column change here")
+ case other => Some(other)
+ }
+ }
+
+ /**
+ * Returns the table change if the field can be resolved, returns None if the column is not
+ * found. An error will be thrown in CheckAnalysis for columns that can't be resolved.
+ */
+ private def resolveFieldNames(
+ schema: StructType,
+ fieldNames: Array[String],
+ copy: Array[String] => TableChange): Option[TableChange] = {
+ val fieldOpt = schema.findNestedField(
+ fieldNames, includeCollections = true, SQLConf.get.resolver)
+ fieldOpt.map { case (path, field) => copy((path :+ field.name).toArray) }
+ }
+
+ private def findColumnPosition(
+ position: ColumnPosition,
+ parentName: String,
+ struct: StructType,
+ fieldsAdded: Seq[String]): ColumnPosition = {
+ position match {
+ case null => null
+ case after: After =>
+ (struct.fieldNames ++ fieldsAdded).find(n => SQLConf.get.resolver(n, after.column())) match {
+ case Some(colName) =>
+ ColumnPosition.after(colName)
+ case None =>
+ throw new AnalysisException("Couldn't find the reference column for " +
+ s"$after at $parentName")
+ }
+ case other => other
+ }
+ }
+
+ object asTable {
+ def unapply(parts: Seq[String]): Option[CatalogTable] = {
+ val identifier = parts match {
+ case Seq(tblName) => TableIdentifier(tblName)
+ case Seq(dbName, tblName) => TableIdentifier(tblName, Some(dbName))
+ case _ =>
+ throw new AnalysisException(
+ s"${parts} is not a valid TableIdentifier as it has more than 2 name parts.")
+ }
+ Some(sparkSession.sessionState.catalog.getTableMetadata(identifier))
+ }
+ }
+
+ private def failNullType(dt: DataType): Unit = {
+ def containsNullType(dt: DataType): Boolean = dt match {
+ case ArrayType(et, _) => containsNullType(et)
+ case MapType(kt, vt, _) => containsNullType(kt) || containsNullType(vt)
+ case StructType(fields) => fields.exists(f => containsNullType(f.dataType))
+ case _ => dt.isInstanceOf[NullType]
+ }
+
+ if (containsNullType(dt)) {
+ throw new AnalysisException(
+ s"Cannot create tables with ${NullType.simpleString} type.")
+ }
+ }
+}
diff --git a/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/hudi/command/Spark30AlterTableCommand.scala b/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/hudi/command/Spark30AlterTableCommand.scala
new file mode 100644
index 0000000000000..7bd1fda77efbd
--- /dev/null
+++ b/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/hudi/command/Spark30AlterTableCommand.scala
@@ -0,0 +1,325 @@
+/*
+ * 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.avro.AvroSchemaUtils.getAvroRecordQualifiedName
+import org.apache.hudi.client.utils.SparkInternalSchemaConverter
+import org.apache.hudi.common.model.{HoodieCommitMetadata, WriteOperationType}
+import org.apache.hudi.{AvroConversionUtils, 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.execution.command.RunnableCommand
+import org.apache.spark.sql.types.StructType
+
+import scala.collection.JavaConverters._
+import scala.util.control.NonFatal
+
+// TODO: we should remove this file when we support datasourceV2 for hoodie on spark3.0.x
+case class Spark30AlterTableCommand(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)
+ 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 other => throw new RuntimeException(s"find unsupported alter command type: ${other}")
+ }
+ Seq.empty[Row]
+ }
+
+ def applyAddAction(sparkSession: SparkSession): Unit = {
+ val (oldSchema, historySchema) = getInternalSchemaAndHistorySchemaStr(sparkSession)
+ val addChange = TableChanges.ColumnAddChange.get(oldSchema)
+ changes.map(_.asInstanceOf[AddColumn]).foreach { addColumn =>
+ val names = addColumn.fieldNames()
+ val parentName = Spark30AlterTableCommand.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 _ =>
+ }
+ }
+ val newSchema = SchemaChangeUtils.applyTableChanges2Schema(oldSchema, addChange)
+ val verifiedHistorySchema = if (historySchema == null || historySchema.isEmpty) {
+ SerDeHelper.inheritSchemas(oldSchema, "")
+ } else {
+ historySchema
+ }
+ Spark30AlterTableCommand.commitWithSchema(newSchema, verifiedHistorySchema, table, sparkSession)
+ logInfo("column add finished")
+ }
+
+ def applyDeleteAction(sparkSession: SparkSession): Unit = {
+ val (oldSchema, historySchema) = getInternalSchemaAndHistorySchemaStr(sparkSession)
+ val deleteChange = TableChanges.ColumnDeleteChange.get(oldSchema)
+ changes.map(_.asInstanceOf[DeleteColumn]).foreach { c =>
+ val originalColName = c.fieldNames().mkString(".");
+ Spark30AlterTableCommand.checkSchemaChange(Seq(originalColName), table)
+ deleteChange.deleteColumn(originalColName)
+ }
+ val newSchema = SchemaChangeUtils.applyTableChanges2Schema(oldSchema, deleteChange)
+ // delete action should not change the getMaxColumnId field.
+ newSchema.setMaxColumnId(oldSchema.getMaxColumnId)
+ val verifiedHistorySchema = if (historySchema == null || historySchema.isEmpty) {
+ SerDeHelper.inheritSchemas(oldSchema, "")
+ } else {
+ historySchema
+ }
+ Spark30AlterTableCommand.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(".")
+ Spark30AlterTableCommand.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 = Spark30AlterTableCommand.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
+ }
+ Spark30AlterTableCommand.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 = Spark30AlterTableCommand.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)
+ }
+}
+
+object Spark30AlterTableCommand 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, getAvroRecordQualifiedName(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)
+ }
+
+ 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")
+ }
+ }
+ }
+}
+
diff --git a/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/parser/HoodieSpark30SqlAstBuilder.scala b/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/parser/HoodieSpark30SqlAstBuilder.scala
new file mode 100644
index 0000000000000..c9dad3d9a2ea7
--- /dev/null
+++ b/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/parser/HoodieSpark30SqlAstBuilder.scala
@@ -0,0 +1,125 @@
+/*
+ * 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.parser
+
+import org.apache.spark.sql.catalyst.parser.ParserUtils.withOrigin
+import org.apache.spark.sql.catalyst.parser.SqlBaseParser._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.execution.SparkSqlAstBuilder
+import org.apache.spark.sql.internal.SQLConf
+
+// TODO: we should remove this file when we support datasourceV2 for hoodie on spark3.0.x
+class HoodieSpark30SqlAstBuilder(conf: SQLConf) extends SparkSqlAstBuilder(conf) {
+
+ /**
+ * Parse a [[AlterTableAlterColumnStatement]] command to alter a column's property.
+ *
+ * For example:
+ * {{{
+ * ALTER TABLE table1 ALTER COLUMN a.b.c TYPE bigint
+ * ALTER TABLE table1 ALTER COLUMN a.b.c SET NOT NULL
+ * ALTER TABLE table1 ALTER COLUMN a.b.c DROP NOT NULL
+ * ALTER TABLE table1 ALTER COLUMN a.b.c COMMENT 'new comment'
+ * ALTER TABLE table1 ALTER COLUMN a.b.c FIRST
+ * ALTER TABLE table1 ALTER COLUMN a.b.c AFTER x
+ * }}}
+ */
+ override def visitAlterTableAlterColumn(ctx: AlterTableAlterColumnContext): LogicalPlan = withOrigin(ctx) {
+ val alter = super.visitAlterTableAlterColumn(ctx).asInstanceOf[AlterTableAlterColumnStatement]
+ HoodieAlterTableAlterColumnStatement(alter.tableName, alter.column, alter.dataType, alter.nullable, alter.comment, alter.position)
+ }
+
+ /**
+ * Parse a [[org.apache.spark.sql.catalyst.plans.logical.AlterTableAddColumnsStatement]] command.
+ *
+ * For example:
+ * {{{
+ * ALTER TABLE table1
+ * ADD COLUMNS (col_name data_type [COMMENT col_comment], ...);
+ * }}}
+ */
+ override def visitAddTableColumns(ctx: AddTableColumnsContext): LogicalPlan = withOrigin(ctx) {
+ val add = super.visitAddTableColumns(ctx).asInstanceOf[AlterTableAddColumnsStatement]
+ HoodieAlterTableAddColumnsStatement(add.tableName, add.columnsToAdd)
+ }
+
+ /**
+ * Parse a [[org.apache.spark.sql.catalyst.plans.logical.AlterTableRenameColumnStatement]] command.
+ *
+ * For example:
+ * {{{
+ * ALTER TABLE table1 RENAME COLUMN a.b.c TO x
+ * }}}
+ */
+ override def visitRenameTableColumn(
+ ctx: RenameTableColumnContext): LogicalPlan = withOrigin(ctx) {
+ val rename = super.visitRenameTableColumn(ctx).asInstanceOf[AlterTableRenameColumnStatement]
+ HoodieAlterTableRenameColumnStatement(rename.tableName, rename.column, rename.newName)
+ }
+
+ /**
+ * Parse a [[AlterTableDropColumnsStatement]] command.
+ *
+ * For example:
+ * {{{
+ * ALTER TABLE table1 DROP COLUMN a.b.c
+ * ALTER TABLE table1 DROP COLUMNS a.b.c, x, y
+ * }}}
+ */
+ override def visitDropTableColumns(
+ ctx: DropTableColumnsContext): LogicalPlan = withOrigin(ctx) {
+ val drop = super.visitDropTableColumns(ctx).asInstanceOf[AlterTableDropColumnsStatement]
+ HoodieAlterTableDropColumnsStatement(drop.tableName, drop.columnsToDrop)
+ }
+
+ /**
+ * Parse [[AlterViewSetPropertiesStatement]] or [[AlterTableSetPropertiesStatement]] commands.
+ *
+ * For example:
+ * {{{
+ * ALTER TABLE table SET TBLPROPERTIES ('table_property' = 'property_value');
+ * ALTER VIEW view SET TBLPROPERTIES ('table_property' = 'property_value');
+ * }}}
+ */
+ override def visitSetTableProperties(
+ ctx: SetTablePropertiesContext): LogicalPlan = withOrigin(ctx) {
+ val set = super.visitSetTableProperties(ctx)
+ set match {
+ case s: AlterTableSetPropertiesStatement => HoodieAlterTableSetPropertiesStatement(s.tableName, s.properties)
+ case other => other
+ }
+ }
+
+ /**
+ * Parse [[AlterViewUnsetPropertiesStatement]] or [[AlterTableUnsetPropertiesStatement]] commands.
+ *
+ * For example:
+ * {{{
+ * ALTER TABLE table UNSET TBLPROPERTIES [IF EXISTS] ('comment', 'key');
+ * ALTER VIEW view UNSET TBLPROPERTIES [IF EXISTS] ('comment', 'key');
+ * }}}
+ */
+ override def visitUnsetTableProperties(
+ ctx: UnsetTablePropertiesContext): LogicalPlan = withOrigin(ctx) {
+ val unset = super.visitUnsetTableProperties(ctx)
+ unset match {
+ case us: AlterTableUnsetPropertiesStatement => HoodieAlterTableUnsetPropertiesStatement(us.tableName, us.propertyKeys, us.ifExists)
+ case other => other
+ }
+ }
+}
diff --git a/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/parser/HoodieSpark3_0ExtendedSqlParser.scala b/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/parser/HoodieSpark3_0ExtendedSqlParser.scala
new file mode 100644
index 0000000000000..523436520667a
--- /dev/null
+++ b/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/parser/HoodieSpark3_0ExtendedSqlParser.scala
@@ -0,0 +1,34 @@
+/*
+ * 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.parser
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.parser.ParserInterface
+import org.apache.spark.sql.execution.{SparkSqlAstBuilder, SparkSqlParser}
+
+// TODO: we should remove this file when we support datasourceV2 for hoodie on spark3.0.x
+class HoodieSpark3_0ExtendedSqlParser(session: SparkSession, delegate: ParserInterface) extends SparkSqlParser(session.sqlContext.conf)
+ with HoodieExtendedParserInterface
+ with Logging {
+
+ override val astBuilder: SparkSqlAstBuilder = new HoodieSpark30SqlAstBuilder(session.sqlContext.conf)
+
+ override def parseMultipartIdentifier(sqlText: String): Seq[String] = super[SparkSqlParser].parseMultipartIdentifier(sqlText)
+}
+
diff --git a/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/vectorized/ColumnarUtils.scala b/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/vectorized/ColumnarUtils.scala
new file mode 100644
index 0000000000000..e6015a65cb0d6
--- /dev/null
+++ b/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/vectorized/ColumnarUtils.scala
@@ -0,0 +1,32 @@
+/*
+ * 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.vectorized
+
+import org.apache.spark.sql.catalyst.InternalRow
+
+object ColumnarUtils {
+
+ /**
+ * Utility verifying whether provided instance of [[InternalRow]] is actually
+ * an instance of [[ColumnarBatchRow]]
+ *
+ * NOTE: This utility is required, since in Spark <= 3.3 [[ColumnarBatchRow]] is package-private
+ */
+ def isColumnarBatchRow(r: InternalRow): Boolean = r.isInstanceOf[ColumnarBatchRow]
+
+}
diff --git a/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/HoodieSpark31CatalystExpressionUtils.scala b/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/HoodieSpark31CatalystExpressionUtils.scala
index f565df80750cc..33e338d3afe8a 100644
--- a/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/HoodieSpark31CatalystExpressionUtils.scala
+++ b/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/HoodieSpark31CatalystExpressionUtils.scala
@@ -19,10 +19,19 @@
package org.apache.spark.sql
import org.apache.spark.sql.HoodieSparkTypeUtils.isCastPreservingOrdering
-import org.apache.spark.sql.catalyst.expressions.{Add, AnsiCast, AttributeReference, BitwiseOr, Cast, DateAdd, DateDiff, DateFormatClass, DateSub, Divide, Exp, Expm1, Expression, FromUTCTimestamp, FromUnixTime, Log, Log10, Log1p, Log2, Lower, Multiply, ParseToDate, ParseToTimestamp, ShiftLeft, ShiftRight, ToUTCTimestamp, ToUnixTimestamp, Upper}
+import org.apache.spark.sql.catalyst.expressions.{Add, AnsiCast, Attribute, AttributeReference, AttributeSet, BitwiseOr, Cast, DateAdd, DateDiff, DateFormatClass, DateSub, Divide, Exp, Expm1, Expression, FromUTCTimestamp, FromUnixTime, Log, Log10, Log1p, Log2, Lower, Multiply, ParseToDate, ParseToTimestamp, PredicateHelper, ShiftLeft, ShiftRight, ToUTCTimestamp, ToUnixTimestamp, Upper}
+import org.apache.spark.sql.execution.datasources.DataSourceStrategy
import org.apache.spark.sql.types.DataType
-object HoodieSpark31CatalystExpressionUtils extends HoodieSpark3CatalystExpressionUtils {
+object HoodieSpark31CatalystExpressionUtils extends HoodieSpark3CatalystExpressionUtils with PredicateHelper {
+
+ override def normalizeExprs(exprs: Seq[Expression], attributes: Seq[Attribute]): Seq[Expression] =
+ DataSourceStrategy.normalizeExprs(exprs, attributes)
+
+ override def extractPredicatesWithinOutputSet(condition: Expression,
+ outputSet: AttributeSet): Option[Expression] = {
+ super[PredicateHelper].extractPredicatesWithinOutputSet(condition, outputSet)
+ }
override def matchCast(expr: Expression): Option[(Expression, DataType, Option[String])] =
expr match {
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 ae515a0ad320e..445b8c3288092 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
@@ -26,7 +26,7 @@ import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression
import org.apache.spark.sql.catalyst.parser.ParserInterface
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Spark31HoodieParquetFileFormat}
-import org.apache.spark.sql.execution.datasources.{FilePartition, FileScanRDD, LogicalRelation, PartitionedFile}
+import org.apache.spark.sql.execution.datasources.{FilePartition, FileScanRDD, PartitionedFile}
import org.apache.spark.sql.hudi.SparkAdapter
import org.apache.spark.sql.parser.{HoodieExtendedParserInterface, HoodieSpark3_1ExtendedSqlParser}
import org.apache.spark.sql.types.{DataType, Metadata, MetadataBuilder, StructType}
@@ -34,14 +34,31 @@ import org.apache.spark.sql.vectorized.ColumnarUtils
import org.apache.spark.sql._
import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases
import org.apache.spark.sql.catalyst.catalog.CatalogTable
+import org.apache.spark.sql.catalyst.planning.PhysicalOperation
import org.apache.spark.sql.connector.catalog.V2TableWithV1Fallback
import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.storage.StorageLevel._
/**
* Implementation of [[SparkAdapter]] for Spark 3.1.x
*/
class Spark3_1Adapter extends BaseSpark3Adapter {
+ override def resolveHoodieTable(plan: LogicalPlan): Option[CatalogTable] = {
+ super.resolveHoodieTable(plan).orElse {
+ EliminateSubqueryAliases(plan) match {
+ // First, we need to weed out unresolved plans
+ case plan if !plan.resolved => None
+ // NOTE: When resolving Hudi table we allow [[Filter]]s and [[Project]]s be applied
+ // on top of it
+ case PhysicalOperation(_, _, DataSourceV2Relation(v2: V2TableWithV1Fallback, _, _, _, _)) if isHoodieTable(v2.v1Table) =>
+ Some(v2.v1Table)
+ case _ => None
+ }
+ }
+ }
+
override def isColumnarBatchRow(r: InternalRow): Boolean = ColumnarUtils.isColumnarBatchRow(r)
def createCatalystMetadataForMetaField: Metadata =
@@ -79,4 +96,25 @@ class Spark3_1Adapter extends BaseSpark3Adapter {
override def extractDeleteCondition(deleteFromTable: Command): Expression = {
deleteFromTable.asInstanceOf[DeleteFromTable].condition.getOrElse(null)
}
+
+ /**
+ * Converts instance of [[StorageLevel]] to a corresponding string
+ */
+ override def convertStorageLevelToString(level: StorageLevel): String = level match {
+ case NONE => "NONE"
+ case DISK_ONLY => "DISK_ONLY"
+ case DISK_ONLY_2 => "DISK_ONLY_2"
+ case DISK_ONLY_3 => "DISK_ONLY_3"
+ case MEMORY_ONLY => "MEMORY_ONLY"
+ case MEMORY_ONLY_2 => "MEMORY_ONLY_2"
+ case MEMORY_ONLY_SER => "MEMORY_ONLY_SER"
+ case MEMORY_ONLY_SER_2 => "MEMORY_ONLY_SER_2"
+ case MEMORY_AND_DISK => "MEMORY_AND_DISK"
+ case MEMORY_AND_DISK_2 => "MEMORY_AND_DISK_2"
+ case MEMORY_AND_DISK_SER => "MEMORY_AND_DISK_SER"
+ case MEMORY_AND_DISK_SER_2 => "MEMORY_AND_DISK_SER_2"
+ case OFF_HEAP => "OFF_HEAP"
+ case _ => throw new IllegalArgumentException(s"Invalid StorageLevel: $level")
+ }
+
}
diff --git a/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/HoodieSpark32CatalystExpressionUtils.scala b/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/HoodieSpark32CatalystExpressionUtils.scala
index c802d38deda6e..9cd85ca8a53ef 100644
--- a/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/HoodieSpark32CatalystExpressionUtils.scala
+++ b/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/HoodieSpark32CatalystExpressionUtils.scala
@@ -18,10 +18,19 @@
package org.apache.spark.sql
import org.apache.spark.sql.HoodieSparkTypeUtils.isCastPreservingOrdering
-import org.apache.spark.sql.catalyst.expressions.{Add, AnsiCast, AttributeReference, BitwiseOr, Cast, DateAdd, DateDiff, DateFormatClass, DateSub, Divide, Exp, Expm1, Expression, FromUTCTimestamp, FromUnixTime, Log, Log10, Log1p, Log2, Lower, Multiply, ParseToDate, ParseToTimestamp, ShiftLeft, ShiftRight, ToUTCTimestamp, ToUnixTimestamp, Upper}
+import org.apache.spark.sql.catalyst.expressions.{Add, AnsiCast, Attribute, AttributeReference, AttributeSet, BitwiseOr, Cast, DateAdd, DateDiff, DateFormatClass, DateSub, Divide, Exp, Expm1, Expression, FromUTCTimestamp, FromUnixTime, Log, Log10, Log1p, Log2, Lower, Multiply, ParseToDate, ParseToTimestamp, PredicateHelper, ShiftLeft, ShiftRight, ToUTCTimestamp, ToUnixTimestamp, Upper}
+import org.apache.spark.sql.execution.datasources.DataSourceStrategy
import org.apache.spark.sql.types.DataType
-object HoodieSpark32CatalystExpressionUtils extends HoodieSpark3CatalystExpressionUtils {
+object HoodieSpark32CatalystExpressionUtils extends HoodieSpark3CatalystExpressionUtils with PredicateHelper {
+
+ override def normalizeExprs(exprs: Seq[Expression], attributes: Seq[Attribute]): Seq[Expression] =
+ DataSourceStrategy.normalizeExprs(exprs, attributes)
+
+ override def extractPredicatesWithinOutputSet(condition: Expression,
+ outputSet: AttributeSet): Option[Expression] = {
+ super[PredicateHelper].extractPredicatesWithinOutputSet(condition, outputSet)
+ }
override def matchCast(expr: Expression): Option[(Expression, DataType, Option[String])] =
expr match {
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 7a4c5c9d172db..1e46bcc2dbf4d 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
@@ -22,9 +22,12 @@ 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.analysis.EliminateSubqueryAliases
+import org.apache.spark.sql.catalyst.catalog.CatalogTable
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}
+import org.apache.spark.sql.catalyst.planning.PhysicalOperation
+import org.apache.spark.sql.catalyst.plans.logical.{Command, DeleteFromTable, LogicalPlan}
import org.apache.spark.sql.catalyst.util.METADATA_COL_ATTR_KEY
import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Spark32PlusHoodieParquetFileFormat}
import org.apache.spark.sql.execution.datasources.{FilePartition, FileScanRDD, PartitionedFile}
@@ -32,11 +35,28 @@ import org.apache.spark.sql.hudi.analysis.TableValuedFunctions
import org.apache.spark.sql.parser.{HoodieExtendedParserInterface, HoodieSpark3_2ExtendedSqlParser}
import org.apache.spark.sql.types.{DataType, Metadata, MetadataBuilder, StructType}
import org.apache.spark.sql.vectorized.ColumnarUtils
+import org.apache.spark.sql.connector.catalog.V2TableWithV1Fallback
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.storage.StorageLevel._
/**
* Implementation of [[SparkAdapter]] for Spark 3.2.x branch
*/
class Spark3_2Adapter extends BaseSpark3Adapter {
+ override def resolveHoodieTable(plan: LogicalPlan): Option[CatalogTable] = {
+ super.resolveHoodieTable(plan).orElse {
+ EliminateSubqueryAliases(plan) match {
+ // First, we need to weed out unresolved plans
+ case plan if !plan.resolved => None
+ // NOTE: When resolving Hudi table we allow [[Filter]]s and [[Project]]s be applied
+ // on top of it
+ case PhysicalOperation(_, _, DataSourceV2Relation(v2: V2TableWithV1Fallback, _, _, _, _)) if isHoodieTable(v2.v1Table) =>
+ Some(v2.v1Table)
+ case _ => None
+ }
+ }
+ }
override def isColumnarBatchRow(r: InternalRow): Boolean = ColumnarUtils.isColumnarBatchRow(r)
@@ -79,4 +99,24 @@ class Spark3_2Adapter extends BaseSpark3Adapter {
override def injectTableFunctions(extensions: SparkSessionExtensions): Unit = {
TableValuedFunctions.funcs.foreach(extensions.injectTableFunction)
}
+
+ /**
+ * Converts instance of [[StorageLevel]] to a corresponding string
+ */
+ override def convertStorageLevelToString(level: StorageLevel): String = level match {
+ case NONE => "NONE"
+ case DISK_ONLY => "DISK_ONLY"
+ case DISK_ONLY_2 => "DISK_ONLY_2"
+ case DISK_ONLY_3 => "DISK_ONLY_3"
+ case MEMORY_ONLY => "MEMORY_ONLY"
+ case MEMORY_ONLY_2 => "MEMORY_ONLY_2"
+ case MEMORY_ONLY_SER => "MEMORY_ONLY_SER"
+ case MEMORY_ONLY_SER_2 => "MEMORY_ONLY_SER_2"
+ case MEMORY_AND_DISK => "MEMORY_AND_DISK"
+ case MEMORY_AND_DISK_2 => "MEMORY_AND_DISK_2"
+ case MEMORY_AND_DISK_SER => "MEMORY_AND_DISK_SER"
+ case MEMORY_AND_DISK_SER_2 => "MEMORY_AND_DISK_SER_2"
+ case OFF_HEAP => "OFF_HEAP"
+ case _ => throw new IllegalArgumentException(s"Invalid StorageLevel: $level")
+ }
}
diff --git a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/HoodieSpark33CatalystExpressionUtils.scala b/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/HoodieSpark33CatalystExpressionUtils.scala
index 94bd089522e7e..3ba5ed3d99910 100644
--- a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/HoodieSpark33CatalystExpressionUtils.scala
+++ b/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/HoodieSpark33CatalystExpressionUtils.scala
@@ -18,10 +18,19 @@
package org.apache.spark.sql
import HoodieSparkTypeUtils.isCastPreservingOrdering
-import org.apache.spark.sql.catalyst.expressions.{Add, AnsiCast, AttributeReference, BitwiseOr, Cast, DateAdd, DateDiff, DateFormatClass, DateSub, Divide, Exp, Expm1, Expression, FromUTCTimestamp, FromUnixTime, Log, Log10, Log1p, Log2, Lower, Multiply, ParseToDate, ParseToTimestamp, ShiftLeft, ShiftRight, ToUTCTimestamp, ToUnixTimestamp, Upper}
+import org.apache.spark.sql.catalyst.expressions.{Add, AnsiCast, Attribute, AttributeReference, AttributeSet, BitwiseOr, Cast, DateAdd, DateDiff, DateFormatClass, DateSub, Divide, Exp, Expm1, Expression, FromUTCTimestamp, FromUnixTime, Log, Log10, Log1p, Log2, Lower, Multiply, ParseToDate, ParseToTimestamp, PredicateHelper, ShiftLeft, ShiftRight, ToUTCTimestamp, ToUnixTimestamp, Upper}
+import org.apache.spark.sql.execution.datasources.DataSourceStrategy
import org.apache.spark.sql.types.DataType
-object HoodieSpark33CatalystExpressionUtils extends HoodieSpark3CatalystExpressionUtils {
+object HoodieSpark33CatalystExpressionUtils extends HoodieSpark3CatalystExpressionUtils with PredicateHelper {
+
+ override def normalizeExprs(exprs: Seq[Expression], attributes: Seq[Attribute]): Seq[Expression] =
+ DataSourceStrategy.normalizeExprs(exprs, attributes)
+
+ override def extractPredicatesWithinOutputSet(condition: Expression,
+ outputSet: AttributeSet): Option[Expression] = {
+ super[PredicateHelper].extractPredicatesWithinOutputSet(condition, outputSet)
+ }
override def matchCast(expr: Expression): Option[(Expression, DataType, Option[String])] =
expr match {
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 857147eb66c25..466e986b80532 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
@@ -22,8 +22,11 @@ import org.apache.hudi.Spark33HoodieFileScanRDD
import org.apache.spark.sql._
import org.apache.spark.sql.avro._
import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases
+import org.apache.spark.sql.catalyst.catalog.CatalogTable
import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression}
import org.apache.spark.sql.catalyst.parser.ParserInterface
+import org.apache.spark.sql.catalyst.planning.PhysicalOperation
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.util.METADATA_COL_ATTR_KEY
import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Spark32PlusHoodieParquetFileFormat}
@@ -32,12 +35,30 @@ import org.apache.spark.sql.hudi.analysis.TableValuedFunctions
import org.apache.spark.sql.parser.{HoodieExtendedParserInterface, HoodieSpark3_3ExtendedSqlParser}
import org.apache.spark.sql.types.{DataType, Metadata, MetadataBuilder, StructType}
import org.apache.spark.sql.vectorized.ColumnarBatchRow
+import org.apache.spark.sql.connector.catalog.V2TableWithV1Fallback
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.storage.StorageLevel._
/**
* Implementation of [[SparkAdapter]] for Spark 3.3.x branch
*/
class Spark3_3Adapter extends BaseSpark3Adapter {
+ override def resolveHoodieTable(plan: LogicalPlan): Option[CatalogTable] = {
+ super.resolveHoodieTable(plan).orElse {
+ EliminateSubqueryAliases(plan) match {
+ // First, we need to weed out unresolved plans
+ case plan if !plan.resolved => None
+ // NOTE: When resolving Hudi table we allow [[Filter]]s and [[Project]]s be applied
+ // on top of it
+ case PhysicalOperation(_, _, DataSourceV2Relation(v2: V2TableWithV1Fallback, _, _, _, _)) if isHoodieTable(v2.v1Table) =>
+ Some(v2.v1Table)
+ case _ => None
+ }
+ }
+ }
+
override def isColumnarBatchRow(r: InternalRow): Boolean = r.isInstanceOf[ColumnarBatchRow]
def createCatalystMetadataForMetaField: Metadata =
@@ -79,4 +100,24 @@ class Spark3_3Adapter extends BaseSpark3Adapter {
override def injectTableFunctions(extensions: SparkSessionExtensions): Unit = {
TableValuedFunctions.funcs.foreach(extensions.injectTableFunction)
}
+
+ /**
+ * Converts instance of [[StorageLevel]] to a corresponding string
+ */
+ override def convertStorageLevelToString(level: StorageLevel): String = level match {
+ case NONE => "NONE"
+ case DISK_ONLY => "DISK_ONLY"
+ case DISK_ONLY_2 => "DISK_ONLY_2"
+ case DISK_ONLY_3 => "DISK_ONLY_3"
+ case MEMORY_ONLY => "MEMORY_ONLY"
+ case MEMORY_ONLY_2 => "MEMORY_ONLY_2"
+ case MEMORY_ONLY_SER => "MEMORY_ONLY_SER"
+ case MEMORY_ONLY_SER_2 => "MEMORY_ONLY_SER_2"
+ case MEMORY_AND_DISK => "MEMORY_AND_DISK"
+ case MEMORY_AND_DISK_2 => "MEMORY_AND_DISK_2"
+ case MEMORY_AND_DISK_SER => "MEMORY_AND_DISK_SER"
+ case MEMORY_AND_DISK_SER_2 => "MEMORY_AND_DISK_SER_2"
+ case OFF_HEAP => "OFF_HEAP"
+ case _ => throw new IllegalArgumentException(s"Invalid StorageLevel: $level")
+ }
}
diff --git a/packaging/bundle-validation/ci_run.sh b/packaging/bundle-validation/ci_run.sh
index bef2ab3282c63..94b3933d8e237 100755
--- a/packaging/bundle-validation/ci_run.sh
+++ b/packaging/bundle-validation/ci_run.sh
@@ -41,6 +41,16 @@ if [[ ${SPARK_RUNTIME} == 'spark2.4' ]]; then
CONFLUENT_VERSION=5.5.12
KAFKA_CONNECT_HDFS_VERSION=10.1.13
IMAGE_TAG=flink1136hive239spark248
+elif [[ ${SPARK_RUNTIME} == 'spark3.0.2' ]]; then
+ HADOOP_VERSION=2.7.7
+ HIVE_VERSION=3.1.3
+ DERBY_VERSION=10.14.1.0
+ FLINK_VERSION=1.14.6
+ SPARK_VERSION=3.0.2
+ SPARK_HADOOP_VERSION=2.7
+ CONFLUENT_VERSION=5.5.12
+ KAFKA_CONNECT_HDFS_VERSION=10.1.13
+ IMAGE_TAG=flink1146hive313spark302
elif [[ ${SPARK_RUNTIME} == 'spark3.1.3' ]]; then
HADOOP_VERSION=2.7.7
HIVE_VERSION=3.1.3
diff --git a/pom.xml b/pom.xml
index 80e77c0431ff8..b14a8ac7ae1b2 100644
--- a/pom.xml
+++ b/pom.xml
@@ -156,6 +156,7 @@
flink-hadoop-compatibility_2.12
5.17.2
3.1.3
+ 3.0.2
3.2.3
3.3.1
hudi-spark2
@@ -2238,6 +2239,44 @@
+
+ spark3.0
+
+ ${spark30.version}
+ ${spark3.version}
+ 3.0
+ ${scala12.version}
+ 2.12
+ hudi-spark3.0.x
+ hudi-spark3-common
+ ${kafka.spark3.version}
+
+ 1.10.1
+ 1.5.13
+ 1.8.2
+ 4.8-1
+ ${fasterxml.spark3.version}
+ ${fasterxml.spark3.version}
+ ${fasterxml.spark3.version}
+ ${fasterxml.spark3.version}
+ ${pulsar.spark.scala12.version}
+ true
+ true
+
+
+ hudi-spark-datasource/hudi-spark3.0.x
+ hudi-spark-datasource/hudi-spark3-common
+
+
+
+ spark3.0
+
+
+
+
spark3.1