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: + *
    + *
  1. Avoiding appending partition values to the rows read from the data file
  2. + *
  3. Schema on-read
  4. + *
+ */ +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