diff --git a/.github/workflows/bot.yml b/.github/workflows/bot.yml
index a26d3c40fb081..ba5ca847a6aa6 100644
--- a/.github/workflows/bot.yml
+++ b/.github/workflows/bot.yml
@@ -177,29 +177,93 @@ jobs:
java-version: '17'
distribution: 'adopt'
architecture: x64
+ cache: maven
+ - name: Verify Java 17 version
+ run: |
+ echo "JAVA_HOME: $JAVA_HOME"
+ java -version
+ which java
- name: Quickstart Test
env:
SCALA_PROFILE: ${{ matrix.scalaProfile }}
SPARK_PROFILE: ${{ matrix.sparkProfile }}
- run:
- mvn test -Punit-tests -Pjava17 -D"$SCALA_PROFILE" -D"$SPARK_PROFILE" -pl hudi-examples/hudi-examples-spark $MVN_ARGS
- - name: UT - Common & Spark
+ run: |
+ export PATH="$JAVA_HOME/bin:$PATH"
+ mvn test -Punit-tests -Pjava17 -D"$SCALA_PROFILE" -D"$SPARK_PROFILE" -DwildcardSuites=skipScalaTests -DfailIfNoTests=false -pl hudi-examples/hudi-examples-spark $MVN_ARGS
+ - name: Java UT - Common & Spark
env:
SCALA_PROFILE: ${{ matrix.scalaProfile }}
SPARK_PROFILE: ${{ matrix.sparkProfile }}
SPARK_MODULES: ${{ matrix.sparkModules }}
- if: ${{ !endsWith(env.SPARK_PROFILE, '3.2') }} # skip test spark 3.2 as it's covered by Azure CI
- run:
- mvn test -Punit-tests -Pjava17 -D"$SCALA_PROFILE" -D"$SPARK_PROFILE" -pl "hudi-common,$SPARK_COMMON_MODULES,$SPARK_MODULES" $MVN_ARGS
- - name: FT - Spark
+ run: |
+ export PATH="$JAVA_HOME/bin:$PATH"
+ mvn test -Punit-tests -Pjava17 -D"$SCALA_PROFILE" -D"$SPARK_PROFILE" -DwildcardSuites=skipScalaTests -DfailIfNoTests=false -pl "hudi-common,$SPARK_COMMON_MODULES,$SPARK_MODULES" $MVN_ARGS
+ - name: Java FT - Spark
env:
SCALA_PROFILE: ${{ matrix.scalaProfile }}
SPARK_PROFILE: ${{ matrix.sparkProfile }}
SPARK_MODULES: ${{ matrix.sparkModules }}
- if: ${{ !endsWith(env.SPARK_PROFILE, '3.2') }} # skip test spark 3.2 as it's covered by Azure CI
- run:
+ if: ${{ !endsWith(env.SPARK_PROFILE, '3.4') }} # skip test spark 3.4 as it's covered by Azure CI
+ run: |
+ export PATH="$JAVA_HOME/bin:$PATH"
mvn test -Pfunctional-tests -Pjava17 -D"$SCALA_PROFILE" -D"$SPARK_PROFILE" -pl "$SPARK_COMMON_MODULES,$SPARK_MODULES" $MVN_ARGS
+ test-spark-java17-scala-tests:
+ runs-on: ubuntu-latest
+ strategy:
+ matrix:
+ include:
+ - scalaProfile: "scala-2.12"
+ sparkProfile: "spark3.3"
+ sparkModules: "hudi-spark-datasource/hudi-spark3.3.x"
+ - scalaProfile: "scala-2.12"
+ sparkProfile: "spark3.4"
+ sparkModules: "hudi-spark-datasource/hudi-spark3.4.x"
+
+ steps:
+ - uses: actions/checkout@v3
+ - name: Set up JDK 8
+ uses: actions/setup-java@v3
+ with:
+ java-version: '8'
+ distribution: 'temurin'
+ architecture: x64
+ cache: maven
+ - name: Build Project
+ env:
+ SCALA_PROFILE: ${{ matrix.scalaProfile }}
+ SPARK_PROFILE: ${{ matrix.sparkProfile }}
+ run:
+ mvn clean install -T 2 -D"$SCALA_PROFILE" -D"$SPARK_PROFILE" -DskipTests=true $MVN_ARGS -am -pl "hudi-examples/hudi-examples-spark,hudi-common,$SPARK_COMMON_MODULES,$SPARK_MODULES"
+ - name: Set up JDK 17
+ uses: actions/setup-java@v3
+ with:
+ java-version: '17'
+ distribution: 'temurin'
+ architecture: x64
+ cache: maven
+ - name: Verify Java 17 version
+ run: |
+ echo "JAVA_HOME: $JAVA_HOME"
+ java -version
+ which java
+ - name: Scala UT - Common & Spark
+ env:
+ SCALA_PROFILE: ${{ matrix.scalaProfile }}
+ SPARK_PROFILE: ${{ matrix.sparkProfile }}
+ SPARK_MODULES: ${{ matrix.sparkModules }}
+ run: |
+ export PATH="$JAVA_HOME/bin:$PATH"
+ mvn test -Punit-tests -Pjava17 -D"$SCALA_PROFILE" -D"$SPARK_PROFILE" -Dtest=skipJavaTests -DfailIfNoTests=false -pl "hudi-common,$SPARK_COMMON_MODULES,$SPARK_MODULES" $MVN_ARGS
+ - name: Scala FT - Spark
+ env:
+ SCALA_PROFILE: ${{ matrix.scalaProfile }}
+ SPARK_PROFILE: ${{ matrix.sparkProfile }}
+ SPARK_MODULES: ${{ matrix.sparkModules }}
+ run: |
+ export PATH="$JAVA_HOME/bin:$PATH"
+ mvn test -Pfunctional-tests -Pjava17 -D"$SCALA_PROFILE" -D"$SPARK_PROFILE" -Dtest=skipJavaTests -DfailIfNoTests=false -pl "$SPARK_COMMON_MODULES,$SPARK_MODULES" $MVN_ARGS
+
test-flink:
runs-on: ubuntu-latest
strategy:
diff --git a/azure-pipelines-20230430.yml b/azure-pipelines-20230430.yml
index a491a76266bdc..78005e4f03255 100644
--- a/azure-pipelines-20230430.yml
+++ b/azure-pipelines-20230430.yml
@@ -47,7 +47,7 @@ parameters:
default:
- 'hudi-spark-datasource'
- 'hudi-spark-datasource/hudi-spark'
- - 'hudi-spark-datasource/hudi-spark3.2.x'
+ - 'hudi-spark-datasource/hudi-spark3.4.x'
- 'hudi-spark-datasource/hudi-spark3.2plus-common'
- 'hudi-spark-datasource/hudi-spark3-common'
- 'hudi-spark-datasource/hudi-spark-common'
@@ -73,7 +73,7 @@ parameters:
- '!hudi-flink-datasource/hudi-flink1.18.x'
- '!hudi-spark-datasource'
- '!hudi-spark-datasource/hudi-spark'
- - '!hudi-spark-datasource/hudi-spark3.2.x'
+ - '!hudi-spark-datasource/hudi-spark3.4.x'
- '!hudi-spark-datasource/hudi-spark3.2plus-common'
- '!hudi-spark-datasource/hudi-spark3-common'
- '!hudi-spark-datasource/hudi-spark-common'
@@ -97,7 +97,7 @@ parameters:
- '!hudi-flink-datasource/hudi-flink1.18.x'
variables:
- BUILD_PROFILES: '-Dscala-2.12 -Dspark3.2 -Dflink1.18'
+ BUILD_PROFILES: '-Dscala-2.12 -Dspark3.4 -Dflink1.18'
PLUGIN_OPTS: '-Dcheckstyle.skip=true -Drat.skip=true -Djacoco.skip=true -ntp -B -V -Pwarn-log -Dorg.slf4j.simpleLogger.log.org.apache.maven.plugins.shade=warn -Dorg.slf4j.simpleLogger.log.org.apache.maven.plugins.dependency=warn'
MVN_OPTS_INSTALL: '-Phudi-platform-service -DskipTests $(BUILD_PROFILES) $(PLUGIN_OPTS) -Dmaven.wagon.httpconnectionManager.ttlSeconds=25 -Dmaven.wagon.http.retryHandler.count=5'
MVN_OPTS_TEST: '-fae -Pwarn-log $(BUILD_PROFILES) $(PLUGIN_OPTS)'
diff --git a/hudi-cli/pom.xml b/hudi-cli/pom.xml
index fcc9a63a87795..de463b42d8315 100644
--- a/hudi-cli/pom.xml
+++ b/hudi-cli/pom.xml
@@ -143,6 +143,13 @@
+
+
+ javax.validation
+ validation-api
+ 2.0.1.Final
+
+
org.scala-lang
diff --git a/hudi-client/hudi-client-common/pom.xml b/hudi-client/hudi-client-common/pom.xml
index 171c3384f82da..4fba73e568081 100644
--- a/hudi-client/hudi-client-common/pom.xml
+++ b/hudi-client/hudi-client-common/pom.xml
@@ -47,6 +47,48 @@
org.apache.hudihudi-timeline-service${project.version}
+
+
+
+ org.eclipse.jetty
+ *
+
+
+
+
+
+
+
+ org.eclipse.jetty
+ jetty-server
+
+
+ org.eclipse.jetty
+ jetty-servlet
+
+
+ org.eclipse.jetty
+ jetty-http
+
+
+ org.eclipse.jetty
+ jetty-io
+
+
+ org.eclipse.jetty
+ jetty-util
+
+
+ org.eclipse.jetty
+ jetty-webapp
+
+
+ org.eclipse.jetty
+ jetty-xml
+
+
+ org.eclipse.jetty
+ jetty-security
@@ -188,6 +230,10 @@
org.pentaho*
+
+ org.codehaus.janino
+ janino
+
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndexUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndexUtils.java
index b6db316a3b677..b2f470197d6b8 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndexUtils.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndexUtils.java
@@ -18,6 +18,7 @@
package org.apache.hudi.index;
+import org.apache.hudi.avro.AvroSchemaUtils;
import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.data.HoodieData;
@@ -241,8 +242,11 @@ private static HoodieData> getExistingRecords(
.filterCompletedInstants()
.lastInstant()
.map(HoodieInstant::getTimestamp);
+ Schema baseFileReaderSchema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getWriteSchema()), config.allowOperationMetadataField());
+ boolean hasTimestampFields = AvroSchemaUtils.isLogicalTimestampRepairNeeded(hoodieTable.getHadoopConf(),
+ () -> baseFileReaderSchema != null && AvroSchemaUtils.hasTimestampMillisField(baseFileReaderSchema));
return partitionLocations.flatMap(p
- -> new HoodieMergedReadHandle(config, instantTime, hoodieTable, Pair.of(p.getPartitionPath(), p.getFileId()))
+ -> new HoodieMergedReadHandle(config, instantTime, hoodieTable, Pair.of(p.getPartitionPath(), p.getFileId()), hasTimestampFields)
.getMergedRecords().iterator());
}
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergedReadHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergedReadHandle.java
index 738688c62193a..361686706bc70 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergedReadHandle.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergedReadHandle.java
@@ -19,6 +19,7 @@
package org.apache.hudi.io;
+import org.apache.hudi.avro.AvroSchemaUtils;
import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.common.model.FileSlice;
import org.apache.hudi.common.model.HoodieLogFile;
@@ -54,14 +55,16 @@ public class HoodieMergedReadHandle extends HoodieReadHandle instantTime,
- HoodieTable hoodieTable,
- Pair partitionPathFileIDPair) {
+ public HoodieMergedReadHandle(HoodieWriteConfig config, Option instantTime,
+ HoodieTable hoodieTable, Pair partitionPathFileIDPair,
+ boolean hasTimestampFields) {
super(config, instantTime, hoodieTable, partitionPathFileIDPair);
- readerSchema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getSchema()), config.allowOperationMetadataField());
+ Schema orignalReaderSchema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getSchema()), config.allowOperationMetadataField());
// config.getSchema is not canonicalized, while config.getWriteSchema is canonicalized. So, we have to use the canonicalized schema to read the existing data.
- baseFileReaderSchema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getWriteSchema()), config.allowOperationMetadataField());
+ this.baseFileReaderSchema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getWriteSchema()), config.allowOperationMetadataField());
+ // Repair reader schema.
+ // Assume writer schema should be correct. If not, no repair happens.
+ readerSchema = hasTimestampFields ? AvroSchemaUtils.getRepairedSchema(orignalReaderSchema, this.baseFileReaderSchema) : orignalReaderSchema;
}
public List> getMergedRecords() {
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/TimestampBasedAvroKeyGenerator.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/TimestampBasedAvroKeyGenerator.java
index 1990b2dab44ef..ea2e0911d3010 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/TimestampBasedAvroKeyGenerator.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/TimestampBasedAvroKeyGenerator.java
@@ -41,6 +41,7 @@
import java.util.TimeZone;
import java.util.concurrent.TimeUnit;
+import static java.util.concurrent.TimeUnit.MICROSECONDS;
import static java.util.concurrent.TimeUnit.MILLISECONDS;
import static java.util.concurrent.TimeUnit.SECONDS;
import static org.apache.hudi.common.config.TimestampKeyGeneratorConfig.DATE_TIME_PARSER;
@@ -54,7 +55,7 @@
*/
public class TimestampBasedAvroKeyGenerator extends SimpleAvroKeyGenerator {
public enum TimestampType implements Serializable {
- UNIX_TIMESTAMP, DATE_STRING, MIXED, EPOCHMILLISECONDS, SCALAR
+ UNIX_TIMESTAMP, DATE_STRING, MIXED, EPOCHMILLISECONDS, EPOCHMICROSECONDS, SCALAR
}
private final TimeUnit timeUnit;
@@ -93,6 +94,9 @@ public TimestampBasedAvroKeyGenerator(TypedProperties config) throws IOException
case EPOCHMILLISECONDS:
timeUnit = MILLISECONDS;
break;
+ case EPOCHMICROSECONDS:
+ timeUnit = MICROSECONDS;
+ break;
case UNIX_TIMESTAMP:
timeUnit = SECONDS;
break;
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java
index dfa464d8af8b5..080479fc417d6 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java
@@ -45,6 +45,7 @@
import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy;
import org.apache.hudi.common.model.HoodieFileFormat;
import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecordLocation;
import org.apache.hudi.common.model.HoodieWriteStat;
import org.apache.hudi.common.table.HoodieTableConfig;
import org.apache.hudi.common.table.HoodieTableMetaClient;
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieMergeHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieMergeHelper.java
index c1523d564e480..21787202c96a3 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieMergeHelper.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieMergeHelper.java
@@ -18,6 +18,7 @@
package org.apache.hudi.table.action.commit;
+import org.apache.hudi.avro.AvroSchemaUtils;
import org.apache.hudi.common.config.HoodieCommonConfig;
import org.apache.hudi.common.model.HoodieBaseFile;
import org.apache.hudi.common.model.HoodieRecord;
@@ -86,7 +87,12 @@ public void runMerge(HoodieTable, ?, ?, ?> table,
HoodieFileReader bootstrapFileReader = null;
Schema writerSchema = mergeHandle.getWriterSchemaWithMetaFields();
- Schema readerSchema = baseFileReader.getSchema();
+ Schema readerSchema;
+ if (!table.isMetadataTable() && AvroSchemaUtils.isLogicalTimestampRepairNeeded(hadoopConf, () -> true)) {
+ readerSchema = AvroSchemaUtils.getRepairedSchema(baseFileReader.getSchema(), writerSchema);
+ } else {
+ readerSchema = baseFileReader.getSchema();
+ }
// In case Advanced Schema Evolution is enabled we might need to rewrite currently
// persisted records to adhere to an evolved schema
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/HoodieCompactor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/HoodieCompactor.java
index 906ea6473a4b1..fdc91e6553c7b 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/HoodieCompactor.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/HoodieCompactor.java
@@ -18,6 +18,7 @@
package org.apache.hudi.table.action.compact;
+import org.apache.hudi.avro.AvroSchemaUtils;
import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.avro.model.HoodieCompactionPlan;
import org.apache.hudi.client.WriteStatus;
@@ -129,6 +130,12 @@ public HoodieData compact(
TaskContextSupplier taskContextSupplier = table.getTaskContextSupplier();
// if this is a MDT, set up the instant range of log reader just like regular MDT snapshot reader.
Option instantRange = CompactHelpers.getInstance().getInstantRange(metaClient);
+ // Since we are using merge handle here, we can directly query the write schema from conf
+ // Write handle provides an option to use overridden write schema as well which is not used by merge handle
+ Schema writerSchema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getWriteSchema()), config.allowOperationMetadataField());
+ if (!table.isMetadataTable()) {
+ AvroSchemaUtils.setLogicalTimestampRepairIfNotSet(table.getHadoopConf(), () -> AvroSchemaUtils.hasTimestampMillisField(writerSchema));
+ }
return context.parallelize(operations).map(operation -> compact(
compactionHandler, metaClient, config, operation, compactionInstantTime, maxInstantTime, instantRange, taskContextSupplier, executionHelper))
.flatMap(List::iterator);
diff --git a/hudi-client/hudi-spark-client/pom.xml b/hudi-client/hudi-spark-client/pom.xml
index fa437494fd9f5..10b98bfb1dbba 100644
--- a/hudi-client/hudi-spark-client/pom.xml
+++ b/hudi-client/hudi-spark-client/pom.xml
@@ -169,6 +169,10 @@
org.pentaho*
+
+ org.codehaus.janino
+ janino
+
@@ -253,6 +257,26 @@
org.apache.ratapache-rat-plugin
+
+ org.codehaus.mojo
+ build-helper-maven-plugin
+ 3.5.0
+
+
+ add-spark32plus-parquet-sources
+ generate-sources
+
+ add-source
+
+
+ ${spark31orEarlier}
+
+ src/parquet/scala
+
+
+
+
+
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java
index 50d8c528594f4..c5d9033157043 100644
--- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java
@@ -116,7 +116,7 @@ public HoodieWriteMetadata> performClustering(final Hood
Stream> writeStatusesStream = FutureUtils.allOf(
clusteringPlan.getInputGroups().stream()
.map(inputGroup -> {
- if (getWriteConfig().getBooleanOrDefault("hoodie.datasource.write.row.writer.enable", true)) {
+ if (getWriteConfig().getBooleanOrDefault("hoodie.datasource.write.row.writer.enable", false)) {
return runClusteringForGroupAsyncAsRow(inputGroup,
clusteringPlan.getStrategy().getStrategyParams(),
shouldPreserveMetadata,
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/utils/SparkInternalSchemaConverter.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/utils/SparkInternalSchemaConverter.java
index 2b14bb3a0665b..7b7c18dd95c66 100644
--- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/utils/SparkInternalSchemaConverter.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/utils/SparkInternalSchemaConverter.java
@@ -61,6 +61,7 @@
import org.apache.spark.sql.types.UserDefinedType;
import org.apache.spark.sql.types.VarcharType;
+import java.lang.reflect.Field;
import java.nio.charset.StandardCharsets;
import java.sql.Date;
import java.util.ArrayList;
@@ -80,6 +81,21 @@ private SparkInternalSchemaConverter() {
public static final String HOODIE_TABLE_PATH = "hoodie.tablePath";
public static final String HOODIE_VALID_COMMITS_LIST = "hoodie.valid.commits.list";
+ /**
+ * Get TimestampNTZType$ using reflection, as it's only available in Spark 3.3+.
+ * Falls back to TimestampType$ if TimestampNTZType is not available.
+ */
+ private static DataType getTimestampNTZType() {
+ try {
+ Class> timestampNTZTypeClass = Class.forName("org.apache.spark.sql.types.TimestampNTZType$");
+ Field moduleField = timestampNTZTypeClass.getField("MODULE$");
+ return (DataType) moduleField.get(null);
+ } catch (ClassNotFoundException | NoSuchFieldException | IllegalAccessException e) {
+ // TimestampNTZType is not available in this Spark version, fall back to TimestampType
+ return TimestampType$.MODULE$;
+ }
+ }
+
public static Type buildTypeFromStructType(DataType sparkType, Boolean firstVisitRoot, AtomicInteger nextId) {
if (sparkType instanceof StructType) {
StructField[] fields = ((StructType) sparkType).fields();
@@ -265,10 +281,14 @@ private static DataType constructSparkSchemaFromType(Type type) {
case DATE:
return DateType$.MODULE$;
case TIME:
+ case TIME_MILLIS:
throw new UnsupportedOperationException(String.format("cannot convert %s type to Spark", type));
case TIMESTAMP:
- // todo support TimeStampNTZ
+ case TIMESTAMP_MILLIS:
return TimestampType$.MODULE$;
+ case LOCAL_TIMESTAMP_MILLIS:
+ case LOCAL_TIMESTAMP_MICROS:
+ return getTimestampNTZType();
case STRING:
return StringType$.MODULE$;
case UUID:
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkFileWriterFactory.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkFileWriterFactory.java
index d2ab83f1481e8..75569a792b207 100644
--- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkFileWriterFactory.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkFileWriterFactory.java
@@ -25,6 +25,7 @@
import org.apache.hudi.common.table.HoodieTableConfig;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.exception.HoodieIOException;
+import org.apache.hudi.io.storage.row.HoodieParquetWriteSupport;
import org.apache.hudi.io.storage.row.HoodieRowParquetConfig;
import org.apache.hudi.io.storage.row.HoodieRowParquetWriteSupport;
@@ -51,7 +52,7 @@ protected HoodieFileWriter newParquetFileWriter(
if (compressionCodecName.isEmpty()) {
compressionCodecName = null;
}
- HoodieRowParquetWriteSupport writeSupport = new HoodieRowParquetWriteSupport(conf,
+ HoodieParquetWriteSupport writeSupport = HoodieRowParquetWriteSupport.getHoodieRowParquetWriteSupport(conf,
HoodieInternalRowUtils.getCachedSchema(schema), filter,
HoodieStorageConfig.newBuilder().fromProperties(config.getProps()).build());
HoodieRowParquetConfig parquetConfig = new HoodieRowParquetConfig(writeSupport,
@@ -71,7 +72,7 @@ protected HoodieFileWriter newParquetFileWriter(
FSDataOutputStream outputStream, Configuration conf, HoodieConfig config, Schema schema) throws IOException {
boolean enableBloomFilter = false;
Option filter = enableBloomFilter ? Option.of(createBloomFilter(config)) : Option.empty();
- HoodieRowParquetWriteSupport writeSupport = new HoodieRowParquetWriteSupport(conf,
+ HoodieParquetWriteSupport writeSupport = HoodieRowParquetWriteSupport.getHoodieRowParquetWriteSupport(conf,
HoodieInternalRowUtils.getCachedSchema(schema), filter,
HoodieStorageConfig.newBuilder().fromProperties(config.getProps()).build());
String compressionCodecName = config.getStringOrDefault(HoodieStorageConfig.PARQUET_COMPRESSION_CODEC_NAME);
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java
index 2a22eacea8c5a..f65a4c43c5283 100644
--- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java
@@ -18,22 +18,25 @@
package org.apache.hudi.io.storage;
-import org.apache.avro.Schema;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
import org.apache.hudi.SparkAdapterSupport$;
+import org.apache.hudi.avro.AvroSchemaUtils;
import org.apache.hudi.avro.HoodieAvroUtils;
-import org.apache.hudi.common.model.HoodieSparkRecord;
import org.apache.hudi.common.bloom.BloomFilter;
import org.apache.hudi.common.model.HoodieFileFormat;
import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieSparkRecord;
import org.apache.hudi.common.util.BaseFileUtils;
-import org.apache.hudi.common.util.collection.ClosableIterator;
-import org.apache.hudi.common.util.collection.CloseableMappingIterator;
+import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.ParquetReaderIterator;
import org.apache.hudi.common.util.ParquetUtils;
import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.collection.ClosableIterator;
+import org.apache.hudi.common.util.collection.CloseableMappingIterator;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.avro.Schema;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
import org.apache.parquet.hadoop.ParquetReader;
import org.apache.parquet.hadoop.api.ReadSupport;
import org.apache.parquet.schema.MessageType;
@@ -53,6 +56,7 @@
import static org.apache.hudi.common.util.TypeUtils.unsafeCast;
import static org.apache.parquet.avro.AvroSchemaConverter.ADD_LIST_ELEMENT_RECORDS;
+import static org.apache.parquet.avro.HoodieAvroParquetSchemaConverter.getAvroSchemaConverter;
public class HoodieSparkParquetReader implements HoodieSparkFileReader {
@@ -60,12 +64,15 @@ public class HoodieSparkParquetReader implements HoodieSparkFileReader {
private final Configuration conf;
private final BaseFileUtils parquetUtils;
private List readerIterators = new ArrayList<>();
+ private Option fileSchemaOption = Option.empty();
+ private Option structTypeOption = Option.empty();
+ private Option schemaOption = Option.empty();
public HoodieSparkParquetReader(Configuration conf, Path path) {
this.path = path;
this.conf = new Configuration(conf);
// Avoid adding record in list element when convert parquet schema to avro schema
- conf.set(ADD_LIST_ELEMENT_RECORDS, "false");
+ this.conf.set(ADD_LIST_ELEMENT_RECORDS, "false");
this.parquetUtils = BaseFileUtils.getInstance(HoodieFileFormat.PARQUET);
}
@@ -118,13 +125,20 @@ private ClosableIterator getInternalRowIterator(Schema readerSchema
if (requestedSchema == null) {
requestedSchema = readerSchema;
}
- StructType readerStructType = HoodieInternalRowUtils.getCachedSchema(readerSchema);
- StructType requestedStructType = HoodieInternalRowUtils.getCachedSchema(requestedSchema);
- conf.set(ParquetReadSupport.PARQUET_READ_SCHEMA, readerStructType.json());
- conf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA(), requestedStructType.json());
- conf.setBoolean(SQLConf.PARQUET_BINARY_AS_STRING().key(), (Boolean) SQLConf.get().getConf(SQLConf.PARQUET_BINARY_AS_STRING()));
- conf.setBoolean(SQLConf.PARQUET_INT96_AS_TIMESTAMP().key(), (Boolean) SQLConf.get().getConf(SQLConf.PARQUET_INT96_AS_TIMESTAMP()));
- ParquetReader reader = ParquetReader.builder((ReadSupport) new ParquetReadSupport(), path)
+ // Set configuration for timestamp_millis type repair (only when not already set).
+ AvroSchemaUtils.setLogicalTimestampRepairIfNotSet(conf, () -> AvroSchemaUtils.hasTimestampMillisField(readerSchema));
+ MessageType fileSchema = getFileSchema();
+ Schema nonNullSchema = AvroSchemaUtils.getNonNullTypeFromUnion(requestedSchema);
+ Option messageSchema = Option.of(getAvroSchemaConverter(conf).convert(nonNullSchema));
+ Pair readerSchemas =
+ SparkAdapterSupport$.MODULE$.sparkAdapter().getReaderSchemas(conf, readerSchema, requestedSchema, fileSchema);
+ conf.set(ParquetReadSupport.PARQUET_READ_SCHEMA, readerSchemas.getLeft().json());
+ conf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA(), readerSchemas.getRight().json());
+ conf.set(SQLConf.PARQUET_BINARY_AS_STRING().key(), SQLConf.get().getConf(SQLConf.PARQUET_BINARY_AS_STRING()).toString());
+ conf.set(SQLConf.PARQUET_INT96_AS_TIMESTAMP().key(), SQLConf.get().getConf(SQLConf.PARQUET_INT96_AS_TIMESTAMP()).toString());
+ ParquetReader reader = ParquetReader.builder(
+ (ReadSupport) SparkAdapterSupport$.MODULE$.sparkAdapter().getParquetReadSupport(conf, messageSchema),
+ path)
.withConf(conf)
.build();
ParquetReaderIterator parquetReaderIterator = new ParquetReaderIterator<>(reader);
@@ -132,21 +146,44 @@ private ClosableIterator getInternalRowIterator(Schema readerSchema
return parquetReaderIterator;
}
+ private MessageType getFileSchema() {
+ if (!fileSchemaOption.isPresent()) {
+ MessageType messageType = ((ParquetUtils) parquetUtils).readSchema(conf, path);
+ fileSchemaOption = Option.of(messageType);
+ }
+ return fileSchemaOption.get();
+ }
+
@Override
public Schema getSchema() {
- // Some types in avro are not compatible with parquet.
- // Avro only supports representing Decimals as fixed byte array
- // and therefore if we convert to Avro directly we'll lose logical type-info.
- MessageType messageType = ((ParquetUtils) parquetUtils).readSchema(conf, path);
- StructType structType = new ParquetToSparkSchemaConverter(conf).convert(messageType);
- return SparkAdapterSupport$.MODULE$.sparkAdapter()
- .getAvroSchemaConverters()
- .toAvroType(structType, true, messageType.getName(), StringUtils.EMPTY_STRING);
+ if (!schemaOption.isPresent()) {
+ // Some types in avro are not compatible with parquet.
+ // Avro only supports representing Decimals as fixed byte array
+ // and therefore if we convert to Avro directly we'll lose logical type-info.
+ MessageType messageType = getFileSchema();
+ StructType structType = getStructSchema();
+ schemaOption = Option.of(SparkAdapterSupport$.MODULE$.sparkAdapter()
+ .getAvroSchemaConverters()
+ .toAvroType(structType, true, messageType.getName(), StringUtils.EMPTY_STRING));
+ }
+ return schemaOption.get();
+ }
+
+ protected StructType getStructSchema() {
+ if (!structTypeOption.isPresent()) {
+ MessageType messageType = getFileSchema();
+ structTypeOption = Option.of(convertToStruct(messageType));
+ }
+ return structTypeOption.get();
+ }
+
+ private StructType convertToStruct(MessageType messageType) {
+ return new ParquetToSparkSchemaConverter(conf).convert(messageType);
}
@Override
public void close() {
- readerIterators.forEach(ParquetReaderIterator::close);
+ readerIterators.forEach(it -> it.close());
}
@Override
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetStreamWriter.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetStreamWriter.java
index 5c6f85af290d0..2e7986a8b0582 100644
--- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetStreamWriter.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetStreamWriter.java
@@ -23,8 +23,8 @@
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.io.storage.row.HoodieParquetWriteSupport;
import org.apache.hudi.io.storage.row.HoodieRowParquetConfig;
-import org.apache.hudi.io.storage.row.HoodieRowParquetWriteSupport;
import org.apache.hudi.parquet.io.OutputStreamBackedOutputFile;
import org.apache.parquet.hadoop.ParquetFileWriter;
import org.apache.parquet.hadoop.ParquetWriter;
@@ -35,7 +35,7 @@
public class HoodieSparkParquetStreamWriter implements HoodieSparkFileWriter, AutoCloseable {
private final ParquetWriter writer;
- private final HoodieRowParquetWriteSupport writeSupport;
+ private final HoodieParquetWriteSupport writeSupport;
public HoodieSparkParquetStreamWriter(FSDataOutputStream outputStream,
HoodieRowParquetConfig parquetConfig) throws IOException {
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetWriter.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetWriter.java
index d601e6ded3e12..d9481c217f777 100644
--- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetWriter.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetWriter.java
@@ -22,8 +22,8 @@
import org.apache.hudi.common.engine.TaskContextSupplier;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.io.storage.row.HoodieParquetWriteSupport;
import org.apache.hudi.io.storage.row.HoodieRowParquetConfig;
-import org.apache.hudi.io.storage.row.HoodieRowParquetWriteSupport;
import org.apache.spark.sql.catalyst.InternalRow;
import org.apache.spark.unsafe.types.UTF8String;
@@ -43,7 +43,7 @@ public class HoodieSparkParquetWriter extends HoodieBaseParquetWriter seqIdGenerator;
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieInternalRowFileWriterFactory.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieInternalRowFileWriterFactory.java
index 8a61c7c44d900..406e7d09995c9 100644
--- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieInternalRowFileWriterFactory.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieInternalRowFileWriterFactory.java
@@ -66,8 +66,8 @@ private static HoodieInternalRowFileWriter newParquetInternalRowFileWriter(Path
Option bloomFilterOpt
)
throws IOException {
- HoodieRowParquetWriteSupport writeSupport =
- new HoodieRowParquetWriteSupport(table.getHadoopConf(), structType, bloomFilterOpt, writeConfig.getStorageConfig());
+ HoodieParquetWriteSupport writeSupport = HoodieRowParquetWriteSupport
+ .getHoodieRowParquetWriteSupport(table.getHadoopConf(), structType, bloomFilterOpt, writeConfig);
return new HoodieInternalRowParquetWriter(
path,
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieInternalRowParquetWriter.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieInternalRowParquetWriter.java
index a7cacd055a63c..dcd99a8bd1b5f 100644
--- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieInternalRowParquetWriter.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieInternalRowParquetWriter.java
@@ -32,9 +32,9 @@
public class HoodieInternalRowParquetWriter extends HoodieBaseParquetWriter
implements HoodieInternalRowFileWriter {
- private final HoodieRowParquetWriteSupport writeSupport;
+ private final HoodieParquetWriteSupport writeSupport;
- public HoodieInternalRowParquetWriter(Path file, HoodieParquetConfig parquetConfig)
+ public HoodieInternalRowParquetWriter(Path file, HoodieParquetConfig parquetConfig)
throws IOException {
super(file, parquetConfig);
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieParquetWriteSupport.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieParquetWriteSupport.java
new file mode 100644
index 0000000000000..a9f91147251c8
--- /dev/null
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieParquetWriteSupport.java
@@ -0,0 +1,43 @@
+/*
+ * 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.io.storage.row;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.spark.sql.execution.datasources.parquet.ParquetWriteSupport;
+import org.apache.spark.unsafe.types.UTF8String;
+
+/**
+ * Common abstract base class for Hudi's Parquet write support implementations.
+ * Extends Spark's ParquetWriteSupport and provides additional methods for
+ * Hadoop configuration access and bloom filter key addition.
+ */
+public abstract class HoodieParquetWriteSupport extends ParquetWriteSupport {
+
+ /**
+ * Get the Hadoop configuration used by this write support.
+ * @return Hadoop Configuration
+ */
+ public abstract Configuration getHadoopConf();
+
+ /**
+ * Add a record key to the bloom filter (if enabled).
+ * @param recordKey the record key to add
+ */
+ public abstract void add(UTF8String recordKey);
+}
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowParquetConfig.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowParquetConfig.java
index f5f6d7b0a5bb1..8957aabebfe82 100644
--- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowParquetConfig.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowParquetConfig.java
@@ -26,9 +26,9 @@
/**
* ParquetConfig for datasource implementation with {@link org.apache.hudi.client.model.HoodieInternalRow}.
*/
-public class HoodieRowParquetConfig extends HoodieParquetConfig {
+public class HoodieRowParquetConfig extends HoodieParquetConfig {
- public HoodieRowParquetConfig(HoodieRowParquetWriteSupport writeSupport, CompressionCodecName compressionCodecName,
+ public HoodieRowParquetConfig(HoodieParquetWriteSupport writeSupport, CompressionCodecName compressionCodecName,
int blockSize, int pageSize, long maxFileSize, Configuration hadoopConf,
double compressionRatio, boolean enableDictionary) {
super(writeSupport, compressionCodecName, blockSize, pageSize, maxFileSize, hadoopConf, compressionRatio, enableDictionary);
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowParquetWriteSupport.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowParquetWriteSupport.java
index 3a1b6d000becc..e5d4dd556704b 100644
--- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowParquetWriteSupport.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowParquetWriteSupport.java
@@ -18,13 +18,17 @@
package org.apache.hudi.io.storage.row;
-import org.apache.hadoop.conf.Configuration;
import org.apache.hudi.avro.HoodieBloomFilterWriteSupport;
import org.apache.hudi.common.bloom.BloomFilter;
+import org.apache.hudi.common.config.HoodieConfig;
import org.apache.hudi.common.config.HoodieStorageConfig;
import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.ReflectionUtils;
+
+import org.apache.hudi.HoodieSparkUtils;
+
+import org.apache.hadoop.conf.Configuration;
import org.apache.parquet.hadoop.api.WriteSupport;
-import org.apache.spark.sql.execution.datasources.parquet.ParquetWriteSupport;
import org.apache.spark.sql.types.StructType;
import org.apache.spark.unsafe.types.UTF8String;
@@ -34,24 +38,26 @@
import static org.apache.hudi.common.config.HoodieStorageConfig.PARQUET_FIELD_ID_WRITE_ENABLED;
/**
- * Hoodie Write Support for directly writing Row to Parquet.
+ * Hoodie Write Support for directly writing Row to Parquet. This is the legacy version for Spark 3.3 and earlier.
+ * For Spark 3.4+, use Spark34PlusHoodieRowParquetWriteSupport which has better logical type support.
*/
-public class HoodieRowParquetWriteSupport extends ParquetWriteSupport {
+public class HoodieRowParquetWriteSupport extends HoodieParquetWriteSupport {
private final Configuration hadoopConf;
private final Option> bloomFilterWriteSupportOpt;
- public HoodieRowParquetWriteSupport(Configuration conf, StructType structType, Option bloomFilterOpt, HoodieStorageConfig config) {
+ public HoodieRowParquetWriteSupport(Configuration conf, StructType structType, Option bloomFilterOpt, HoodieConfig config) {
Configuration hadoopConf = new Configuration(conf);
- hadoopConf.set("spark.sql.parquet.writeLegacyFormat", config.getString(HoodieStorageConfig.PARQUET_WRITE_LEGACY_FORMAT_ENABLED));
- hadoopConf.set("spark.sql.parquet.outputTimestampType", config.getString(HoodieStorageConfig.PARQUET_OUTPUT_TIMESTAMP_TYPE));
- hadoopConf.set("spark.sql.parquet.fieldId.write.enabled", config.getString(PARQUET_FIELD_ID_WRITE_ENABLED));
+ hadoopConf.set("spark.sql.parquet.writeLegacyFormat", config.getStringOrDefault(HoodieStorageConfig.PARQUET_WRITE_LEGACY_FORMAT_ENABLED));
+ hadoopConf.set("spark.sql.parquet.outputTimestampType", config.getStringOrDefault(HoodieStorageConfig.PARQUET_OUTPUT_TIMESTAMP_TYPE));
+ hadoopConf.set("spark.sql.parquet.fieldId.write.enabled", config.getStringOrDefault(PARQUET_FIELD_ID_WRITE_ENABLED));
setSchema(structType, hadoopConf);
this.hadoopConf = hadoopConf;
this.bloomFilterWriteSupportOpt = bloomFilterOpt.map(HoodieBloomFilterRowWriteSupport::new);
}
+ @Override
public Configuration getHadoopConf() {
return hadoopConf;
}
@@ -65,6 +71,7 @@ public WriteSupport.FinalizedWriteContext finalizeWrite() {
return new WriteSupport.FinalizedWriteContext(extraMetadata);
}
+ @Override
public void add(UTF8String recordKey) {
this.bloomFilterWriteSupportOpt.ifPresent(bloomFilterWriteSupport ->
bloomFilterWriteSupport.addKey(recordKey));
@@ -89,4 +96,22 @@ protected UTF8String dereference(UTF8String key) {
}
}
+ public static HoodieParquetWriteSupport getHoodieRowParquetWriteSupport(Configuration conf, StructType structType,
+ Option bloomFilterOpt, HoodieConfig config) {
+ if (HoodieSparkUtils.gteqSpark3_4()) {
+ return (HoodieParquetWriteSupport) ReflectionUtils.loadClass(
+ "org.apache.hudi.io.storage.row.Spark34PlusHoodieRowParquetWriteSupport",
+ new Class>[] {Configuration.class, StructType.class, Option.class, HoodieConfig.class},
+ conf, structType, bloomFilterOpt, config);
+ }
+
+ String className = config.getStringOrDefault(HoodieStorageConfig.HOODIE_PARQUET_SPARK_ROW_WRITE_SUPPORT_CLASS);
+ if (className.equals("org.apache.hudi.io.storage.row.Spark34PlusHoodieRowParquetWriteSupport")) {
+ className = "org.apache.hudi.io.storage.row.HoodieRowParquetWriteSupport";
+ }
+ return (HoodieParquetWriteSupport) ReflectionUtils.loadClass(
+ className,
+ new Class>[] {Configuration.class, StructType.class, Option.class, HoodieConfig.class},
+ conf, structType, bloomFilterOpt, config);
+ }
}
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/ParquetBootstrapMetadataHandler.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/ParquetBootstrapMetadataHandler.java
index 2c3ddfdcda2ce..f6874e28d5d95 100644
--- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/ParquetBootstrapMetadataHandler.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/ParquetBootstrapMetadataHandler.java
@@ -40,7 +40,6 @@
import org.apache.avro.generic.GenericData;
import org.apache.avro.generic.GenericRecord;
import org.apache.hadoop.fs.Path;
-import org.apache.parquet.avro.AvroSchemaConverter;
import org.apache.parquet.format.converter.ParquetMetadataConverter;
import org.apache.parquet.hadoop.ParquetFileReader;
import org.apache.parquet.hadoop.metadata.ParquetMetadata;
@@ -56,6 +55,7 @@
import java.util.function.Function;
import static org.apache.hudi.io.HoodieBootstrapHandle.METADATA_BOOTSTRAP_RECORD_SCHEMA;
+import static org.apache.parquet.avro.HoodieAvroParquetSchemaConverter.getAvroSchemaConverter;
class ParquetBootstrapMetadataHandler extends BaseBootstrapMetadataHandler {
@@ -68,7 +68,7 @@ Schema getAvroSchema(Path sourceFilePath) throws IOException {
ParquetMetadata readFooter = ParquetFileReader.readFooter(table.getHadoopConf(), sourceFilePath,
ParquetMetadataConverter.NO_FILTER);
MessageType parquetSchema = readFooter.getFileMetaData().getSchema();
- return new AvroSchemaConverter().convert(parquetSchema);
+ return getAvroSchemaConverter(table.getHadoopConf()).convert(parquetSchema);
}
@Override
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java
index 196c6c5c52b23..969c68a41d7e5 100644
--- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java
@@ -18,6 +18,7 @@
package org.apache.hudi.table.action.commit;
+import org.apache.hudi.avro.AvroSchemaUtils;
import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.client.utils.SparkPartitionUtils;
import org.apache.hudi.client.clustering.update.strategy.SparkAllowUpdateStrategy;
@@ -32,6 +33,7 @@
import org.apache.hudi.common.model.HoodieRecordLocation;
import org.apache.hudi.common.model.HoodieWriteStat;
import org.apache.hudi.common.model.WriteOperationType;
+import org.apache.hudi.common.table.TableSchemaResolver;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.util.CommitUtils;
@@ -42,6 +44,7 @@
import org.apache.hudi.data.HoodieJavaPairRDD;
import org.apache.hudi.data.HoodieJavaRDD;
import org.apache.hudi.exception.HoodieCommitException;
+import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.exception.HoodieUpsertException;
import org.apache.hudi.execution.SparkLazyInsertIterable;
@@ -252,6 +255,22 @@ private HoodieData mapPartitionsAsRDD(HoodieData> d
// Partition only
partitionedRDD = mappedRDD.partitionBy(partitioner);
}
+
+ if (!table.isMetadataTable() && table.getMetaClient().getActiveTimeline().getCommitsTimeline().filterCompletedInstants().countInstants() > 0) {
+ TableSchemaResolver schemaResolver = new TableSchemaResolver(table.getMetaClient());
+ try {
+ AvroSchemaUtils.setLogicalTimestampRepairIfNotSet(table.getHadoopConf(), () -> {
+ try {
+ return AvroSchemaUtils.hasTimestampMillisField(schemaResolver.getTableAvroSchema());
+ } catch (Exception e) {
+ return true;
+ }
+ });
+ } catch (Exception e) {
+ throw new HoodieException("Failed to set logical ts related configs", e);
+ }
+ }
+
return HoodieJavaRDD.of(partitionedRDD.map(Tuple2::_2).mapPartitionsWithIndex((partition, recordItr) -> {
if (WriteOperationType.isChangingRecords(operationType)) {
return handleUpsertPartition(instantTime, partition, recordItr, partitioner);
diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/AvroConversionUtils.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/AvroConversionUtils.scala
index d84679eaf923a..7948b1a4baa05 100644
--- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/AvroConversionUtils.scala
+++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/AvroConversionUtils.scala
@@ -84,7 +84,7 @@ object AvroConversionUtils {
recordNamespace: String): Row => GenericRecord = {
val serde = sparkAdapter.createSparkRowSerDe(sourceSqlType)
val avroSchema = AvroConversionUtils.convertStructTypeToAvroSchema(sourceSqlType, structName, recordNamespace)
- val nullable = AvroSchemaUtils.resolveNullableSchema(avroSchema) != avroSchema
+ val nullable = AvroSchemaUtils.getNonNullTypeFromUnion(avroSchema) != avroSchema
val converter = AvroConversionUtils.createInternalRowToAvroConverter(sourceSqlType, avroSchema, nullable)
@@ -97,8 +97,11 @@ object AvroConversionUtils {
* TODO convert directly from GenericRecord into InternalRow instead
*/
def createDataFrame(rdd: RDD[GenericRecord], schemaStr: String, ss: SparkSession): Dataset[Row] = {
- if (rdd.isEmpty()) {
- ss.emptyDataFrame
+ // Avoid calling isEmpty() which can cause serialization issues with Ordering$Reverse
+ // Check partition count instead, which doesn't require task serialization
+ val structType = convertAvroSchemaToStructType(new Schema.Parser().parse(schemaStr))
+ if (rdd.getNumPartitions == 0) {
+ ss.createDataFrame(ss.sparkContext.emptyRDD[Row], structType)
} else {
ss.createDataFrame(rdd.mapPartitions { records =>
if (records.isEmpty) Iterator.empty
@@ -108,7 +111,7 @@ object AvroConversionUtils {
val converter = createConverterToRow(schema, dataType)
records.map { r => converter(r) }
}
- }, convertAvroSchemaToStructType(new Schema.Parser().parse(schemaStr)))
+ }, structType)
}
}
diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieSparkUtils.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieSparkUtils.scala
index a0fe879b3dbea..efd32bae76487 100644
--- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieSparkUtils.scala
+++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieSparkUtils.scala
@@ -93,7 +93,7 @@ object HoodieSparkUtils extends SparkAdapterSupport with SparkVersionsSupport wi
// making Spark deserialize its internal representation [[InternalRow]] into [[Row]] for subsequent conversion
// (and back)
val sameSchema = writerAvroSchema.equals(readerAvroSchema)
- val nullable = AvroSchemaUtils.resolveNullableSchema(writerAvroSchema) != writerAvroSchema
+ val nullable = AvroSchemaUtils.getNonNullTypeFromUnion(writerAvroSchema) != writerAvroSchema
// NOTE: We have to serialize Avro schema, and then subsequently parse it on the executor node, since Spark
// serializer is not able to digest it
@@ -152,7 +152,7 @@ object HoodieSparkUtils extends SparkAdapterSupport with SparkVersionsSupport wi
// making Spark deserialize its internal representation [[InternalRow]] into [[Row]] for subsequent conversion
// (and back)
val sameSchema = writerAvroSchema.equals(readerAvroSchema)
- val nullable = AvroSchemaUtils.resolveNullableSchema(writerAvroSchema) != writerAvroSchema
+ val nullable = AvroSchemaUtils.getNonNullTypeFromUnion(writerAvroSchema) != writerAvroSchema
// NOTE: We have to serialize Avro schema, and then subsequently parse it on the executor node, since Spark
// serializer is not able to digest it
diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkConversionUtils.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkConversionUtils.scala
index 799bda2f8e5ba..736f69fd17157 100644
--- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkConversionUtils.scala
+++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkConversionUtils.scala
@@ -27,7 +27,9 @@ import org.apache.spark.sql.{DataFrameUtil, Dataset, Row, SparkSession}
object SparkConversionUtils {
def createDataFrame[T](rdd: RDD[HoodieRecord[T]], ss: SparkSession, structType: StructType): Dataset[Row] = {
- if (rdd.isEmpty()) {
+ // Avoid calling isEmpty() which can cause serialization issues with Ordering$Reverse
+ // Check partition count instead, which doesn't require task serialization
+ if (rdd.getNumPartitions == 0) {
ss.emptyDataFrame
} else {
DataFrameUtil.createFromInternalRows(ss, structType, rdd.map(_.getData.asInstanceOf[InternalRow]))
diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/HoodieSparkKryoRegistrar.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/HoodieSparkKryoRegistrar.scala
index dd98227d4407c..b22a0f070364f 100644
--- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/HoodieSparkKryoRegistrar.scala
+++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/HoodieSparkKryoRegistrar.scala
@@ -89,6 +89,8 @@ object HoodieSparkKryoRegistrar {
private val KRYO_USER_REGISTRATORS = "spark.kryo.registrator"
def register(conf: SparkConf): SparkConf = {
- conf.set(KRYO_USER_REGISTRATORS, Seq(classOf[HoodieSparkKryoRegistrar].getName).mkString(","))
+ // Use class name directly to avoid Scala collection binary compatibility issues
+ // when compiled with Scala 2.13 but running with Spark 3.5 (Scala 2.12)
+ conf.set(KRYO_USER_REGISTRATORS, classOf[HoodieSparkKryoRegistrar].getName)
}
}
diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieParquetFileFormatHelper.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieParquetFileFormatHelper.scala
similarity index 78%
rename from hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieParquetFileFormatHelper.scala
rename to hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieParquetFileFormatHelper.scala
index 599bbebe4f6c4..3e13e3bf7f9de 100644
--- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieParquetFileFormatHelper.scala
+++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieParquetFileFormatHelper.scala
@@ -1,34 +1,41 @@
/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
*
- * http://www.apache.org/licenses/LICENSE-2.0
+ * http://www.apache.org/licenses/LICENSE-2.0
*
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * 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.
+ * 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.parquet.hadoop.metadata.FileMetaData
-import org.apache.spark.sql.types.{ArrayType, DataType, MapType, StructField, StructType}
+import org.apache.spark.sql.types._
object HoodieParquetFileFormatHelper {
-
def buildImplicitSchemaChangeInfo(hadoopConf: Configuration,
parquetFileMetaData: FileMetaData,
requiredSchema: StructType): (java.util.Map[Integer, org.apache.hudi.common.util.collection.Pair[DataType, DataType]], StructType) = {
- val implicitTypeChangeInfo: java.util.Map[Integer, org.apache.hudi.common.util.collection.Pair[DataType, DataType]] = new java.util.HashMap()
val convert = new ParquetToSparkSchemaConverter(hadoopConf)
val fileStruct = convert.convert(parquetFileMetaData.getSchema)
+ buildImplicitSchemaChangeInfo(fileStruct, requiredSchema)
+ }
+
+ def buildImplicitSchemaChangeInfo(fileStruct: StructType,
+ requiredSchema: StructType): (java.util.Map[Integer, org.apache.hudi.common.util.collection.Pair[DataType, DataType]], StructType) = {
+ val implicitTypeChangeInfo: java.util.Map[Integer, org.apache.hudi.common.util.collection.Pair[DataType, DataType]] = new java.util.HashMap()
+
val fileStructMap = fileStruct.fields.map(f => (f.name, f.dataType)).toMap
// if there are missing fields or if field's data type needs to be changed while reading, we handle it here.
val sparkRequestStructFields = requiredSchema.map(f => {
@@ -45,6 +52,7 @@ object HoodieParquetFileFormatHelper {
}
def isDataTypeEqual(requiredType: DataType, fileType: DataType): Boolean = (requiredType, fileType) match {
+
case (requiredType, fileType) if requiredType == fileType => true
case (ArrayType(rt, _), ArrayType(ft, _)) =>
diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala
index 1c6111afe47f3..11c2179dc0a37 100644
--- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala
+++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala
@@ -19,14 +19,18 @@
package org.apache.spark.sql.hudi
import org.apache.avro.Schema
+import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.Path
import org.apache.hudi.client.utils.SparkRowSerDe
import org.apache.hudi.common.table.HoodieTableMetaClient
+
+import org.apache.parquet.schema.MessageType
import org.apache.spark.sql._
import org.apache.spark.sql.avro.{HoodieAvroDeserializer, HoodieAvroSchemaConverters, HoodieAvroSerializer}
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, InterpretedPredicate}
+import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Expression, InterpretedPredicate}
import org.apache.spark.sql.catalyst.parser.ParserInterface
import org.apache.spark.sql.catalyst.planning.PhysicalOperation
import org.apache.spark.sql.catalyst.plans.logical.{Command, LogicalPlan}
@@ -36,6 +40,7 @@ import org.apache.spark.sql.execution.datasources._
import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat
import org.apache.spark.sql.parser.HoodieExtendedParserInterface
import org.apache.spark.sql.sources.{BaseRelation, Filter}
+import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types.{DataType, Metadata, StructType}
import org.apache.spark.sql.vectorized.{ColumnVector, ColumnarBatch}
import org.apache.spark.storage.StorageLevel
@@ -48,10 +53,21 @@ import java.util.{Locale, TimeZone}
trait SparkAdapter extends Serializable {
/**
- * Checks whether provided instance of [[InternalRow]] is actually an instance of [[ColumnarBatchRow]]
+ * Checks whether provided instance of [[InternalRow]] is actually an instance of [[org.apache.spark.sql.vectorized.ColumnarBatchRow]]
*/
def isColumnarBatchRow(r: InternalRow): Boolean
+ def getParquetReadSupport(conf: Configuration,
+ messageSchema: org.apache.hudi.common.util.Option[MessageType]):
+ org.apache.parquet.hadoop.api.ReadSupport[_]
+
+ def repairSchemaIfSpecified(shouldRepair: Boolean,
+ fileSchema: MessageType,
+ tableSchemaOpt: org.apache.hudi.common.util.Option[MessageType]): MessageType
+
+ def getReaderSchemas(conf: Configuration, readerSchema: Schema, requestedSchema: Schema, fileSchema: MessageType):
+ org.apache.hudi.common.util.collection.Pair[StructType, StructType]
+
/**
* Creates Catalyst [[Metadata]] for Hudi's meta-fields (designating these w/
* [[METADATA_COL_ATTR_KEY]] if available (available in Spark >= 3.2)
@@ -65,7 +81,7 @@ trait SparkAdapter extends Serializable {
/**
* Returns an instance of [[HoodieCatalogUtils]] providing for common utils operating on Spark's
- * [[TableCatalog]]s
+ * [[org.apache.spark.sql.connector.catalog.TableCatalog]]s
*/
def getCatalogUtils: HoodieCatalogUtils
@@ -169,7 +185,7 @@ trait SparkAdapter extends Serializable {
/**
* Create instance of [[ParquetFileFormat]]
*/
- def createLegacyHoodieParquetFileFormat(appendPartitionValues: Boolean): Option[ParquetFileFormat]
+ def createLegacyHoodieParquetFileFormat(appendPartitionValues: Boolean, tableAvroSchema: Schema, hasTimestampMillisFieldInTableSchema: Boolean): Option[ParquetFileFormat]
def makeColumnarBatch(vectors: Array[ColumnVector], numRows: Int): ColumnarBatch
@@ -200,7 +216,7 @@ trait SparkAdapter extends Serializable {
metadataColumns: Seq[AttributeReference] = Seq.empty): FileScanRDD
/**
- * Extract condition in [[DeleteFromTable]]
+ * Extract condition in [[org.apache.spark.sql.catalyst.plans.logical.DeleteFromTable]]
* SPARK-38626 condition is no longer Option in Spark 3.3
*/
def extractDeleteCondition(deleteFromTable: Command): Expression
@@ -214,4 +230,33 @@ trait SparkAdapter extends Serializable {
* Tries to translate a Catalyst Expression into data source Filter
*/
def translateFilter(predicate: Expression, supportNestedPredicatePushdown: Boolean = false): Option[Filter]
+
+ /**
+ * @param sparkSession Spark session (required for Spark 3.5 to access Analyzer)
+ * @param tableName table name
+ * @param expected expected attributes
+ * @param query query logical plan
+ * @param byName whether to match by name
+ * @param conf SQL configuration
+ * @return resolved logical plan
+ */
+ def resolveOutputColumns(sparkSession: SparkSession,
+ tableName: String,
+ expected: Seq[Attribute],
+ query: LogicalPlan,
+ byName: Boolean,
+ conf: SQLConf): LogicalPlan = {
+ // Default implementation delegates to CatalystPlanUtils (for Spark < 3.5)
+ getCatalystPlanUtils.resolveOutputColumns(tableName, expected, query, byName, conf)
+ }
+
+ def isTimestampNTZType(dataType: DataType): Boolean
+
+ def getDateTimeRebaseMode(): Object = throw new UnsupportedOperationException("getDateTimeRebaseMode not supported on this Spark version")
+
+ def isLegacyBehaviorPolicy(value: Object): Boolean = throw new UnsupportedOperationException("isLegacyBehaviorPolicy not supported on this Spark version")
+
+ def applyFieldIdToType(parquetType: org.apache.parquet.schema.Type, structField: org.apache.spark.sql.types.StructField): org.apache.parquet.schema.Type = {
+ parquetType
+ }
}
diff --git a/hudi-client/hudi-spark-client/src/parquet/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieLegacyParquetFileFormatHelper.scala b/hudi-client/hudi-spark-client/src/parquet/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieLegacyParquetFileFormatHelper.scala
new file mode 100644
index 0000000000000..b5669019bd066
--- /dev/null
+++ b/hudi-client/hudi-spark-client/src/parquet/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieLegacyParquetFileFormatHelper.scala
@@ -0,0 +1,121 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.parquet
+
+import org.apache.spark.sql.HoodieSchemaUtils
+import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection
+import org.apache.spark.sql.catalyst.expressions.{ArrayTransform, Attribute, Cast, CreateNamedStruct, CreateStruct, Expression, GetStructField, LambdaFunction, Literal, MapEntries, MapFromEntries, NamedLambdaVariable, UnsafeProjection}
+import org.apache.spark.sql.types._
+
+object HoodieLegacyParquetFileFormatHelper {
+ def generateUnsafeProjection(fullSchema: Seq[Attribute],
+ timeZoneId: Option[String],
+ typeChangeInfos: java.util.Map[Integer, org.apache.hudi.common.util.collection.Pair[DataType, DataType]],
+ requiredSchema: StructType,
+ partitionSchema: StructType,
+ schemaUtils: HoodieSchemaUtils): UnsafeProjection = {
+ val addedCastCache = scala.collection.mutable.HashMap.empty[(DataType, DataType), Boolean]
+
+ def hasUnsupportedConversion(src: DataType, dst: DataType): Boolean = {
+ addedCastCache.getOrElseUpdate((src, dst), {
+ (src, dst) match {
+ case (FloatType, DoubleType) => true
+ case (IntegerType, DecimalType()) => true
+ case (LongType, DecimalType()) => true
+ case (FloatType, DecimalType()) => true
+ case (DoubleType, DecimalType()) => true
+ case (StringType, DecimalType()) => true
+ case (StringType, DateType) => true
+ case (StructType(srcFields), StructType(dstFields)) =>
+ srcFields.zip(dstFields).exists { case (sf, df) => hasUnsupportedConversion(sf.dataType, df.dataType) }
+ case (ArrayType(sElem, _), ArrayType(dElem, _)) =>
+ hasUnsupportedConversion(sElem, dElem)
+ case (MapType(sKey, sVal, _), MapType(dKey, dVal, _)) =>
+ hasUnsupportedConversion(sKey, dKey) || hasUnsupportedConversion(sVal, dVal)
+ case _ => false
+ }
+ })
+ }
+
+ def recursivelyCastExpressions(expr: Expression, srcType: DataType, dstType: DataType): Expression = {
+ lazy val needTimeZone = Cast.needsTimeZone(srcType, dstType)
+ (srcType, dstType) match {
+ case (FloatType, DoubleType) =>
+ val toStr = Cast(expr, StringType, if (needTimeZone) timeZoneId else None)
+ Cast(toStr, dstType, if (needTimeZone) timeZoneId else None)
+ case (IntegerType | LongType | FloatType | DoubleType, dec: DecimalType) =>
+ val toStr = Cast(expr, StringType, if (needTimeZone) timeZoneId else None)
+ Cast(toStr, dec, if (needTimeZone) timeZoneId else None)
+ case (StringType, dec: DecimalType) =>
+ Cast(expr, dec, if (needTimeZone) timeZoneId else None)
+ case (StringType, DateType) =>
+ Cast(expr, DateType, if (needTimeZone) timeZoneId else None)
+ case (s: StructType, d: StructType) if hasUnsupportedConversion(s, d) =>
+ val structFields = s.fields.zip(d.fields).zipWithIndex.map {
+ case ((srcField, dstField), i) =>
+ val child = GetStructField(expr, i, Some(dstField.name))
+ recursivelyCastExpressions(child, srcField.dataType, dstField.dataType)
+ }
+ CreateNamedStruct(d.fields.zip(structFields).flatMap {
+ case (f, c) => Seq(Literal(f.name), c)
+ })
+ case (ArrayType(sElementType, containsNull), ArrayType(dElementType, _)) if hasUnsupportedConversion(sElementType, dElementType) =>
+ val lambdaVar = NamedLambdaVariable("element", sElementType, containsNull)
+ val body = recursivelyCastExpressions(lambdaVar, sElementType, dElementType)
+ val func = LambdaFunction(body, Seq(lambdaVar))
+ ArrayTransform(expr, func)
+ case (MapType(sKeyType, sValType, vnull), MapType(dKeyType, dValType, _))
+ if hasUnsupportedConversion(sKeyType, dKeyType) || hasUnsupportedConversion(sValType, dValType) =>
+ val kv = NamedLambdaVariable("kv", new StructType()
+ .add("key", sKeyType, nullable = false)
+ .add("value", sValType, nullable = vnull), nullable = false)
+ val newKey = recursivelyCastExpressions(GetStructField(kv, 0), sKeyType, dKeyType)
+ val newVal = recursivelyCastExpressions(GetStructField(kv, 1), sValType, dValType)
+ val entry = CreateStruct(Seq(newKey, newVal))
+ val func = LambdaFunction(entry, Seq(kv))
+ val transformed = ArrayTransform(MapEntries(expr), func)
+ MapFromEntries(transformed)
+ case _ =>
+ // most cases should be covered here we only need to do the recursive work for float to double
+ Cast(expr, dstType, if (needTimeZone) timeZoneId else None)
+ }
+ }
+
+ if (typeChangeInfos.isEmpty) {
+ GenerateUnsafeProjection.generate(fullSchema, fullSchema)
+ } else {
+ // find type changed.
+ val newSchema = 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
+ })
+ val newFullSchema = newSchema.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
+ recursivelyCastExpressions(attr, srcType, dstType)
+ } else attr
+ }
+ GenerateUnsafeProjection.generate(castSchema, newFullSchema)
+ }
+ }
+}
diff --git a/hudi-client/hudi-spark-client/src/parquet/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieParquetReadSupport.scala b/hudi-client/hudi-spark-client/src/parquet/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieParquetReadSupport.scala
new file mode 100644
index 0000000000000..f19a2952a6637
--- /dev/null
+++ b/hudi-client/hudi-spark-client/src/parquet/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieParquetReadSupport.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.execution.datasources.parquet
+
+import org.apache.hudi.SparkAdapterSupport
+import org.apache.hudi.common.util.{Option => HOption}
+import org.apache.hudi.common.util.ValidationUtils
+import org.apache.parquet.hadoop.api.InitContext
+import org.apache.parquet.hadoop.api.ReadSupport.ReadContext
+import org.apache.parquet.schema.{GroupType, MessageType, Type, Types}
+import org.apache.spark.sql.catalyst.util.RebaseDateTime.RebaseSpec
+
+import java.time.ZoneId
+import scala.collection.JavaConverters._
+
+class HoodieParquetReadSupport(convertTz: Option[ZoneId],
+ enableVectorizedReader: Boolean,
+ val enableTimestampFieldRepair: Boolean,
+ datetimeRebaseSpec: RebaseSpec,
+ int96RebaseSpec: RebaseSpec,
+ tableSchemaOpt: HOption[MessageType] = HOption.empty())
+ extends ParquetReadSupport(convertTz, enableVectorizedReader, datetimeRebaseSpec, int96RebaseSpec) with SparkAdapterSupport {
+
+ override def init(context: InitContext): ReadContext = {
+ val readContext = super.init(context)
+ // Repairing is needed here because this is the schema that is used by the reader to decide what
+ // conversions are necessary
+ val requestedParquetSchema = if (enableTimestampFieldRepair) {
+ HoodieParquetReadSupport.getRepairedSchema(readContext.getRequestedSchema, tableSchemaOpt)
+ } else {
+ readContext.getRequestedSchema
+ }
+ val trimmedParquetSchema = HoodieParquetReadSupport.trimParquetSchema(requestedParquetSchema, context.getFileSchema)
+ new ReadContext(trimmedParquetSchema, readContext.getReadSupportMetadata)
+ }
+}
+
+object HoodieParquetReadSupport {
+ /**
+ * Removes any fields from the parquet schema that do not have any child fields in the actual file schema after the
+ * schema is trimmed down to the requested fields. This can happen when the table schema evolves and only a subset of
+ * the nested fields are required by the query.
+ *
+ * @param requestedSchema the initial parquet schema requested by Spark
+ * @param fileSchema the actual parquet schema of the file
+ * @return a potentially updated schema with empty struct fields removed
+ */
+ def trimParquetSchema(requestedSchema: MessageType, fileSchema: MessageType): MessageType = {
+ val trimmedFields = requestedSchema.getFields.asScala.map(field => {
+ if (fileSchema.containsField(field.getName)) {
+ trimParquetType(field, fileSchema.asGroupType().getType(field.getName))
+ } else {
+ Some(field)
+ }
+ }).filter(_.isDefined).map(_.get).toArray[Type]
+ Types.buildMessage().addFields(trimmedFields: _*).named(requestedSchema.getName)
+ }
+
+ private def trimParquetType(requestedType: Type, fileType: Type): Option[Type] = {
+ if (requestedType.equals(fileType)) {
+ Some(requestedType)
+ } else {
+ requestedType match {
+ case groupType: GroupType =>
+ ValidationUtils.checkState(!fileType.isPrimitive,
+ "Group type provided by requested schema but existing type in the file is a primitive")
+ val fileTypeGroup = fileType.asGroupType()
+ var hasMatchingField = false
+ val fields = groupType.getFields.asScala.map(field => {
+ if (fileTypeGroup.containsField(field.getName)) {
+ hasMatchingField = true
+ trimParquetType(field, fileType.asGroupType().getType(field.getName))
+ } else {
+ Some(field)
+ }
+ }).filter(_.isDefined).map(_.get).asJava
+ if (hasMatchingField && !fields.isEmpty) {
+ Some(groupType.withNewFields(fields))
+ } else {
+ None
+ }
+ case _ => Some(requestedType)
+ }
+ }
+ }
+
+ def getRepairedSchema(fileSchema: MessageType,
+ tableSchema: org.apache.hudi.common.util.Option[MessageType]): MessageType = {
+ try {
+ val schemaRepairClass = Class.forName("org.apache.parquet.schema.SchemaRepair")
+ val repairMethod = schemaRepairClass.getMethod(
+ "repairLogicalTypes", classOf[MessageType], classOf[org.apache.hudi.common.util.Option[MessageType]])
+ repairMethod.invoke(null, fileSchema, tableSchema).asInstanceOf[MessageType]
+ } catch {
+ case _: Exception => fileSchema
+ }
+ }
+}
diff --git a/hudi-client/hudi-spark-client/src/parquet/scala/org/apache/spark/sql/execution/datasources/parquet/SparkBasicSchemaEvolution.scala b/hudi-client/hudi-spark-client/src/parquet/scala/org/apache/spark/sql/execution/datasources/parquet/SparkBasicSchemaEvolution.scala
new file mode 100644
index 0000000000000..6a6a094bbac3c
--- /dev/null
+++ b/hudi-client/hudi-spark-client/src/parquet/scala/org/apache/spark/sql/execution/datasources/parquet/SparkBasicSchemaEvolution.scala
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.parquet
+
+import org.apache.spark.sql.types.StructType
+
+/**
+ * Intended to be used just with HoodieSparkParquetReader to avoid any java/scala issues
+ */
+class SparkBasicSchemaEvolution(fileSchema: StructType,
+ requiredSchema: StructType,
+ sessionLocalTimeZone: String) {
+
+ val (implicitTypeChangeInfo, sparkRequestSchema) = HoodieParquetFileFormatHelper.buildImplicitSchemaChangeInfo(fileSchema, requiredSchema)
+
+ def getRequestSchema: StructType = {
+ if (implicitTypeChangeInfo.isEmpty) {
+ requiredSchema
+ } else {
+ sparkRequestSchema
+ }
+ }
+}
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestTableSchemaEvolution.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestTableSchemaEvolution.java
index 1a0d5a95f9a0f..856d8a3c9e88e 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestTableSchemaEvolution.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestTableSchemaEvolution.java
@@ -72,7 +72,7 @@ public class TestTableSchemaEvolution extends HoodieClientTestBase {
public static final String EXTRA_FIELD_WITHOUT_DEFAULT_SCHEMA =
"{\"name\": \"new_field_without_default\", \"type\": \"boolean\"},";
public static final String EXTRA_FIELD_NULLABLE_SCHEMA =
- ",{\"name\": \"new_field_without_default\", \"type\": [\"boolean\", \"null\"]}";
+ "{\"name\": \"new_field_without_default\", \"type\": [\"null\", \"boolean\"], \"default\": null},";
// TRIP_EXAMPLE_SCHEMA with a new_field added
public static final String TRIP_EXAMPLE_SCHEMA_EVOLVED_COL_ADDED = TRIP_SCHEMA_PREFIX + EXTRA_TYPE_SCHEMA + MAP_TYPE_SCHEMA
@@ -152,7 +152,7 @@ public void testSchemaCompatibilityBasic() {
"Added field without default and not nullable is not compatible (Evolved Schema)");
assertTrue(isSchemaCompatible(TRIP_EXAMPLE_SCHEMA, TRIP_SCHEMA_PREFIX + EXTRA_TYPE_SCHEMA + MAP_TYPE_SCHEMA
- + FARE_NESTED_SCHEMA + TIP_NESTED_SCHEMA + TRIP_SCHEMA_SUFFIX + EXTRA_FIELD_NULLABLE_SCHEMA, false),
+ + FARE_NESTED_SCHEMA + TIP_NESTED_SCHEMA + EXTRA_FIELD_NULLABLE_SCHEMA + TRIP_SCHEMA_SUFFIX, false),
"Added nullable field is compatible (Evolved Schema)");
}
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestExternalPathHandling.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestExternalPathHandling.java
index 0785f9eea76d9..c92b81b8680eb 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestExternalPathHandling.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestExternalPathHandling.java
@@ -77,6 +77,8 @@ public class TestExternalPathHandling extends HoodieClientTestBase {
private static final String FIELD_1 = "field1";
private static final String FIELD_2 = "field2";
+ private static final String TEST_SCHEMA = "{\"type\":\"record\",\"name\":\"TestRecord\",\"namespace\":\"org.apache.hudi.test\","
+ + "\"fields\":[{\"name\":\"field1\",\"type\":\"int\"},{\"name\":\"field2\",\"type\":[\"null\",\"string\"],\"default\":null}]}";
private HoodieWriteConfig writeConfig;
@ParameterizedTest
@@ -89,6 +91,7 @@ public void testFlow(FileIdAndNameGenerator fileIdAndNameGenerator, List
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(INMEMORY).build())
.withPath(metaClient.getBasePathV2().toString())
.withEmbeddedTimelineServerEnabled(false)
+ .withSchema(TEST_SCHEMA)
.withMetadataConfig(HoodieMetadataConfig.newBuilder()
.withMaxNumDeltaCommitsBeforeCompaction(2)
.enable(true)
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieMergedReadHandle.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieMergedReadHandle.java
index fd0e878d482a8..cee0ceac97494 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieMergedReadHandle.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieMergedReadHandle.java
@@ -19,6 +19,8 @@
package org.apache.hudi.io;
+import org.apache.hudi.avro.AvroSchemaUtils;
+import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.client.SparkRDDWriteClient;
import org.apache.hudi.common.config.HoodieMetadataConfig;
import org.apache.hudi.common.model.AWSDmsAvroPayload;
@@ -41,6 +43,7 @@
import org.apache.hudi.table.HoodieTable;
import org.apache.hudi.testutils.SparkClientFunctionalTestHarness;
+import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import org.junit.jupiter.params.ParameterizedTest;
@@ -198,7 +201,10 @@ private void doMergedReadAndValidate(HoodieTableMetaClient metaClient, HoodieWri
.collect(Collectors.toList());
assertEquals(1, partitionPathAndFileIDPairs.size());
String latestCommitTime = table.getActiveTimeline().lastInstant().get().getTimestamp();
- HoodieMergedReadHandle mergedReadHandle = new HoodieMergedReadHandle<>(writeConfig, Option.of(latestCommitTime), table, partitionPathAndFileIDPairs.get(0));
+ Schema baseFileReaderSchema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(writeConfig.getWriteSchema()), writeConfig.allowOperationMetadataField());
+ boolean hasTimestampFields = baseFileReaderSchema != null && AvroSchemaUtils.hasTimestampMillisField(baseFileReaderSchema);
+ HoodieMergedReadHandle mergedReadHandle = new HoodieMergedReadHandle<>(writeConfig, Option.of(latestCommitTime), table, partitionPathAndFileIDPairs.get(0),
+ hasTimestampFields);
List mergedRecords = mergedReadHandle.getMergedRecords();
assertEquals(totalRecords, mergedRecords.size());
List sortedMergedRecords = mergedRecords.stream()
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/storage/row/TestHoodieInternalRowParquetWriter.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/storage/row/TestHoodieInternalRowParquetWriter.java
index fb4559263125e..9b9bc603324e8 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/storage/row/TestHoodieInternalRowParquetWriter.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/storage/row/TestHoodieInternalRowParquetWriter.java
@@ -85,7 +85,7 @@ public void testProperWriting(boolean parquetWriteLegacyFormatEnabled) throws Ex
HoodieRowParquetWriteSupport writeSupport = getWriteSupport(writeConfigBuilder, hadoopConf, parquetWriteLegacyFormatEnabled);
HoodieWriteConfig cfg = writeConfigBuilder.build();
- HoodieParquetConfig parquetConfig = new HoodieParquetConfig<>(writeSupport,
+ HoodieParquetConfig parquetConfig = new HoodieParquetConfig<>(writeSupport,
CompressionCodecName.SNAPPY, cfg.getParquetBlockSize(), cfg.getParquetPageSize(), cfg.getParquetMaxFileSize(),
writeSupport.getHadoopConf(), cfg.getParquetCompressionRatio(), cfg.parquetDictionaryEnabled());
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestTimestampBasedKeyGenerator.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestTimestampBasedKeyGenerator.java
index 136ed7c4b5107..e7c198cc01750 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestTimestampBasedKeyGenerator.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestTimestampBasedKeyGenerator.java
@@ -152,7 +152,7 @@ void testTimestampBasedKeyGenerator() throws IOException {
// timezone is GMT+8:00, createTime is BigDecimal
BigDecimal decimal = new BigDecimal("1578283932000.0001");
Conversions.DecimalConversion conversion = new Conversions.DecimalConversion();
- Schema resolvedNullableSchema = AvroSchemaUtils.resolveNullableSchema(schema.getField("createTimeDecimal").schema());
+ Schema resolvedNullableSchema = AvroSchemaUtils.getNonNullTypeFromUnion(schema.getField("createTimeDecimal").schema());
GenericFixed avroDecimal = conversion.toFixed(decimal, resolvedNullableSchema, LogicalTypes.decimal(20, 4));
baseRecord.put("createTimeDecimal", avroDecimal);
properties = getBaseKeyConfig("createTimeDecimal", "EPOCHMILLISECONDS", "yyyy-MM-dd hh", "GMT+8:00", null);
@@ -208,6 +208,31 @@ void testTimestampBasedKeyGenerator() throws IOException {
assertEquals("1970-01-01 12:00:00", keyGen.getPartitionPath(baseRow));
internalRow = KeyGeneratorTestUtilities.getInternalRow(baseRow);
assertEquals(UTF8String.fromString("1970-01-01 12:00:00"), keyGen.getPartitionPath(internalRow, baseRow.schema()));
+
+ // Timestamp field is in long type, with `EPOCHMICROSECONDS` timestamp type in the key generator
+ baseRecord.put("createTime", 1578283932123456L);
+ properties = getBaseKeyConfig("createTime", "EPOCHMICROSECONDS", "yyyy-MM-dd hh", "GMT+8:00", null);
+ keyGen = new TimestampBasedKeyGenerator(properties);
+ HoodieKey key = keyGen.getKey(baseRecord);
+ assertEquals("2020-01-06 12", key.getPartitionPath());
+ baseRow = genericRecordToRow(baseRecord);
+ assertEquals("2020-01-06 12", keyGen.getPartitionPath(baseRow));
+ internalRow = KeyGeneratorTestUtilities.getInternalRow(baseRow);
+ assertEquals(UTF8String.fromString("2020-01-06 12"), keyGen.getPartitionPath(internalRow, baseRow.schema()));
+
+ // Timestamp field is in decimal type, with `EPOCHMICROSECONDS` timestamp type in the key generator
+ decimal = new BigDecimal("1578283932123456.0001");
+ resolvedNullableSchema = AvroSchemaUtils.getNonNullTypeFromUnion(
+ schema.getField("createTimeDecimal").schema());
+ avroDecimal = conversion.toFixed(decimal, resolvedNullableSchema, LogicalTypes.decimal(20, 4));
+ baseRecord.put("createTimeDecimal", avroDecimal);
+ properties = getBaseKeyConfig(
+ "createTimeDecimal", "EPOCHMICROSECONDS", "yyyy-MM-dd hh", "GMT+8:00", null);
+ keyGen = new TimestampBasedKeyGenerator(properties);
+ bigDecimalKey = keyGen.getKey(baseRecord);
+ assertEquals("2020-01-06 12", bigDecimalKey.getPartitionPath());
+ baseRow = genericRecordToRow(baseRecord);
+ assertEquals("2020-01-06 12", keyGen.getPartitionPath(baseRow));
}
@Test
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java
index 2188d7246faa5..2a3d50084c0c8 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java
@@ -691,7 +691,7 @@ public void testHandleUpdateWithMultiplePartitions() throws Exception {
BaseSparkDeltaCommitActionExecutor actionExecutor = new SparkDeleteDeltaCommitActionExecutor(context(), cfg, hoodieTable,
newDeleteTime, HoodieJavaRDD.of(deleteRDD));
actionExecutor.getUpsertPartitioner(new WorkloadProfile(buildProfile(deleteRDD)));
- final List> deleteStatus = jsc().parallelize(Arrays.asList(1)).map(x -> {
+ final List> deleteStatus = (List>) (List) jsc().parallelize(Arrays.asList(1)).map(x -> {
return actionExecutor.handleUpdate(partitionPath, fileId, fewRecordsForDelete.iterator());
}).map(Transformations::flatten).collect();
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestCopyOnWriteActionExecutor.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestCopyOnWriteActionExecutor.java
index 24b66911613ea..d5c6043810c69 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestCopyOnWriteActionExecutor.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestCopyOnWriteActionExecutor.java
@@ -464,7 +464,7 @@ public void testInsertUpsertWithHoodieAvroPayload() throws Exception {
final List inserts = dataGen.generateInsertsWithHoodieAvroPayload(instantTime, 100);
BaseSparkCommitActionExecutor actionExecutor = new SparkInsertCommitActionExecutor(context, config, table,
instantTime, context.parallelize(inserts));
- final List> ws = jsc.parallelize(Arrays.asList(1)).map(x -> {
+ final List> ws = (List>) (List) jsc.parallelize(Arrays.asList(1)).map(x -> {
return actionExecutor.handleInsert(UUID.randomUUID().toString(), inserts.iterator());
}).map(Transformations::flatten).collect();
@@ -478,7 +478,7 @@ public void testInsertUpsertWithHoodieAvroPayload() throws Exception {
table = (HoodieSparkCopyOnWriteTable) HoodieSparkTable.create(config, context, HoodieTableMetaClient.reload(metaClient));
BaseSparkCommitActionExecutor newActionExecutor = new SparkUpsertCommitActionExecutor(context, config, table,
instantTime, context.parallelize(updates));
- final List> updateStatus = jsc.parallelize(Arrays.asList(1)).map(x -> {
+ final List> updateStatus = (List>) (List) jsc.parallelize(Arrays.asList(1)).map(x -> {
return newActionExecutor.handleUpdate(partitionPath, fileId, updates.iterator());
}).map(Transformations::flatten).collect();
assertEquals(updates.size() - numRecordsInPartition, updateStatus.get(0).get(0).getTotalErrorRecords());
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestUtils.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestUtils.java
index f5f17e1d9bc55..ddc36adf123e6 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestUtils.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestUtils.java
@@ -109,9 +109,16 @@ public static SparkConf getSparkConfForTest(String appName) {
sparkConf.set("spark.sql.extensions", "org.apache.spark.sql.hudi.HoodieSparkSessionExtension");
}
- if (canLoadClass("org.apache.spark.sql.hudi.catalog.HoodieCatalog") && HoodieSparkUtils.gteqSpark3_2()) {
- sparkConf.set("spark.sql.catalog.spark_catalog",
- "org.apache.spark.sql.hudi.catalog.HoodieCatalog");
+ try {
+ if (canLoadClass("org.apache.spark.sql.hudi.catalog.HoodieCatalog") && HoodieSparkUtils.gteqSpark3_2()) {
+ sparkConf.set("spark.sql.catalog.spark_catalog",
+ "org.apache.spark.sql.hudi.catalog.HoodieCatalog");
+ }
+ } catch (LinkageError e) {
+ // Handle Scala version compatibility issues (e.g., Scala 2.12 vs 2.13)
+ // If we can't determine the Spark version, skip setting the catalog
+ // This can happen when Scala code compiled with 2.12 references types removed in 2.13
+ // LinkageError catches both NoClassDefFoundError and other linkage errors
}
String evlogDir = System.getProperty("SPARK_EVLOG_DIR");
@@ -344,6 +351,12 @@ private static boolean canLoadClass(String className) {
return ReflectionUtils.getClass(className) != null;
} catch (Exception e) {
return false;
+ } catch (NoClassDefFoundError e) {
+ // Handle cases where class exists but dependencies are missing (e.g., Scala 2.12 vs 2.13 compatibility)
+ return false;
+ } catch (Throwable e) {
+ // Catch any other errors (LinkageError, etc.) that might occur during class loading
+ return false;
}
}
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkClientFunctionalTestHarness.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkClientFunctionalTestHarness.java
index 511613d904438..9546cb5349530 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkClientFunctionalTestHarness.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkClientFunctionalTestHarness.java
@@ -112,8 +112,15 @@ public static Map getSparkSqlConf() {
Map sqlConf = new HashMap<>();
sqlConf.put("spark.sql.extensions", "org.apache.spark.sql.hudi.HoodieSparkSessionExtension");
- if (HoodieSparkUtils.gteqSpark3_2()) {
- sqlConf.put("spark.sql.catalog.spark_catalog", "org.apache.spark.sql.hudi.catalog.HoodieCatalog");
+ try {
+ if (HoodieSparkUtils.gteqSpark3_2()) {
+ sqlConf.put("spark.sql.catalog.spark_catalog", "org.apache.spark.sql.hudi.catalog.HoodieCatalog");
+ }
+ } catch (LinkageError e) {
+ // Handle Scala version compatibility issues (e.g., Scala 2.12 vs 2.13)
+ // If we can't determine the Spark version, skip setting the catalog
+ // This can happen when Scala code compiled with 2.12 references types removed in 2.13
+ // LinkageError catches both NoClassDefFoundError and other linkage errors
}
return sqlConf;
diff --git a/hudi-common/pom.xml b/hudi-common/pom.xml
index 1d5cda0f75e82..9014d1a056325 100644
--- a/hudi-common/pom.xml
+++ b/hudi-common/pom.xml
@@ -54,6 +54,41 @@
false
+
+ org.codehaus.mojo
+ build-helper-maven-plugin
+ 3.5.0
+
+
+ add-spark34plus-avro-sources
+ generate-sources
+
+ add-source
+
+
+ ${spark33orEarlier}
+
+ src/avro/java
+ src/parquet/java
+
+
+
+
+ add-spark34plus-avro-test-sources
+ generate-test-sources
+
+ add-test-source
+
+
+ ${spark33orEarlier}
+
+ src/avro/test/java
+ src/parquet/test/java
+
+
+
+
+ org.apache.ratapache-rat-plugin
diff --git a/hudi-common/src/avro/java/org/apache/parquet/avro/AvroSchemaConverterWithTimestampNTZ.java b/hudi-common/src/avro/java/org/apache/parquet/avro/AvroSchemaConverterWithTimestampNTZ.java
new file mode 100644
index 0000000000000..c1bda6ef0ffb7
--- /dev/null
+++ b/hudi-common/src/avro/java/org/apache/parquet/avro/AvroSchemaConverterWithTimestampNTZ.java
@@ -0,0 +1,598 @@
+/*
+ * 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.parquet.avro;
+
+import org.apache.avro.LogicalType;
+import org.apache.avro.LogicalTypes;
+import org.apache.avro.Schema;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.parquet.schema.ConversionPatterns;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.LogicalTypeAnnotation;
+import org.apache.parquet.schema.LogicalTypeAnnotation.UUIDLogicalTypeAnnotation;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
+import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName;
+import org.apache.parquet.schema.Type;
+import org.apache.parquet.schema.Types;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static org.apache.avro.JsonProperties.NULL_VALUE;
+import static org.apache.parquet.avro.AvroReadSupport.READ_INT96_AS_FIXED;
+import static org.apache.parquet.avro.AvroReadSupport.READ_INT96_AS_FIXED_DEFAULT;
+import static org.apache.parquet.avro.AvroWriteSupport.WRITE_OLD_LIST_STRUCTURE;
+import static org.apache.parquet.avro.AvroWriteSupport.WRITE_OLD_LIST_STRUCTURE_DEFAULT;
+import static org.apache.parquet.avro.AvroWriteSupport.WRITE_PARQUET_UUID;
+import static org.apache.parquet.avro.AvroWriteSupport.WRITE_PARQUET_UUID_DEFAULT;
+import static org.apache.parquet.schema.LogicalTypeAnnotation.TimeUnit.MICROS;
+import static org.apache.parquet.schema.LogicalTypeAnnotation.TimeUnit.MILLIS;
+import static org.apache.parquet.schema.LogicalTypeAnnotation.dateType;
+import static org.apache.parquet.schema.LogicalTypeAnnotation.decimalType;
+import static org.apache.parquet.schema.LogicalTypeAnnotation.enumType;
+import static org.apache.parquet.schema.LogicalTypeAnnotation.stringType;
+import static org.apache.parquet.schema.LogicalTypeAnnotation.timeType;
+import static org.apache.parquet.schema.LogicalTypeAnnotation.timestampType;
+import static org.apache.parquet.schema.LogicalTypeAnnotation.uuidType;
+import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.BINARY;
+import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.BOOLEAN;
+import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.DOUBLE;
+import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY;
+import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.FLOAT;
+import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT32;
+import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT64;
+import static org.apache.parquet.schema.Type.Repetition.REPEATED;
+
+/**
+ *
+ * Converts an Avro schema into a Parquet schema, or vice versa. See package
+ * documentation for details of the mapping.
+ *
+ * This was taken from parquet-java 1.13.1 AvroSchemaConverter and modified
+ * to support local timestamp types by copying a few methods from 1.14.0 AvroSchemaConverter.
+ */
+@SuppressWarnings("all")
+public class AvroSchemaConverterWithTimestampNTZ extends HoodieAvroParquetSchemaConverter {
+
+ public static final String ADD_LIST_ELEMENT_RECORDS =
+ "parquet.avro.add-list-element-records";
+ private static final boolean ADD_LIST_ELEMENT_RECORDS_DEFAULT = true;
+
+ private final boolean assumeRepeatedIsListElement;
+ private final boolean writeOldListStructure;
+ private final boolean writeParquetUUID;
+ private final boolean readInt96AsFixed;
+ private final Set pathsToInt96;
+
+ public AvroSchemaConverterWithTimestampNTZ() {
+ this(ADD_LIST_ELEMENT_RECORDS_DEFAULT);
+ }
+
+ /**
+ * Constructor used by {@link AvroRecordConverter#isElementType}, which always
+ * uses the 2-level list conversion.
+ *
+ * @param assumeRepeatedIsListElement whether to assume 2-level lists
+ */
+ AvroSchemaConverterWithTimestampNTZ(boolean assumeRepeatedIsListElement) {
+ this.assumeRepeatedIsListElement = assumeRepeatedIsListElement;
+ this.writeOldListStructure = WRITE_OLD_LIST_STRUCTURE_DEFAULT;
+ this.writeParquetUUID = WRITE_PARQUET_UUID_DEFAULT;
+ this.readInt96AsFixed = READ_INT96_AS_FIXED_DEFAULT;
+ this.pathsToInt96 = Collections.emptySet();
+ }
+
+ public AvroSchemaConverterWithTimestampNTZ(Configuration conf) {
+ this.assumeRepeatedIsListElement = conf.getBoolean(
+ ADD_LIST_ELEMENT_RECORDS, ADD_LIST_ELEMENT_RECORDS_DEFAULT);
+ this.writeOldListStructure = conf.getBoolean(
+ WRITE_OLD_LIST_STRUCTURE, WRITE_OLD_LIST_STRUCTURE_DEFAULT);
+ this.writeParquetUUID = conf.getBoolean(WRITE_PARQUET_UUID, WRITE_PARQUET_UUID_DEFAULT);
+ this.readInt96AsFixed = conf.getBoolean(READ_INT96_AS_FIXED, READ_INT96_AS_FIXED_DEFAULT);
+ this.pathsToInt96 = new HashSet<>(Arrays.asList(conf.getStrings("parquet.avro.writeFixedAsInt96", new String[0])));
+ }
+
+ /**
+ * Given a schema, check to see if it is a union of a null type and a regular schema,
+ * and then return the non-null sub-schema. Otherwise, return the given schema.
+ *
+ * @param schema The schema to check
+ * @return The non-null portion of a union schema, or the given schema
+ */
+ public static Schema getNonNull(Schema schema) {
+ if (schema.getType().equals(Schema.Type.UNION)) {
+ List schemas = schema.getTypes();
+ if (schemas.size() == 2) {
+ if (schemas.get(0).getType().equals(Schema.Type.NULL)) {
+ return schemas.get(1);
+ } else if (schemas.get(1).getType().equals(Schema.Type.NULL)) {
+ return schemas.get(0);
+ } else {
+ return schema;
+ }
+ } else {
+ return schema;
+ }
+ } else {
+ return schema;
+ }
+ }
+
+ @Override
+ public MessageType convert(Schema avroSchema) {
+ if (!avroSchema.getType().equals(Schema.Type.RECORD)) {
+ throw new IllegalArgumentException("Avro schema must be a record.");
+ }
+ return new MessageType(avroSchema.getFullName(), convertFields(avroSchema.getFields(), ""));
+ }
+
+ private List convertFields(List fields, String schemaPath) {
+ List types = new ArrayList();
+ for (Schema.Field field : fields) {
+ if (field.schema().getType().equals(Schema.Type.NULL)) {
+ continue; // Avro nulls are not encoded, unless they are null unions
+ }
+ types.add(convertField(field, appendPath(schemaPath, field.name())));
+ }
+ return types;
+ }
+
+ private Type convertField(String fieldName, Schema schema, String schemaPath) {
+ return convertField(fieldName, schema, Type.Repetition.REQUIRED, schemaPath);
+ }
+
+ @SuppressWarnings("deprecation")
+ private Type convertField(String fieldName, Schema schema, Type.Repetition repetition, String schemaPath) {
+ Types.PrimitiveBuilder builder;
+ Schema.Type type = schema.getType();
+ LogicalType logicalType = schema.getLogicalType();
+ if (type.equals(Schema.Type.BOOLEAN)) {
+ builder = Types.primitive(BOOLEAN, repetition);
+ } else if (type.equals(Schema.Type.INT)) {
+ builder = Types.primitive(INT32, repetition);
+ } else if (type.equals(Schema.Type.LONG)) {
+ builder = Types.primitive(INT64, repetition);
+ } else if (type.equals(Schema.Type.FLOAT)) {
+ builder = Types.primitive(FLOAT, repetition);
+ } else if (type.equals(Schema.Type.DOUBLE)) {
+ builder = Types.primitive(DOUBLE, repetition);
+ } else if (type.equals(Schema.Type.BYTES)) {
+ builder = Types.primitive(BINARY, repetition);
+ } else if (type.equals(Schema.Type.STRING)) {
+ if (logicalType != null && logicalType.getName().equals(LogicalTypes.uuid().getName()) && writeParquetUUID) {
+ builder = Types.primitive(FIXED_LEN_BYTE_ARRAY, repetition)
+ .length(LogicalTypeAnnotation.UUIDLogicalTypeAnnotation.BYTES);
+ } else {
+ builder = Types.primitive(BINARY, repetition).as(stringType());
+ }
+ } else if (type.equals(Schema.Type.RECORD)) {
+ return new GroupType(repetition, fieldName, convertFields(schema.getFields(), schemaPath));
+ } else if (type.equals(Schema.Type.ENUM)) {
+ builder = Types.primitive(BINARY, repetition).as(enumType());
+ } else if (type.equals(Schema.Type.ARRAY)) {
+ if (writeOldListStructure) {
+ return ConversionPatterns.listType(repetition, fieldName,
+ convertField("array", schema.getElementType(), REPEATED, schemaPath));
+ } else {
+ return ConversionPatterns.listOfElements(repetition, fieldName,
+ convertField(AvroWriteSupport.LIST_ELEMENT_NAME, schema.getElementType(), schemaPath));
+ }
+ } else if (type.equals(Schema.Type.MAP)) {
+ Type valType = convertField("value", schema.getValueType(), schemaPath);
+ // avro map key type is always string
+ return ConversionPatterns.stringKeyMapType(repetition, fieldName, valType);
+ } else if (type.equals(Schema.Type.FIXED)) {
+ if (pathsToInt96.contains(schemaPath)) {
+ if (schema.getFixedSize() != 12) {
+ throw new IllegalArgumentException(
+ "The size of the fixed type field " + schemaPath + " must be 12 bytes for INT96 conversion");
+ }
+ builder = Types.primitive(PrimitiveTypeName.INT96, repetition);
+ } else {
+ builder = Types.primitive(FIXED_LEN_BYTE_ARRAY, repetition).length(schema.getFixedSize());
+ }
+ } else if (type.equals(Schema.Type.UNION)) {
+ return convertUnion(fieldName, schema, repetition, schemaPath);
+ } else {
+ throw new UnsupportedOperationException("Cannot convert Avro type " + type);
+ }
+
+ // schema translation can only be done for known logical types because this
+ // creates an equivalence
+ if (logicalType != null) {
+ if (logicalType instanceof LogicalTypes.Decimal) {
+ LogicalTypes.Decimal decimal = (LogicalTypes.Decimal) logicalType;
+ builder = builder.as(decimalType(decimal.getScale(), decimal.getPrecision()));
+ } else {
+ LogicalTypeAnnotation annotation = convertLogicalType(logicalType);
+ if (annotation != null) {
+ builder.as(annotation);
+ }
+ }
+ }
+
+ return builder.named(fieldName);
+ }
+
+ private Type convertUnion(String fieldName, Schema schema, Type.Repetition repetition, String schemaPath) {
+ List nonNullSchemas = new ArrayList(schema.getTypes().size());
+ // Found any schemas in the union? Required for the edge case, where the union contains only a single type.
+ boolean foundNullSchema = false;
+ for (Schema childSchema : schema.getTypes()) {
+ if (childSchema.getType().equals(Schema.Type.NULL)) {
+ foundNullSchema = true;
+ if (Type.Repetition.REQUIRED == repetition) {
+ repetition = Type.Repetition.OPTIONAL;
+ }
+ } else {
+ nonNullSchemas.add(childSchema);
+ }
+ }
+ // If we only get a null and one other type then its a simple optional field
+ // otherwise construct a union container
+ switch (nonNullSchemas.size()) {
+ case 0:
+ throw new UnsupportedOperationException("Cannot convert Avro union of only nulls");
+
+ case 1:
+ return foundNullSchema ? convertField(fieldName, nonNullSchemas.get(0), repetition, schemaPath) :
+ convertUnionToGroupType(fieldName, repetition, nonNullSchemas, schemaPath);
+
+ default: // complex union type
+ return convertUnionToGroupType(fieldName, repetition, nonNullSchemas, schemaPath);
+ }
+ }
+
+ private Type convertUnionToGroupType(String fieldName, Type.Repetition repetition, List nonNullSchemas,
+ String schemaPath) {
+ List unionTypes = new ArrayList(nonNullSchemas.size());
+ int index = 0;
+ for (Schema childSchema : nonNullSchemas) {
+ unionTypes.add( convertField("member" + index++, childSchema, Type.Repetition.OPTIONAL, schemaPath));
+ }
+ return new GroupType(repetition, fieldName, unionTypes);
+ }
+
+ private Type convertField(Schema.Field field, String schemaPath) {
+ return convertField(field.name(), field.schema(), schemaPath);
+ }
+
+ @Override
+ public Schema convert(MessageType parquetSchema) {
+ return convertFields(parquetSchema.getName(), parquetSchema.getFields(), new HashMap<>());
+ }
+
+ Schema convert(GroupType parquetSchema) {
+ return convertFields(parquetSchema.getName(), parquetSchema.getFields(), new HashMap<>());
+ }
+
+ private Schema convertFields(String name, List parquetFields, Map names) {
+ List fields = new ArrayList();
+ Integer nameCount = names.merge(name, 1, (oldValue, value) -> oldValue + 1);
+ for (Type parquetType : parquetFields) {
+ Schema fieldSchema = convertField(parquetType, names);
+ if (parquetType.isRepetition(REPEATED)) {
+ throw new UnsupportedOperationException("REPEATED not supported outside LIST or MAP. Type: " + parquetType);
+ } else if (parquetType.isRepetition(Type.Repetition.OPTIONAL)) {
+ fields.add(new Schema.Field(
+ parquetType.getName(), optional(fieldSchema), null, NULL_VALUE));
+ } else { // REQUIRED
+ fields.add(new Schema.Field(
+ parquetType.getName(), fieldSchema, null, (Object) null));
+ }
+ }
+ Schema schema = Schema.createRecord(name, null, nameCount > 1 ? name + nameCount : null, false);
+ schema.setFields(fields);
+ return schema;
+ }
+
+ private Schema convertField(final Type parquetType, Map names) {
+ if (parquetType.isPrimitive()) {
+ final PrimitiveType asPrimitive = parquetType.asPrimitiveType();
+ final PrimitiveTypeName parquetPrimitiveTypeName =
+ asPrimitive.getPrimitiveTypeName();
+ final LogicalTypeAnnotation annotation = parquetType.getLogicalTypeAnnotation();
+ Schema schema = parquetPrimitiveTypeName.convert(
+ new PrimitiveType.PrimitiveTypeNameConverter() {
+ @Override
+ public Schema convertBOOLEAN(PrimitiveTypeName primitiveTypeName) {
+ return Schema.create(Schema.Type.BOOLEAN);
+ }
+ @Override
+ public Schema convertINT32(PrimitiveTypeName primitiveTypeName) {
+ return Schema.create(Schema.Type.INT);
+ }
+ @Override
+ public Schema convertINT64(PrimitiveTypeName primitiveTypeName) {
+ return Schema.create(Schema.Type.LONG);
+ }
+ @Override
+ public Schema convertINT96(PrimitiveTypeName primitiveTypeName) {
+ if (readInt96AsFixed) {
+ return Schema.createFixed("INT96", "INT96 represented as byte[12]", null, 12);
+ }
+ throw new IllegalArgumentException(
+ "INT96 is deprecated. As interim enable READ_INT96_AS_FIXED flag to read as byte array.");
+ }
+ @Override
+ public Schema convertFLOAT(PrimitiveTypeName primitiveTypeName) {
+ return Schema.create(Schema.Type.FLOAT);
+ }
+ @Override
+ public Schema convertDOUBLE(PrimitiveTypeName primitiveTypeName) {
+ return Schema.create(Schema.Type.DOUBLE);
+ }
+ @Override
+ public Schema convertFIXED_LEN_BYTE_ARRAY(PrimitiveTypeName primitiveTypeName) {
+ if (annotation instanceof LogicalTypeAnnotation.UUIDLogicalTypeAnnotation) {
+ return Schema.create(Schema.Type.STRING);
+ } else {
+ int size = parquetType.asPrimitiveType().getTypeLength();
+ return Schema.createFixed(parquetType.getName(), null, null, size);
+ }
+ }
+ @Override
+ public Schema convertBINARY(PrimitiveTypeName primitiveTypeName) {
+ if (annotation instanceof LogicalTypeAnnotation.StringLogicalTypeAnnotation ||
+ annotation instanceof LogicalTypeAnnotation.EnumLogicalTypeAnnotation) {
+ return Schema.create(Schema.Type.STRING);
+ } else {
+ return Schema.create(Schema.Type.BYTES);
+ }
+ }
+ });
+
+ LogicalType logicalType = convertLogicalType(annotation);
+ if (logicalType != null && (!(annotation instanceof LogicalTypeAnnotation.DecimalLogicalTypeAnnotation) ||
+ parquetPrimitiveTypeName == BINARY ||
+ parquetPrimitiveTypeName == FIXED_LEN_BYTE_ARRAY)) {
+ schema = logicalType.addToSchema(schema);
+ }
+
+ return schema;
+
+ } else {
+ GroupType parquetGroupType = parquetType.asGroupType();
+ LogicalTypeAnnotation logicalTypeAnnotation = parquetGroupType.getLogicalTypeAnnotation();
+ if (logicalTypeAnnotation != null) {
+ return logicalTypeAnnotation.accept(new LogicalTypeAnnotation.LogicalTypeAnnotationVisitor() {
+ @Override
+ public java.util.Optional visit(LogicalTypeAnnotation.ListLogicalTypeAnnotation listLogicalType) {
+ if (parquetGroupType.getFieldCount()!= 1) {
+ throw new UnsupportedOperationException("Invalid list type " + parquetGroupType);
+ }
+ Type repeatedType = parquetGroupType.getType(0);
+ if (!repeatedType.isRepetition(REPEATED)) {
+ throw new UnsupportedOperationException("Invalid list type " + parquetGroupType);
+ }
+ if (isElementType(repeatedType, parquetGroupType.getName())) {
+ // repeated element types are always required
+ return java.util.Optional.of(Schema.createArray(convertField(repeatedType, names)));
+ } else {
+ Type elementType = repeatedType.asGroupType().getType(0);
+ if (elementType.isRepetition(Type.Repetition.OPTIONAL)) {
+ return java.util.Optional.of(Schema.createArray(optional(convertField(elementType, names))));
+ } else {
+ return java.util.Optional.of(Schema.createArray(convertField(elementType, names)));
+ }
+ }
+ }
+
+ @Override
+ // for backward-compatibility
+ public java.util.Optional visit(LogicalTypeAnnotation.MapKeyValueTypeAnnotation mapKeyValueLogicalType) {
+ return visitMapOrMapKeyValue();
+ }
+
+ @Override
+ public java.util.Optional visit(LogicalTypeAnnotation.MapLogicalTypeAnnotation mapLogicalType) {
+ return visitMapOrMapKeyValue();
+ }
+
+ private java.util.Optional visitMapOrMapKeyValue() {
+ if (parquetGroupType.getFieldCount() != 1 || parquetGroupType.getType(0).isPrimitive()) {
+ throw new UnsupportedOperationException("Invalid map type " + parquetGroupType);
+ }
+ GroupType mapKeyValType = parquetGroupType.getType(0).asGroupType();
+ if (!mapKeyValType.isRepetition(REPEATED) ||
+ mapKeyValType.getFieldCount()!=2) {
+ throw new UnsupportedOperationException("Invalid map type " + parquetGroupType);
+ }
+ Type keyType = mapKeyValType.getType(0);
+ if (!keyType.isPrimitive() ||
+ !keyType.asPrimitiveType().getPrimitiveTypeName().equals(BINARY) ||
+ !keyType.getLogicalTypeAnnotation().equals(stringType())) {
+ throw new IllegalArgumentException("Map key type must be binary (UTF8): "
+ + keyType);
+ }
+ Type valueType = mapKeyValType.getType(1);
+ if (valueType.isRepetition(Type.Repetition.OPTIONAL)) {
+ return java.util.Optional.of(Schema.createMap(optional(convertField(valueType, names))));
+ } else {
+ return java.util.Optional.of(Schema.createMap(convertField(valueType, names)));
+ }
+ }
+
+ @Override
+ public java.util.Optional visit(LogicalTypeAnnotation.EnumLogicalTypeAnnotation enumLogicalType) {
+ return java.util.Optional.of(Schema.create(Schema.Type.STRING));
+ }
+ }).orElseThrow(() -> new UnsupportedOperationException("Cannot convert Parquet type " + parquetType));
+ } else {
+ // if no original type then it's a record
+ return convertFields(parquetGroupType.getName(), parquetGroupType.getFields(), names);
+ }
+ }
+ }
+
+ private LogicalTypeAnnotation convertLogicalType(LogicalType logicalType) {
+ if (logicalType == null) {
+ return null;
+ } else if (logicalType instanceof LogicalTypes.Decimal) {
+ LogicalTypes.Decimal decimal = (LogicalTypes.Decimal) logicalType;
+ return decimalType(decimal.getScale(), decimal.getPrecision());
+ } else if (logicalType instanceof LogicalTypes.Date) {
+ return dateType();
+ } else if (logicalType instanceof LogicalTypes.TimeMillis) {
+ return timeType(true, MILLIS);
+ } else if (logicalType instanceof LogicalTypes.TimeMicros) {
+ return timeType(true, MICROS);
+ } else if (logicalType instanceof LogicalTypes.TimestampMillis) {
+ return timestampType(true, MILLIS);
+ } else if (logicalType instanceof LogicalTypes.TimestampMicros) {
+ return timestampType(true, MICROS);
+ } else if (logicalType.getName().equals(LogicalTypes.uuid().getName()) && writeParquetUUID) {
+ return uuidType();
+ }
+
+ if (avroVersionSupportsLocalTimestampTypes()) {
+ if (logicalType instanceof LogicalTypes.LocalTimestampMillis) {
+ return timestampType(false, MILLIS);
+ } else if (logicalType instanceof LogicalTypes.LocalTimestampMicros) {
+ return timestampType(false, MICROS);
+ }
+ }
+
+ return null;
+ }
+
+ private LogicalType convertLogicalType(LogicalTypeAnnotation annotation) {
+ if (annotation == null) {
+ return null;
+ }
+ return annotation
+ .accept(new LogicalTypeAnnotation.LogicalTypeAnnotationVisitor() {
+ @Override
+ public java.util.Optional visit(
+ LogicalTypeAnnotation.DecimalLogicalTypeAnnotation decimalLogicalType) {
+ return java.util.Optional.of(
+ LogicalTypes.decimal(decimalLogicalType.getPrecision(), decimalLogicalType.getScale()));
+ }
+
+ @Override
+ public java.util.Optional visit(
+ LogicalTypeAnnotation.DateLogicalTypeAnnotation dateLogicalType) {
+ return java.util.Optional.of(LogicalTypes.date());
+ }
+
+ @Override
+ public java.util.Optional visit(
+ LogicalTypeAnnotation.TimeLogicalTypeAnnotation timeLogicalType) {
+ LogicalTypeAnnotation.TimeUnit unit = timeLogicalType.getUnit();
+ switch (unit) {
+ case MILLIS:
+ return java.util.Optional.of(LogicalTypes.timeMillis());
+ case MICROS:
+ return java.util.Optional.of(LogicalTypes.timeMicros());
+ }
+ return java.util.Optional.empty();
+ }
+
+ @Override
+ public java.util.Optional visit(
+ LogicalTypeAnnotation.TimestampLogicalTypeAnnotation timestampLogicalType) {
+ LogicalTypeAnnotation.TimeUnit unit = timestampLogicalType.getUnit();
+ boolean isAdjustedToUTC = timestampLogicalType.isAdjustedToUTC();
+
+ if (isAdjustedToUTC || !avroVersionSupportsLocalTimestampTypes()) {
+ switch (unit) {
+ case MILLIS:
+ return java.util.Optional.of(LogicalTypes.timestampMillis());
+ case MICROS:
+ return java.util.Optional.of(LogicalTypes.timestampMicros());
+ }
+ return java.util.Optional.empty();
+ } else {
+ switch (unit) {
+ case MILLIS:
+ return java.util.Optional.of(LogicalTypes.localTimestampMillis());
+ case MICROS:
+ return java.util.Optional.of(LogicalTypes.localTimestampMicros());
+ }
+ return java.util.Optional.empty();
+ }
+ }
+
+ @Override
+ public java.util.Optional visit(UUIDLogicalTypeAnnotation uuidLogicalType) {
+ return java.util.Optional.of(LogicalTypes.uuid());
+ }
+ })
+ .orElse(null);
+ }
+
+ /**
+ * Implements the rules for interpreting existing data from the logical type
+ * spec for the LIST annotation. This is used to produce the expected schema.
+ *
+ * The AvroArrayConverter will decide whether the repeated type is the array
+ * element type by testing whether the element schema and repeated type are
+ * the same. This ensures that the LIST rules are followed when there is no
+ * schema and that a schema can be provided to override the default behavior.
+ */
+ private boolean isElementType(Type repeatedType, String parentName) {
+ return (
+ // can't be a synthetic layer because it would be invalid
+ repeatedType.isPrimitive() ||
+ repeatedType.asGroupType().getFieldCount() > 1 ||
+ repeatedType.asGroupType().getType(0).isRepetition(REPEATED) ||
+ // known patterns without the synthetic layer
+ repeatedType.getName().equals("array") ||
+ repeatedType.getName().equals(parentName + "_tuple") ||
+ // default assumption
+ assumeRepeatedIsListElement
+ );
+ }
+
+ private static Schema optional(Schema original) {
+ // null is first in the union because Parquet's default is always null
+ return Schema.createUnion(Arrays.asList(
+ Schema.create(Schema.Type.NULL),
+ original));
+ }
+
+ private static String appendPath(String path, String fieldName) {
+ if (path == null || path.isEmpty()) {
+ return fieldName;
+ }
+ return path + '.' + fieldName;
+ }
+
+ /* Avro <= 1.9 does not support conversions to LocalTimestamp{Micros, Millis} classes */
+ private static boolean avroVersionSupportsLocalTimestampTypes() {
+ final String avroVersion = getRuntimeAvroVersion();
+
+ return avroVersion == null
+ || !(avroVersion.startsWith("1.7.")
+ || avroVersion.startsWith("1.8.")
+ || avroVersion.startsWith("1.9."));
+ }
+
+ private static String getRuntimeAvroVersion() {
+ return Schema.Parser.class.getPackage().getImplementationVersion();
+ }
+}
diff --git a/hudi-common/src/avro/java/org/apache/parquet/schema/AvroSchemaRepair.java b/hudi-common/src/avro/java/org/apache/parquet/schema/AvroSchemaRepair.java
new file mode 100644
index 0000000000000..8546b1b8a5e3a
--- /dev/null
+++ b/hudi-common/src/avro/java/org/apache/parquet/schema/AvroSchemaRepair.java
@@ -0,0 +1,245 @@
+/*
+ * 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.parquet.schema;
+
+import org.apache.hudi.avro.AvroSchemaCache;
+import org.apache.hudi.avro.AvroSchemaUtils;
+import org.apache.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.exception.HoodieException;
+
+import org.apache.avro.LogicalType;
+import org.apache.avro.LogicalTypes;
+import org.apache.avro.Schema;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+public class AvroSchemaRepair {
+
+ public static Schema repairLogicalTypes(Schema fileSchema, Schema tableSchema) {
+ Schema repairedSchema = repairAvroSchema(fileSchema, tableSchema);
+ if (repairedSchema != fileSchema) {
+ return AvroSchemaCache.intern(repairedSchema);
+ }
+ return fileSchema;
+ }
+
+ /**
+ * Performs schema repair on a schema, handling nullable unions.
+ */
+ private static Schema repairAvroSchema(Schema fileSchema, Schema tableSchema) {
+ // Always resolve nullable schemas first (returns unchanged if not a union)
+ Schema nonNullFileSchema = AvroSchemaUtils.getNonNullTypeFromUnion(fileSchema);
+ Schema nonNullTableSchema = AvroSchemaUtils.getNonNullTypeFromUnion(tableSchema);
+
+ // Perform repair on the non-null types
+ Schema nonNullRepairedSchema = repairAvroSchemaNonNull(nonNullFileSchema, nonNullTableSchema);
+
+ // If nothing changed, return the original schema
+ if (nonNullRepairedSchema == nonNullFileSchema) {
+ return fileSchema;
+ }
+
+ // If the original was a union, wrap the repaired schema back in a nullable union
+ if (fileSchema.getType() == Schema.Type.UNION) {
+ return AvroSchemaUtils.createNullableSchema(nonNullRepairedSchema);
+ }
+
+ return nonNullRepairedSchema;
+ }
+
+ /**
+ * Repairs non-nullable schemas (after unions have been resolved).
+ */
+ private static Schema repairAvroSchemaNonNull(Schema fileSchema, Schema tableSchema) {
+ // If schemas are already equal, nothing to repair
+ if (fileSchema.equals(tableSchema)) {
+ return fileSchema;
+ }
+
+ // If types are different, no repair can be done
+ if (fileSchema.getType() != tableSchema.getType()) {
+ return fileSchema;
+ }
+
+ // Handle record types (nested structs)
+ if (fileSchema.getType() == Schema.Type.RECORD) {
+ return repairRecord(fileSchema, tableSchema);
+ }
+
+ // Handle array types
+ if (fileSchema.getType() == Schema.Type.ARRAY) {
+ Schema repairedElementSchema = repairAvroSchema(fileSchema.getElementType(), tableSchema.getElementType());
+ // If element didn't change, return original array schema
+ if (repairedElementSchema == fileSchema.getElementType()) {
+ return fileSchema;
+ }
+ return Schema.createArray(repairedElementSchema);
+ }
+
+ // Handle map types
+ if (fileSchema.getType() == Schema.Type.MAP) {
+ Schema repairedValueSchema = repairAvroSchema(fileSchema.getValueType(), tableSchema.getValueType());
+ // If value didn't change, return original map schema
+ if (repairedValueSchema == fileSchema.getValueType()) {
+ return fileSchema;
+ }
+ return Schema.createMap(repairedValueSchema);
+ }
+
+ // Check primitive if we need to repair
+ if (needsLogicalTypeRepair(fileSchema, tableSchema)) {
+ // If we need to repair, return the table schema
+ return tableSchema;
+ }
+
+ // Default: return file schema
+ return fileSchema;
+ }
+
+ /**
+ * Quick check if a logical type repair is needed (no allocations).
+ */
+ private static boolean needsLogicalTypeRepair(Schema fileSchema, Schema tableSchema) {
+ if (fileSchema.getType() != Schema.Type.LONG || tableSchema.getType() != Schema.Type.LONG) {
+ return false;
+ }
+
+ LogicalType fileSchemaLogicalType = fileSchema.getLogicalType();
+ LogicalType tableSchemaLogicalType = tableSchema.getLogicalType();
+
+ // if file scheam has no logical type, and the table has a local timestamp, then we need to repair
+ if (fileSchemaLogicalType == null) {
+ try {
+ return tableSchemaLogicalType instanceof LogicalTypes.LocalTimestampMillis
+ || tableSchemaLogicalType instanceof LogicalTypes.LocalTimestampMicros;
+ } catch (Exception e) {
+ return false;
+ }
+ }
+
+ // if file schema is timestamp-micros, and the table is timestamp-millis, then we need to repair
+ return fileSchemaLogicalType instanceof LogicalTypes.TimestampMicros
+ && tableSchemaLogicalType instanceof LogicalTypes.TimestampMillis;
+ }
+
+ /**
+ * Performs record repair, returning the original schema if nothing changed.
+ */
+ private static Schema repairRecord(Schema fileSchema, Schema tableSchema) {
+ List fields = fileSchema.getFields();
+
+ // First pass: find the first field that changes
+ int firstChangedIndex = -1;
+ Schema firstRepairedSchema = null;
+
+ for (int i = 0; i < fields.size(); i++) {
+ Schema.Field requestedField = fields.get(i);
+ Schema.Field tableField = tableSchema.getField(requestedField.name());
+ if (tableField != null) {
+ Schema repairedSchema = repairAvroSchema(requestedField.schema(), tableField.schema());
+ if (repairedSchema != requestedField.schema()) {
+ firstChangedIndex = i;
+ firstRepairedSchema = repairedSchema;
+ break;
+ }
+ }
+ }
+
+ // If nothing changed, return the original schema
+ if (firstChangedIndex == -1) {
+ return fileSchema;
+ }
+
+ // Second pass: build the new schema with repaired fields
+ List repairedFields = new ArrayList<>(fields.size());
+
+ // Copy all fields before the first changed field
+ for (int i = 0; i < firstChangedIndex; i++) {
+ Schema.Field field = fields.get(i);
+ // Must create new Field since they cannot be reused
+ repairedFields.add(HoodieAvroUtils.createNewSchemaField(field));
+ }
+
+ // Add the first changed field (using cached repaired schema)
+ Schema.Field firstChangedField = fields.get(firstChangedIndex);
+ repairedFields.add(HoodieAvroUtils.createNewSchemaField(
+ firstChangedField.name(),
+ firstRepairedSchema,
+ firstChangedField.doc(),
+ firstChangedField.defaultVal()
+ ));
+
+ // Process remaining fields
+ for (int i = firstChangedIndex + 1; i < fields.size(); i++) {
+ Schema.Field requestedField = fields.get(i);
+ Schema.Field tableField = tableSchema.getField(requestedField.name());
+ Schema repairedSchema;
+
+ if (tableField != null) {
+ repairedSchema = repairAvroSchema(requestedField.schema(), tableField.schema());
+ } else {
+ repairedSchema = requestedField.schema();
+ }
+
+ // Must create new Field since they cannot be reused
+ repairedFields.add(HoodieAvroUtils.createNewSchemaField(
+ requestedField.name(),
+ repairedSchema,
+ requestedField.doc(),
+ requestedField.defaultVal()
+ ));
+ }
+
+ return Schema.createRecord(
+ fileSchema.getName(),
+ fileSchema.getDoc(),
+ fileSchema.getNamespace(),
+ fileSchema.isError(),
+ repairedFields
+ );
+ }
+
+ public static boolean hasTimestampMillisField(Schema tableSchema) {
+ switch (tableSchema.getType()) {
+ case RECORD:
+ for (Schema.Field field : tableSchema.getFields()) {
+ if (hasTimestampMillisField(field.schema())) {
+ return true;
+ }
+ }
+ return false;
+
+ case ARRAY:
+ return hasTimestampMillisField(tableSchema.getElementType());
+
+ case MAP:
+ return hasTimestampMillisField(tableSchema.getValueType());
+
+ case UNION:
+ return hasTimestampMillisField(AvroSchemaUtils.getNonNullTypeFromUnion(tableSchema));
+
+ default:
+ return tableSchema.getType() == Schema.Type.LONG
+ && (tableSchema.getLogicalType() instanceof LogicalTypes.TimestampMillis || tableSchema.getLogicalType() instanceof LogicalTypes.LocalTimestampMillis);
+ }
+ }
+}
diff --git a/hudi-common/src/avro/test/java/org/apache/parquet/schema/TestAvroSchemaRepair.java b/hudi-common/src/avro/test/java/org/apache/parquet/schema/TestAvroSchemaRepair.java
new file mode 100644
index 0000000000000..fb3d7e375b2c4
--- /dev/null
+++ b/hudi-common/src/avro/test/java/org/apache/parquet/schema/TestAvroSchemaRepair.java
@@ -0,0 +1,984 @@
+/*
+ * 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.parquet.schema;
+
+import org.apache.hudi.avro.AvroSchemaUtils;
+
+import org.apache.avro.LogicalTypes;
+import org.apache.avro.Schema;
+import org.apache.avro.SchemaBuilder;
+import org.junit.jupiter.api.Test;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotSame;
+import static org.junit.jupiter.api.Assertions.assertSame;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+/**
+ * Tests {@link AvroSchemaRepair}.
+ */
+public class TestAvroSchemaRepair {
+
+ @Test
+ public void testNoRepairNeededIdenticalSchemas() {
+ Schema requestedSchema = Schema.create(Schema.Type.LONG);
+ Schema tableSchema = Schema.create(Schema.Type.LONG);
+
+ Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema);
+
+ assertSame(requestedSchema, result, "When schemas are identical, should return same instance");
+
+ }
+
+ @Test
+ public void testNoRepairNeededDifferentPrimitiveTypes() {
+ Schema requestedSchema = Schema.create(Schema.Type.STRING);
+ Schema tableSchema = Schema.create(Schema.Type.INT);
+
+ Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema);
+
+ assertSame(requestedSchema, result, "When types differ, should return original schema");
+ }
+
+ @Test
+ public void testRepairLongWithoutLogicalTypeToLocalTimestampMillis() {
+ Schema requestedSchema = Schema.create(Schema.Type.LONG);
+ Schema tableSchema = LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG));
+
+ Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema);
+
+ assertNotSame(requestedSchema, result, "Should create a new schema with logical type");
+ assertEquals(Schema.Type.LONG, result.getType());
+ assertEquals(LogicalTypes.localTimestampMillis(), result.getLogicalType());
+ }
+
+ @Test
+ public void testRepairLongWithoutLogicalTypeToLocalTimestampMicros() {
+ Schema requestedSchema = Schema.create(Schema.Type.LONG);
+ Schema tableSchema = LogicalTypes.localTimestampMicros().addToSchema(Schema.create(Schema.Type.LONG));
+
+ Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema);
+
+ assertNotSame(requestedSchema, result, "Should create a new schema with logical type");
+ assertEquals(Schema.Type.LONG, result.getType());
+ assertEquals(LogicalTypes.localTimestampMicros(), result.getLogicalType());
+ }
+
+ @Test
+ public void testRepairTimestampMicrosToTimestampMillis() {
+ Schema requestedSchema = LogicalTypes.timestampMicros().addToSchema(Schema.create(Schema.Type.LONG));
+ Schema tableSchema = LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG));
+
+ Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema);
+
+ assertNotSame(requestedSchema, result, "Should create a new schema with timestamp-millis");
+ assertEquals(Schema.Type.LONG, result.getType());
+ assertEquals(LogicalTypes.timestampMillis(), result.getLogicalType());
+ }
+
+ @Test
+ public void testNoRepairNeededTimestampMillisToTimestampMicros() {
+ // This direction should NOT trigger repair
+ Schema requestedSchema = LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG));
+ Schema tableSchema = LogicalTypes.timestampMicros().addToSchema(Schema.create(Schema.Type.LONG));
+
+ Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema);
+
+ assertSame(requestedSchema, result, "Should not repair timestamp-millis to timestamp-micros");
+ }
+
+ @Test
+ public void testNoRepairNeededNonLongTypes() {
+ Schema requestedSchema = Schema.create(Schema.Type.INT);
+ Schema tableSchema = LogicalTypes.date().addToSchema(Schema.create(Schema.Type.INT));
+
+ Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema);
+
+ assertSame(requestedSchema, result, "Should not repair non-LONG types");
+ }
+
+ @Test
+ public void testRepairNullableSchemaLongToLocalTimestampMillis() {
+ Schema requestedSchema = Schema.createUnion(
+ Schema.create(Schema.Type.NULL),
+ Schema.create(Schema.Type.LONG)
+ );
+ Schema tableSchema = Schema.createUnion(
+ Schema.create(Schema.Type.NULL),
+ LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG))
+ );
+
+ Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema);
+
+ assertNotSame(requestedSchema, result, "Should create new nullable schema with repaired type");
+ assertEquals(Schema.Type.UNION, result.getType());
+ assertEquals(2, result.getTypes().size());
+
+ Schema nonNullType = AvroSchemaUtils.getNonNullTypeFromUnion(result);
+ assertEquals(LogicalTypes.localTimestampMillis(), nonNullType.getLogicalType());
+ }
+
+ @Test
+ public void testRepairNullableSchemaTimestampMicrosToMillis() {
+ Schema requestedSchema = Schema.createUnion(
+ Schema.create(Schema.Type.NULL),
+ LogicalTypes.timestampMicros().addToSchema(Schema.create(Schema.Type.LONG))
+ );
+ Schema tableSchema = Schema.createUnion(
+ Schema.create(Schema.Type.NULL),
+ LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG))
+ );
+
+ Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema);
+
+ assertNotSame(requestedSchema, result, "Should create new nullable schema");
+ assertEquals(Schema.Type.UNION, result.getType());
+
+ Schema nonNullType = AvroSchemaUtils.getNonNullTypeFromUnion(result);
+ assertEquals(LogicalTypes.timestampMillis(), nonNullType.getLogicalType());
+ }
+
+ @Test
+ public void testRepairRecordSingleField() {
+ Schema requestedSchema = SchemaBuilder.record("TestRecord")
+ .fields()
+ .name("timestamp").type().longType().noDefault()
+ .endRecord();
+
+ Schema tableSchema = SchemaBuilder.record("TestRecord")
+ .fields()
+ .name("timestamp").type(LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG))).noDefault()
+ .endRecord();
+
+ Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema);
+
+ assertNotSame(requestedSchema, result, "Should create new record schema");
+ assertEquals(Schema.Type.RECORD, result.getType());
+ assertEquals("TestRecord", result.getName());
+ assertEquals(1, result.getFields().size());
+
+ Schema.Field field = result.getField("timestamp");
+ assertEquals(LogicalTypes.localTimestampMillis(), field.schema().getLogicalType());
+ }
+
+ @Test
+ public void testRepairRecordMultipleFieldsOnlyOneNeedsRepair() {
+ Schema requestedSchema = SchemaBuilder.record("TestRecord")
+ .fields()
+ .name("id").type().intType().noDefault()
+ .name("timestamp").type().longType().noDefault()
+ .name("name").type().stringType().noDefault()
+ .endRecord();
+
+ Schema tableSchema = SchemaBuilder.record("TestRecord")
+ .fields()
+ .name("id").type().intType().noDefault()
+ .name("timestamp").type(LogicalTypes.localTimestampMicros().addToSchema(Schema.create(Schema.Type.LONG))).noDefault()
+ .name("name").type().stringType().noDefault()
+ .endRecord();
+
+ Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema);
+
+ assertNotSame(requestedSchema, result, "Should create new record schema");
+ assertEquals(3, result.getFields().size());
+
+ // Verify id field unchanged - should be same schema instance
+ assertSame(requestedSchema.getField("id").schema(), result.getField("id").schema());
+
+ // Verify timestamp field repaired
+ assertEquals(LogicalTypes.localTimestampMicros(), result.getField("timestamp").schema().getLogicalType());
+
+ // Verify name field unchanged - should be same schema instance
+ assertSame(requestedSchema.getField("name").schema(), result.getField("name").schema());
+ }
+
+ @Test
+ public void testRepairRecordNestedRecord() {
+ Schema nestedRequestedSchema = SchemaBuilder.record("NestedRecord")
+ .fields()
+ .name("timestamp").type().longType().noDefault()
+ .endRecord();
+
+ Schema nestedTableSchema = SchemaBuilder.record("NestedRecord")
+ .fields()
+ .name("timestamp").type(LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG))).noDefault()
+ .endRecord();
+
+ Schema requestedSchema = SchemaBuilder.record("OuterRecord")
+ .fields()
+ .name("id").type().intType().noDefault()
+ .name("nested").type(nestedRequestedSchema).noDefault()
+ .endRecord();
+
+ Schema tableSchema = SchemaBuilder.record("OuterRecord")
+ .fields()
+ .name("id").type().intType().noDefault()
+ .name("nested").type(nestedTableSchema).noDefault()
+ .endRecord();
+
+ Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema);
+
+ assertNotSame(requestedSchema, result, "Should create new schema for nested record");
+
+ // Verify id field unchanged - should be same schema instance
+ assertSame(requestedSchema.getField("id").schema(), result.getField("id").schema());
+
+ // Verify nested record was repaired
+ Schema nestedResult = result.getField("nested").schema();
+ assertEquals(Schema.Type.RECORD, nestedResult.getType());
+ assertEquals(LogicalTypes.localTimestampMillis(),
+ nestedResult.getField("timestamp").schema().getLogicalType());
+ }
+
+ @Test
+ public void testRepairRecordNullableNestedField() {
+ Schema requestedSchema = SchemaBuilder.record("TestRecord")
+ .fields()
+ .name("timestamp").type().optional().longType()
+ .endRecord();
+
+ Schema tableSchema = SchemaBuilder.record("TestRecord")
+ .fields()
+ .name("timestamp").type().optional().type(LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG)))
+ .endRecord();
+
+ Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema);
+
+ assertNotSame(requestedSchema, result, "Should create new schema");
+
+ Schema fieldSchema = result.getField("timestamp").schema();
+ assertEquals(Schema.Type.UNION, fieldSchema.getType());
+
+ Schema nonNullType = AvroSchemaUtils.getNonNullTypeFromUnion(fieldSchema);
+ assertEquals(LogicalTypes.localTimestampMillis(), nonNullType.getLogicalType());
+ }
+
+ @Test
+ public void testRepairArrayElementNeedsRepair() {
+ Schema requestedSchema = Schema.createArray(Schema.create(Schema.Type.LONG));
+ Schema tableSchema = Schema.createArray(
+ LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG))
+ );
+
+ Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema);
+
+ assertNotSame(requestedSchema, result, "Should create new array schema");
+ assertEquals(Schema.Type.ARRAY, result.getType());
+ assertEquals(LogicalTypes.localTimestampMillis(), result.getElementType().getLogicalType());
+ }
+
+ @Test
+ public void testRepairArrayNoRepairNeeded() {
+ Schema elementSchema = Schema.create(Schema.Type.STRING);
+ Schema requestedSchema = Schema.createArray(elementSchema);
+ Schema tableSchema = Schema.createArray(elementSchema);
+
+ Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema);
+
+ assertSame(requestedSchema, result, "Should return same array when no repair needed");
+ }
+
+ @Test
+ public void testRepairArrayNullableElements() {
+ Schema requestedSchema = Schema.createArray(
+ Schema.createUnion(Schema.create(Schema.Type.NULL), Schema.create(Schema.Type.LONG))
+ );
+ Schema tableSchema = Schema.createArray(
+ Schema.createUnion(
+ Schema.create(Schema.Type.NULL),
+ LogicalTypes.localTimestampMicros().addToSchema(Schema.create(Schema.Type.LONG))
+ )
+ );
+
+ Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema);
+
+ assertNotSame(requestedSchema, result, "Should create new array schema");
+ Schema elementSchema = result.getElementType();
+ assertEquals(Schema.Type.UNION, elementSchema.getType());
+
+ Schema nonNullType = AvroSchemaUtils.getNonNullTypeFromUnion(elementSchema);
+ assertEquals(LogicalTypes.localTimestampMicros(), nonNullType.getLogicalType());
+ }
+
+ @Test
+ public void testRepairMapValueNeedsRepair() {
+ Schema requestedSchema = Schema.createMap(Schema.create(Schema.Type.LONG));
+ Schema tableSchema = Schema.createMap(
+ LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG))
+ );
+
+ Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema);
+
+ assertNotSame(requestedSchema, result, "Should create new map schema");
+ assertEquals(Schema.Type.MAP, result.getType());
+ assertEquals(LogicalTypes.localTimestampMillis(), result.getValueType().getLogicalType());
+ }
+
+ @Test
+ public void testRepairMapNoRepairNeeded() {
+ Schema valueSchema = Schema.create(Schema.Type.STRING);
+ Schema requestedSchema = Schema.createMap(valueSchema);
+ Schema tableSchema = Schema.createMap(valueSchema);
+
+ Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema);
+
+ assertSame(requestedSchema, result, "Should return same map when no repair needed");
+ }
+
+ @Test
+ public void testRepairMapNullableValues() {
+ Schema requestedSchema = Schema.createMap(
+ Schema.createUnion(Schema.create(Schema.Type.NULL), Schema.create(Schema.Type.LONG))
+ );
+ Schema tableSchema = Schema.createMap(
+ Schema.createUnion(
+ Schema.create(Schema.Type.NULL),
+ LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG))
+ )
+ );
+
+ Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema);
+
+ assertNotSame(requestedSchema, result, "Should create new map schema");
+ Schema valueSchema = result.getValueType();
+ assertEquals(Schema.Type.UNION, valueSchema.getType());
+
+ Schema nonNullType = AvroSchemaUtils.getNonNullTypeFromUnion(valueSchema);
+ assertEquals(LogicalTypes.localTimestampMillis(), nonNullType.getLogicalType());
+ }
+
+ @Test
+ public void testComplexSchemaMultiLevelNesting() {
+ // Create a complex schema with nested records, arrays, and maps
+ Schema innerRecordRequested = SchemaBuilder.record("Inner")
+ .fields()
+ .name("timestamp").type().longType().noDefault()
+ .endRecord();
+
+ Schema innerRecordTable = SchemaBuilder.record("Inner")
+ .fields()
+ .name("timestamp").type(LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG))).noDefault()
+ .endRecord();
+
+ Schema requestedSchema = SchemaBuilder.record("Outer")
+ .fields()
+ .name("id").type().intType().noDefault()
+ .name("records").type().array().items(innerRecordRequested).noDefault()
+ .name("mapping").type().map().values(Schema.create(Schema.Type.LONG)).noDefault()
+ .endRecord();
+
+ Schema tableSchema = SchemaBuilder.record("Outer")
+ .fields()
+ .name("id").type().intType().noDefault()
+ .name("records").type().array().items(innerRecordTable).noDefault()
+ .name("mapping").type().map().values(
+ LogicalTypes.localTimestampMicros().addToSchema(Schema.create(Schema.Type.LONG))
+ ).noDefault()
+ .endRecord();
+
+ Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema);
+
+ assertNotSame(requestedSchema, result, "Should create new complex schema");
+
+ // Verify id field unchanged - should be same schema instance
+ assertSame(requestedSchema.getField("id").schema(), result.getField("id").schema());
+
+ // Verify array of records was repaired
+ Schema arrayElementSchema = result.getField("records").schema().getElementType();
+ assertEquals(LogicalTypes.localTimestampMillis(),
+ arrayElementSchema.getField("timestamp").schema().getLogicalType());
+
+ // Verify map values were repaired
+ Schema mapValueSchema = result.getField("mapping").schema().getValueType();
+ assertEquals(LogicalTypes.localTimestampMicros(), mapValueSchema.getLogicalType());
+ }
+
+ @Test
+ public void testRepairRecordMissingFieldInTableSchema() {
+ // Requested schema has a field not present in table schema
+ Schema requestedSchema = SchemaBuilder.record("TestRecord")
+ .fields()
+ .name("id").type().intType().noDefault()
+ .name("newField").type().longType().noDefault()
+ .endRecord();
+
+ Schema tableSchema = SchemaBuilder.record("TestRecord")
+ .fields()
+ .name("id").type().intType().noDefault()
+ .endRecord();
+
+ Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema);
+
+ // Should return original schema unchanged since newField doesn't exist in table schema
+ assertSame(requestedSchema, result, "Should return original when field missing in table schema");
+ }
+
+ @Test
+ public void testRepairRecordMultipleFieldsMissingInTableSchema() {
+ // Requested schema has multiple fields not present in table schema
+ Schema requestedSchema = SchemaBuilder.record("TestRecord")
+ .fields()
+ .name("id").type().intType().noDefault()
+ .name("newField1").type().longType().noDefault()
+ .name("name").type().stringType().noDefault()
+ .name("newField2").type().longType().noDefault()
+ .endRecord();
+
+ Schema tableSchema = SchemaBuilder.record("TestRecord")
+ .fields()
+ .name("id").type().intType().noDefault()
+ .name("name").type().stringType().noDefault()
+ .endRecord();
+
+ Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema);
+
+ // Should return original schema unchanged since new fields don't exist in table schema
+ assertSame(requestedSchema, result, "Should return original when multiple fields missing in table schema");
+ }
+
+ @Test
+ public void testRepairRecordMixedMissingAndRepairableFields() {
+ // Requested schema has some fields missing in table, some needing repair, some unchanged
+ Schema requestedSchema = SchemaBuilder.record("TestRecord")
+ .fields()
+ .name("id").type().intType().noDefault()
+ .name("timestamp").type().longType().noDefault()
+ .name("newField").type().longType().noDefault()
+ .name("name").type().stringType().noDefault()
+ .endRecord();
+
+ Schema tableSchema = SchemaBuilder.record("TestRecord")
+ .fields()
+ .name("id").type().intType().noDefault()
+ .name("timestamp").type(LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG))).noDefault()
+ .name("name").type().stringType().noDefault()
+ .endRecord();
+
+ Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema);
+
+ // Should create new schema with timestamp repaired, but newField preserved from requested
+ assertNotSame(requestedSchema, result, "Should create new schema");
+ assertEquals(4, result.getFields().size());
+
+ // Verify id field unchanged
+ assertSame(requestedSchema.getField("id").schema(), result.getField("id").schema());
+
+ // Verify timestamp field repaired
+ assertEquals(LogicalTypes.localTimestampMillis(), result.getField("timestamp").schema().getLogicalType());
+
+ // Verify newField preserved from requested schema (not in table)
+ assertSame(requestedSchema.getField("newField").schema(), result.getField("newField").schema());
+
+ // Verify name field unchanged
+ assertSame(requestedSchema.getField("name").schema(), result.getField("name").schema());
+ }
+
+ @Test
+ public void testRepairNestedRecordFieldMissingInTableSchema() {
+ // Requested nested record has a field not present in table's nested record
+ Schema nestedRequestedSchema = SchemaBuilder.record("NestedRecord")
+ .fields()
+ .name("timestamp").type().longType().noDefault()
+ .name("extraField").type().stringType().noDefault()
+ .endRecord();
+
+ Schema nestedTableSchema = SchemaBuilder.record("NestedRecord")
+ .fields()
+ .name("timestamp").type(LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG))).noDefault()
+ .endRecord();
+
+ Schema requestedSchema = SchemaBuilder.record("OuterRecord")
+ .fields()
+ .name("id").type().intType().noDefault()
+ .name("nested").type(nestedRequestedSchema).noDefault()
+ .endRecord();
+
+ Schema tableSchema = SchemaBuilder.record("OuterRecord")
+ .fields()
+ .name("id").type().intType().noDefault()
+ .name("nested").type(nestedTableSchema).noDefault()
+ .endRecord();
+
+ Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema);
+
+ assertNotSame(requestedSchema, result, "Should create new schema");
+
+ // Verify id field unchanged
+ assertSame(requestedSchema.getField("id").schema(), result.getField("id").schema());
+
+ // Verify nested record was repaired but still has extraField
+ Schema nestedResult = result.getField("nested").schema();
+ assertEquals(Schema.Type.RECORD, nestedResult.getType());
+ assertEquals(2, nestedResult.getFields().size());
+
+ // Timestamp should be repaired
+ assertEquals(LogicalTypes.localTimestampMillis(),
+ nestedResult.getField("timestamp").schema().getLogicalType());
+
+ // extraField should be preserved from requested schema
+ assertSame(nestedRequestedSchema.getField("extraField").schema(),
+ nestedResult.getField("extraField").schema());
+ }
+
+ @Test
+ public void testRepairRecordWholeNestedRecordMissingInTableSchema() {
+ // Requested schema has a nested record field that doesn't exist in table schema
+ Schema nestedRequestedSchema = SchemaBuilder.record("NestedRecord")
+ .fields()
+ .name("timestamp").type().longType().noDefault()
+ .endRecord();
+
+ Schema requestedSchema = SchemaBuilder.record("OuterRecord")
+ .fields()
+ .name("id").type().intType().noDefault()
+ .name("newNested").type(nestedRequestedSchema).noDefault()
+ .endRecord();
+
+ Schema tableSchema = SchemaBuilder.record("OuterRecord")
+ .fields()
+ .name("id").type().intType().noDefault()
+ .endRecord();
+
+ Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema);
+
+ // Should return original schema unchanged since newNested field doesn't exist in table
+ assertSame(requestedSchema, result, "Should return original when nested field missing in table schema");
+ }
+
+ @Test
+ public void testRepairRecordPreservesFieldMetadata() {
+ Schema requestedSchema = SchemaBuilder.record("TestRecord")
+ .doc("Test documentation")
+ .fields()
+ .name("timestamp").doc("Timestamp field").type().longType().noDefault()
+ .endRecord();
+
+ Schema tableSchema = SchemaBuilder.record("TestRecord")
+ .fields()
+ .name("timestamp").type(LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG))).noDefault()
+ .endRecord();
+
+ Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema);
+
+ assertNotSame(requestedSchema, result);
+ assertEquals("TestRecord", result.getName());
+ assertEquals("Test documentation", result.getDoc());
+ assertEquals("Timestamp field", result.getField("timestamp").doc());
+ }
+
+ @Test
+ public void testEdgeCaseEmptyRecord() {
+ Schema requestedSchema = SchemaBuilder.record("EmptyRecord").fields().endRecord();
+ Schema tableSchema = SchemaBuilder.record("EmptyRecord").fields().endRecord();
+
+ Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema);
+
+ assertSame(requestedSchema, result, "Empty records should return same instance");
+ }
+
+ @Test
+ public void testRepairRecordFirstFieldChanged() {
+ // Test the optimization path where the first field needs repair
+ Schema requestedSchema = SchemaBuilder.record("TestRecord")
+ .fields()
+ .name("timestamp1").type().longType().noDefault()
+ .name("timestamp2").type().longType().noDefault()
+ .endRecord();
+
+ Schema tableSchema = SchemaBuilder.record("TestRecord")
+ .fields()
+ .name("timestamp1").type(LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG))).noDefault()
+ .name("timestamp2").type(LogicalTypes.localTimestampMicros().addToSchema(Schema.create(Schema.Type.LONG))).noDefault()
+ .endRecord();
+
+ Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema);
+
+ assertNotSame(requestedSchema, result);
+ assertEquals(LogicalTypes.localTimestampMillis(), result.getField("timestamp1").schema().getLogicalType());
+ assertEquals(LogicalTypes.localTimestampMicros(), result.getField("timestamp2").schema().getLogicalType());
+ }
+
+ @Test
+ public void testRepairRecordLastFieldChanged() {
+ // Test the optimization path where only the last field needs repair
+ Schema requestedSchema = SchemaBuilder.record("TestRecord")
+ .fields()
+ .name("id").type().intType().noDefault()
+ .name("name").type().stringType().noDefault()
+ .name("timestamp").type().longType().noDefault()
+ .endRecord();
+
+ Schema tableSchema = SchemaBuilder.record("TestRecord")
+ .fields()
+ .name("id").type().intType().noDefault()
+ .name("name").type().stringType().noDefault()
+ .name("timestamp").type(LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG))).noDefault()
+ .endRecord();
+
+ Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema);
+
+ assertNotSame(requestedSchema, result);
+ // Verify id and name fields unchanged - should be same schema instances
+ assertSame(requestedSchema.getField("id").schema(), result.getField("id").schema());
+ assertSame(requestedSchema.getField("name").schema(), result.getField("name").schema());
+ // Verify timestamp field repaired
+ assertEquals(LogicalTypes.localTimestampMillis(), result.getField("timestamp").schema().getLogicalType());
+ }
+
+ @Test
+ public void testHasTimestampMillisFieldPrimitiveLongWithTimestampMillis() {
+ Schema schema = LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG));
+ assertTrue(AvroSchemaRepair.hasTimestampMillisField(schema),
+ "Should return true for LONG with timestamp-millis logical type");
+ }
+
+ @Test
+ public void testHasTimestampMillisFieldPrimitiveLongWithoutLogicalType() {
+ Schema schema = Schema.create(Schema.Type.LONG);
+ assertFalse(AvroSchemaRepair.hasTimestampMillisField(schema),
+ "Should return false for LONG without logical type");
+ }
+
+ @Test
+ public void testHasTimestampMillisFieldPrimitiveLongWithTimestampMicros() {
+ Schema schema = LogicalTypes.timestampMicros().addToSchema(Schema.create(Schema.Type.LONG));
+ assertFalse(AvroSchemaRepair.hasTimestampMillisField(schema),
+ "Should return false for LONG with timestamp-micros logical type");
+ }
+
+ @Test
+ public void testHasTimestampMillisFieldPrimitiveLongWithLocalTimestampMillis() {
+ Schema schema = LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG));
+ assertTrue(AvroSchemaRepair.hasTimestampMillisField(schema),
+ "Should return true for LONG with local-timestamp-millis logical type");
+ }
+
+ @Test
+ public void testHasTimestampMillisFieldPrimitiveLongWithLocalTimestampMicros() {
+ Schema schema = LogicalTypes.localTimestampMicros().addToSchema(Schema.create(Schema.Type.LONG));
+ assertFalse(AvroSchemaRepair.hasTimestampMillisField(schema),
+ "Should return false for LONG with local-timestamp-micros logical type");
+ }
+
+ @Test
+ public void testHasTimestampMillisFieldOtherPrimitiveTypes() {
+ assertFalse(AvroSchemaRepair.hasTimestampMillisField(Schema.create(Schema.Type.STRING)),
+ "Should return false for STRING type");
+ assertFalse(AvroSchemaRepair.hasTimestampMillisField(Schema.create(Schema.Type.INT)),
+ "Should return false for INT type");
+ assertFalse(AvroSchemaRepair.hasTimestampMillisField(Schema.create(Schema.Type.FLOAT)),
+ "Should return false for FLOAT type");
+ assertFalse(AvroSchemaRepair.hasTimestampMillisField(Schema.create(Schema.Type.DOUBLE)),
+ "Should return false for DOUBLE type");
+ assertFalse(AvroSchemaRepair.hasTimestampMillisField(Schema.create(Schema.Type.BOOLEAN)),
+ "Should return false for BOOLEAN type");
+ }
+
+ @Test
+ public void testHasTimestampMillisFieldRecordWithTimestampMillis() {
+ Schema schema = SchemaBuilder.record("TestRecord")
+ .fields()
+ .name("id").type().intType().noDefault()
+ .name("timestamp").type(LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG))).noDefault()
+ .name("name").type().stringType().noDefault()
+ .endRecord();
+
+ assertTrue(AvroSchemaRepair.hasTimestampMillisField(schema),
+ "Should return true for record containing timestamp-millis field");
+ }
+
+ @Test
+ public void testHasTimestampMillisFieldRecordWithoutTimestampMillis() {
+ Schema schema = SchemaBuilder.record("TestRecord")
+ .fields()
+ .name("id").type().intType().noDefault()
+ .name("timestamp").type(LogicalTypes.timestampMicros().addToSchema(Schema.create(Schema.Type.LONG))).noDefault()
+ .name("name").type().stringType().noDefault()
+ .endRecord();
+
+ assertFalse(AvroSchemaRepair.hasTimestampMillisField(schema),
+ "Should return false for record without timestamp-millis field");
+ }
+
+ @Test
+ public void testHasTimestampMillisFieldRecordEmpty() {
+ Schema schema = SchemaBuilder.record("EmptyRecord").fields().endRecord();
+
+ assertFalse(AvroSchemaRepair.hasTimestampMillisField(schema),
+ "Should return false for empty record");
+ }
+
+ @Test
+ public void testHasTimestampMillisFieldNestedRecord() {
+ Schema innerSchema = SchemaBuilder.record("InnerRecord")
+ .fields()
+ .name("timestamp").type(LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG))).noDefault()
+ .endRecord();
+
+ Schema outerSchema = SchemaBuilder.record("OuterRecord")
+ .fields()
+ .name("id").type().intType().noDefault()
+ .name("inner").type(innerSchema).noDefault()
+ .endRecord();
+
+ assertTrue(AvroSchemaRepair.hasTimestampMillisField(outerSchema),
+ "Should return true for nested record containing timestamp-millis field");
+ }
+
+ @Test
+ public void testHasTimestampMillisFieldDeeplyNestedRecord() {
+ Schema level3 = SchemaBuilder.record("Level3")
+ .fields()
+ .name("timestamp").type(LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG))).noDefault()
+ .endRecord();
+
+ Schema level2 = SchemaBuilder.record("Level2")
+ .fields()
+ .name("data").type(level3).noDefault()
+ .endRecord();
+
+ Schema level1 = SchemaBuilder.record("Level1")
+ .fields()
+ .name("nested").type(level2).noDefault()
+ .endRecord();
+
+ assertTrue(AvroSchemaRepair.hasTimestampMillisField(level1),
+ "Should return true for deeply nested record containing timestamp-millis field");
+ }
+
+ @Test
+ public void testHasTimestampMillisFieldArrayWithTimestampMillis() {
+ Schema schema = Schema.createArray(
+ LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG))
+ );
+
+ assertTrue(AvroSchemaRepair.hasTimestampMillisField(schema),
+ "Should return true for array with timestamp-millis elements");
+ }
+
+ @Test
+ public void testHasTimestampMillisFieldArrayWithoutTimestampMillis() {
+ Schema schema = Schema.createArray(Schema.create(Schema.Type.STRING));
+
+ assertFalse(AvroSchemaRepair.hasTimestampMillisField(schema),
+ "Should return false for array without timestamp-millis elements");
+ }
+
+ @Test
+ public void testHasTimestampMillisFieldArrayOfRecordsWithTimestampMillis() {
+ Schema elementSchema = SchemaBuilder.record("Element")
+ .fields()
+ .name("timestamp").type(LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG))).noDefault()
+ .endRecord();
+
+ Schema schema = Schema.createArray(elementSchema);
+
+ assertTrue(AvroSchemaRepair.hasTimestampMillisField(schema),
+ "Should return true for array of records containing timestamp-millis field");
+ }
+
+ @Test
+ public void testHasTimestampMillisFieldMapWithTimestampMillis() {
+ Schema schema = Schema.createMap(
+ LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG))
+ );
+
+ assertTrue(AvroSchemaRepair.hasTimestampMillisField(schema),
+ "Should return true for map with timestamp-millis values");
+ }
+
+ @Test
+ public void testHasTimestampMillisFieldMapWithoutTimestampMillis() {
+ Schema schema = Schema.createMap(Schema.create(Schema.Type.STRING));
+
+ assertFalse(AvroSchemaRepair.hasTimestampMillisField(schema),
+ "Should return false for map without timestamp-millis values");
+ }
+
+ @Test
+ public void testHasTimestampMillisFieldMapOfRecordsWithTimestampMillis() {
+ Schema valueSchema = SchemaBuilder.record("Value")
+ .fields()
+ .name("timestamp").type(LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG))).noDefault()
+ .endRecord();
+
+ Schema schema = Schema.createMap(valueSchema);
+
+ assertTrue(AvroSchemaRepair.hasTimestampMillisField(schema),
+ "Should return true for map of records containing timestamp-millis field");
+ }
+
+ @Test
+ public void testHasTimestampMillisFieldUnionWithTimestampMillis() {
+ Schema schema = Schema.createUnion(
+ Schema.create(Schema.Type.NULL),
+ LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG))
+ );
+
+ assertTrue(AvroSchemaRepair.hasTimestampMillisField(schema),
+ "Should return true for nullable union with timestamp-millis");
+ }
+
+ @Test
+ public void testHasTimestampMillisFieldUnionWithoutTimestampMillis() {
+ Schema schema = Schema.createUnion(
+ Schema.create(Schema.Type.NULL),
+ Schema.create(Schema.Type.LONG)
+ );
+
+ assertFalse(AvroSchemaRepair.hasTimestampMillisField(schema),
+ "Should return false for nullable union without timestamp-millis");
+ }
+
+ @Test
+ public void testHasTimestampMillisFieldUnionWithRecordContainingTimestampMillis() {
+ Schema recordSchema = SchemaBuilder.record("Record")
+ .fields()
+ .name("timestamp").type(LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG))).noDefault()
+ .endRecord();
+
+ Schema schema = Schema.createUnion(
+ Schema.create(Schema.Type.NULL),
+ recordSchema
+ );
+
+ assertTrue(AvroSchemaRepair.hasTimestampMillisField(schema),
+ "Should return true for nullable union with record containing timestamp-millis");
+ }
+
+ @Test
+ public void testHasTimestampMillisFieldComplexNestedStructure() {
+ // Create a complex schema with arrays, maps, and nested records
+ Schema innerRecordSchema = SchemaBuilder.record("InnerRecord")
+ .fields()
+ .name("timestamp").type(LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG))).noDefault()
+ .endRecord();
+
+ Schema complexSchema = SchemaBuilder.record("ComplexRecord")
+ .fields()
+ .name("id").type().intType().noDefault()
+ .name("arrayOfRecords").type().array().items(innerRecordSchema).noDefault()
+ .name("mapOfStrings").type().map().values().stringType().noDefault()
+ .endRecord();
+
+ assertTrue(AvroSchemaRepair.hasTimestampMillisField(complexSchema),
+ "Should return true for complex nested structure containing timestamp-millis field");
+ }
+
+ @Test
+ public void testHasTimestampMillisFieldComplexStructureWithoutTimestampMillis() {
+ Schema innerRecordSchema = SchemaBuilder.record("InnerRecord")
+ .fields()
+ .name("value").type().longType().noDefault()
+ .endRecord();
+
+ Schema complexSchema = SchemaBuilder.record("ComplexRecord")
+ .fields()
+ .name("id").type().intType().noDefault()
+ .name("arrayOfRecords").type().array().items(innerRecordSchema).noDefault()
+ .name("mapOfLongs").type().map().values(
+ LogicalTypes.timestampMicros().addToSchema(Schema.create(Schema.Type.LONG))
+ ).noDefault()
+ .endRecord();
+
+ assertFalse(AvroSchemaRepair.hasTimestampMillisField(complexSchema),
+ "Should return false for complex structure without timestamp-millis field");
+ }
+
+ @Test
+ public void testHasTimestampMillisFieldFirstFieldMatches() {
+ Schema schema = SchemaBuilder.record("TestRecord")
+ .fields()
+ .name("timestamp").type(LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG))).noDefault()
+ .name("id").type().intType().noDefault()
+ .name("name").type().stringType().noDefault()
+ .endRecord();
+
+ assertTrue(AvroSchemaRepair.hasTimestampMillisField(schema),
+ "Should return true when first field is timestamp-millis");
+ }
+
+ @Test
+ public void testHasTimestampMillisFieldLastFieldMatches() {
+ Schema schema = SchemaBuilder.record("TestRecord")
+ .fields()
+ .name("id").type().intType().noDefault()
+ .name("name").type().stringType().noDefault()
+ .name("timestamp").type(LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG))).noDefault()
+ .endRecord();
+
+ assertTrue(AvroSchemaRepair.hasTimestampMillisField(schema),
+ "Should return true when last field is timestamp-millis");
+ }
+
+ @Test
+ public void testHasTimestampMillisFieldMultipleTimestampMillisFields() {
+ Schema schema = SchemaBuilder.record("TestRecord")
+ .fields()
+ .name("createdAt").type(LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG))).noDefault()
+ .name("id").type().intType().noDefault()
+ .name("updatedAt").type(LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG))).noDefault()
+ .endRecord();
+
+ assertTrue(AvroSchemaRepair.hasTimestampMillisField(schema),
+ "Should return true when multiple timestamp-millis fields exist");
+ }
+
+ @Test
+ public void testHasTimestampMillisFieldNullableFieldWithTimestampMillis() {
+ Schema schema = SchemaBuilder.record("TestRecord")
+ .fields()
+ .name("id").type().intType().noDefault()
+ .name("timestamp").type().optional().type(
+ LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG))
+ )
+ .endRecord();
+
+ assertTrue(AvroSchemaRepair.hasTimestampMillisField(schema),
+ "Should return true for nullable field with timestamp-millis");
+ }
+
+ @Test
+ public void testHasTimestampMillisFieldArrayOfNullableTimestampMillis() {
+ Schema elementSchema = Schema.createUnion(
+ Schema.create(Schema.Type.NULL),
+ LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG))
+ );
+
+ Schema schema = Schema.createArray(elementSchema);
+
+ assertTrue(AvroSchemaRepair.hasTimestampMillisField(schema),
+ "Should return true for array of nullable timestamp-millis elements");
+ }
+
+ @Test
+ public void testHasTimestampMillisFieldMapOfNullableTimestampMillis() {
+ Schema valueSchema = Schema.createUnion(
+ Schema.create(Schema.Type.NULL),
+ LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG))
+ );
+
+ Schema schema = Schema.createMap(valueSchema);
+
+ assertTrue(AvroSchemaRepair.hasTimestampMillisField(schema),
+ "Should return true for map of nullable timestamp-millis values");
+ }
+}
diff --git a/hudi-common/src/main/java/org/apache/hudi/avro/AvroSchemaCache.java b/hudi-common/src/main/java/org/apache/hudi/avro/AvroSchemaCache.java
new file mode 100644
index 0000000000000..cecc420813c3b
--- /dev/null
+++ b/hudi-common/src/main/java/org/apache/hudi/avro/AvroSchemaCache.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.avro;
+
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.LoadingCache;
+import org.apache.avro.Schema;
+
+/**
+ * An avro schema cache implementation for reusing avro schema instantces in JVM/process scope.
+ * This is a global cache which works for a JVM lifecycle.
+ * A collection of schema instants are maintained.
+ *
+ *
NOTE: The schema which be used frequently should be cached through this cache.
+ */
+public class AvroSchemaCache {
+
+ // Ensure that there is only one variable instance of the same schema within an entire JVM lifetime
+ private static final LoadingCache SCHEMA_CACHE = Caffeine.newBuilder().weakValues().maximumSize(1024).build(k -> k);
+
+ /**
+ * Get schema variable from global cache. If not found, put it into the cache and then return it.
+ * @param schema schema to get
+ * @return if found, return the exist schema variable, otherwise return the param itself.
+ */
+ public static Schema intern(Schema schema) {
+ return SCHEMA_CACHE.get(schema);
+ }
+}
diff --git a/hudi-common/src/main/java/org/apache/hudi/avro/AvroSchemaUtils.java b/hudi-common/src/main/java/org/apache/hudi/avro/AvroSchemaUtils.java
index 3c5486c47c742..b12d89ff0c8ff 100644
--- a/hudi-common/src/main/java/org/apache/hudi/avro/AvroSchemaUtils.java
+++ b/hudi-common/src/main/java/org/apache/hudi/avro/AvroSchemaUtils.java
@@ -18,17 +18,25 @@
package org.apache.hudi.avro;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.exception.HoodieAvroSchemaException;
import org.apache.hudi.exception.SchemaCompatibilityException;
+import org.apache.hudi.io.storage.HoodieFileReader;
import org.apache.avro.AvroRuntimeException;
import org.apache.avro.Schema;
import org.apache.avro.SchemaCompatibility;
+import org.apache.hadoop.conf.Configuration;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
import java.util.Collections;
import java.util.List;
import java.util.Objects;
import java.util.Set;
import java.util.function.BiFunction;
+import java.util.function.Supplier;
import java.util.stream.Collectors;
import static org.apache.hudi.common.util.ValidationUtils.checkState;
@@ -38,6 +46,17 @@
*/
public class AvroSchemaUtils {
+ private static final Class> AVRO_SCHEMA_REPAIR_CLASS;
+ static {
+ Class> clazz = null;
+ try {
+ clazz = Class.forName("org.apache.parquet.schema.AvroSchemaRepair");
+ } catch (ClassNotFoundException e) {
+ // AvroSchemaRepair not on classpath (e.g. when parquet schema not available)
+ }
+ AVRO_SCHEMA_REPAIR_CLASS = clazz;
+ }
+
private AvroSchemaUtils() {}
/**
@@ -218,6 +237,21 @@ private static boolean isProjectionOfInternal(Schema sourceSchema,
return atomicTypeEqualityPredicate.apply(sourceSchema, targetSchema);
}
+ public static Option findNestedFieldSchema(Schema schema, String fieldName) {
+ if (StringUtils.isNullOrEmpty(fieldName)) {
+ return Option.empty();
+ }
+ String[] parts = fieldName.split("\\.");
+ for (String part : parts) {
+ Schema.Field foundField = getNonNullTypeFromUnion(schema).getField(part);
+ if (foundField == null) {
+ throw new HoodieAvroSchemaException(fieldName + " not a field in " + schema);
+ }
+ schema = foundField.schema();
+ }
+ return Option.of(getNonNullTypeFromUnion(schema));
+ }
+
/**
* Appends provided new fields at the end of the given schema
*
@@ -251,7 +285,7 @@ public static Schema resolveUnionSchema(Schema schema, String fieldSchemaFullNam
List innerTypes = schema.getTypes();
if (innerTypes.size() == 2 && isNullable(schema)) {
// this is a basic nullable field so handle it more efficiently
- return resolveNullableSchema(schema);
+ return getNonNullTypeFromUnion(schema);
}
Schema nonNullType =
@@ -285,7 +319,7 @@ public static boolean isNullable(Schema schema) {
* Resolves typical Avro's nullable schema definition: {@code Union(Schema.Type.NULL, )},
* decomposing union and returning the target non-null type
*/
- public static Schema resolveNullableSchema(Schema schema) {
+ public static Schema getNonNullTypeFromUnion(Schema schema) {
if (schema.getType() != Schema.Type.UNION) {
return schema;
}
@@ -373,4 +407,55 @@ public static void checkSchemaCompatible(
throw new SchemaCompatibilityException(errorDetails);
}
}
+
+ public static Schema getRepairedSchema(Schema writerSchema, Schema readerSchema) {
+ if (AVRO_SCHEMA_REPAIR_CLASS == null) {
+ return writerSchema;
+ }
+ try {
+ Method repairMethod =
+ AVRO_SCHEMA_REPAIR_CLASS.getMethod("repairLogicalTypes", Schema.class, Schema.class);
+ return (Schema) repairMethod.invoke(null, writerSchema, readerSchema);
+ } catch (NoSuchMethodException | IllegalAccessException | InvocationTargetException e) {
+ return writerSchema;
+ }
+ }
+
+ /**
+ * Returns true if the given Avro schema contains any timestamp-millis logical type.
+ * Used to decide whether to enable logical timestamp field repair when reading log blocks.
+ */
+ public static boolean hasTimestampMillisField(Schema schema) {
+ if (schema == null || AVRO_SCHEMA_REPAIR_CLASS == null) {
+ return false;
+ }
+ try {
+ Method hasTimestampMillisFieldMethod =
+ AVRO_SCHEMA_REPAIR_CLASS.getMethod("hasTimestampMillisField", Schema.class);
+ return (Boolean) hasTimestampMillisFieldMethod.invoke(null, schema);
+ } catch (NoSuchMethodException | IllegalAccessException | InvocationTargetException e) {
+ return false;
+ }
+ }
+
+ /**
+ * Sets logical timestamp repair needed key in conf to true
+ */
+ public static void setLogicalTimestampRepairIfNotSet(Configuration conf, Supplier valueSupplier) {
+ if (conf.get(HoodieFileReader.ENABLE_LOGICAL_TIMESTAMP_REPAIR) == null) {
+ conf.set(HoodieFileReader.ENABLE_LOGICAL_TIMESTAMP_REPAIR, valueSupplier.get().toString());
+ }
+ }
+
+ /**
+ * Returns true if logical timestamp repair needed key is set to true or if it is not present in config
+ */
+ public static boolean isLogicalTimestampRepairNeeded(Configuration conf, Supplier defaultValueSupplier) {
+ String value = conf.get(HoodieFileReader.ENABLE_LOGICAL_TIMESTAMP_REPAIR);
+ if (StringUtils.isNullOrEmpty(value)) {
+ return defaultValueSupplier.get();
+ } else {
+ return Boolean.parseBoolean(value);
+ }
+ }
}
diff --git a/hudi-common/src/main/java/org/apache/hudi/avro/ConvertingGenericData.java b/hudi-common/src/main/java/org/apache/hudi/avro/ConvertingGenericData.java
index 9d36e214fb852..70a653726f4e8 100644
--- a/hudi-common/src/main/java/org/apache/hudi/avro/ConvertingGenericData.java
+++ b/hudi-common/src/main/java/org/apache/hudi/avro/ConvertingGenericData.java
@@ -25,6 +25,7 @@
import org.apache.avro.generic.GenericData;
import org.apache.avro.generic.GenericFixed;
+import java.lang.reflect.Constructor;
import java.util.Map;
/**
@@ -42,13 +43,12 @@ public class ConvertingGenericData extends GenericData {
private static final TimeConversions.TimeMicrosConversion TIME_MICROS_CONVERSION = new TimeConversions.TimeMicrosConversion();
private static final TimeConversions.TimestampMicrosConversion TIMESTAMP_MICROS_CONVERSION = new TimeConversions.TimestampMicrosConversion();
- // NOTE: Those are not supported in Avro 1.8.2
- // TODO re-enable upon upgrading to 1.10
- // private static final TimeConversions.TimestampMillisConversion TIMESTAMP_MILLIS_CONVERSION = new TimeConversions.TimestampMillisConversion();
- // private static final TimeConversions.TimeMillisConversion TIME_MILLIS_CONVERSION = new TimeConversions.TimeMillisConversion();
- // private static final TimeConversions.LocalTimestampMillisConversion LOCAL_TIMESTAMP_MILLIS_CONVERSION = new TimeConversions.LocalTimestampMillisConversion();
- // private static final TimeConversions.LocalTimestampMicrosConversion LOCAL_TIMESTAMP_MICROS_CONVERSION = new TimeConversions.LocalTimestampMicrosConversion();
-
+ // NOTE: Those are not supported in Avro 1.8.2 (used by Spark 2)
+ // Use reflection to conditionally initialize them only if available
+ private static final Object TIMESTAMP_MILLIS_CONVERSION = createConversionIfAvailable("org.apache.avro.data.TimeConversions$TimestampMillisConversion");
+ private static final Object TIME_MILLIS_CONVERSION = createConversionIfAvailable("org.apache.avro.data.TimeConversions$TimeMillisConversion");
+ private static final Object LOCAL_TIMESTAMP_MILLIS_CONVERSION = createConversionIfAvailable("org.apache.avro.data.TimeConversions$LocalTimestampMillisConversion");
+ private static final Object LOCAL_TIMESTAMP_MICROS_CONVERSION = createConversionIfAvailable("org.apache.avro.data.TimeConversions$LocalTimestampMicrosConversion");
public static final GenericData INSTANCE = new ConvertingGenericData();
private ConvertingGenericData() {
@@ -57,12 +57,20 @@ private ConvertingGenericData() {
addLogicalTypeConversion(DATE_CONVERSION);
addLogicalTypeConversion(TIME_MICROS_CONVERSION);
addLogicalTypeConversion(TIMESTAMP_MICROS_CONVERSION);
- // NOTE: Those are not supported in Avro 1.8.2
- // TODO re-enable upon upgrading to 1.10
- // addLogicalTypeConversion(TIME_MILLIS_CONVERSION);
- // addLogicalTypeConversion(TIMESTAMP_MILLIS_CONVERSION);
- // addLogicalTypeConversion(LOCAL_TIMESTAMP_MILLIS_CONVERSION);
- // addLogicalTypeConversion(LOCAL_TIMESTAMP_MICROS_CONVERSION);
+ // NOTE: Those are not supported in Avro 1.8.2 (used by Spark 2)
+ // Only add conversions if they're available
+ if (TIME_MILLIS_CONVERSION != null) {
+ addLogicalTypeConversionReflectively(TIME_MILLIS_CONVERSION);
+ }
+ if (TIMESTAMP_MILLIS_CONVERSION != null) {
+ addLogicalTypeConversionReflectively(TIMESTAMP_MILLIS_CONVERSION);
+ }
+ if (LOCAL_TIMESTAMP_MILLIS_CONVERSION != null) {
+ addLogicalTypeConversionReflectively(LOCAL_TIMESTAMP_MILLIS_CONVERSION);
+ }
+ if (LOCAL_TIMESTAMP_MICROS_CONVERSION != null) {
+ addLogicalTypeConversionReflectively(LOCAL_TIMESTAMP_MICROS_CONVERSION);
+ }
}
@Override
@@ -125,9 +133,31 @@ public boolean validate(Schema schema, Object datum) {
return isInteger(datum)
|| DATE_CONVERSION.getConvertedType().isInstance(datum);
case LONG:
- return isLong(datum)
- || TIME_MICROS_CONVERSION.getConvertedType().isInstance(datum)
- || TIMESTAMP_MICROS_CONVERSION.getConvertedType().isInstance(datum);
+ if (isLong(datum)) {
+ return true;
+ }
+ if (TIME_MICROS_CONVERSION.getConvertedType().isInstance(datum)
+ || TIMESTAMP_MICROS_CONVERSION.getConvertedType().isInstance(datum)) {
+ return true;
+ }
+ // Check optional conversions that may not be available in Avro 1.8.2
+ Class> convertedType;
+ if (TIMESTAMP_MILLIS_CONVERSION != null
+ && (convertedType = getConvertedType(TIMESTAMP_MILLIS_CONVERSION)) != null
+ && convertedType.isInstance(datum)) {
+ return true;
+ }
+ if (LOCAL_TIMESTAMP_MICROS_CONVERSION != null
+ && (convertedType = getConvertedType(LOCAL_TIMESTAMP_MICROS_CONVERSION)) != null
+ && convertedType.isInstance(datum)) {
+ return true;
+ }
+ if (LOCAL_TIMESTAMP_MILLIS_CONVERSION != null
+ && (convertedType = getConvertedType(LOCAL_TIMESTAMP_MILLIS_CONVERSION)) != null
+ && convertedType.isInstance(datum)) {
+ return true;
+ }
+ return false;
case FLOAT:
return isFloat(datum);
case DOUBLE:
@@ -140,5 +170,43 @@ public boolean validate(Schema schema, Object datum) {
return false;
}
}
+
+ /**
+ * Creates a conversion instance using reflection if the class is available.
+ * Returns null if the class doesn't exist (e.g., in Avro 1.8.2).
+ */
+ private static Object createConversionIfAvailable(String className) {
+ try {
+ Class> clazz = Class.forName(className);
+ Constructor> constructor = clazz.getConstructor();
+ return constructor.newInstance();
+ } catch (ClassNotFoundException | NoSuchMethodException | InstantiationException
+ | IllegalAccessException | java.lang.reflect.InvocationTargetException e) {
+ // Class doesn't exist or can't be instantiated (e.g., Avro 1.8.2)
+ return null;
+ }
+ }
+
+ /**
+ * Gets the converted type from a conversion object using reflection.
+ */
+ private static Class> getConvertedType(Object conversion) {
+ try {
+ return (Class>) conversion.getClass().getMethod("getConvertedType").invoke(conversion);
+ } catch (Exception e) {
+ // Should not happen if conversion is valid, but handle gracefully
+ return null;
+ }
+ }
+
+ /**
+ * Adds a logical type conversion using unchecked cast to avoid compile-time dependency
+ * on classes that may not exist in older Avro versions.
+ */
+ private void addLogicalTypeConversionReflectively(Object conversion) {
+ // Cast to Conversion> since we know it's a Conversion if not null
+ // This avoids compile-time dependency on specific Conversion subclasses
+ addLogicalTypeConversion((org.apache.avro.Conversion>) conversion);
+ }
}
diff --git a/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java b/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java
index d04e986487b5e..7221ed9f00b2c 100644
--- a/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java
+++ b/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java
@@ -33,11 +33,13 @@
import org.apache.hudi.common.model.HoodieAvroRecord;
import org.apache.hudi.common.model.HoodieOperation;
import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.DateTimeUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.SpillableMapUtils;
import org.apache.hudi.common.util.StringUtils;
import org.apache.hudi.common.util.ValidationUtils;
import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.exception.HoodieAvroSchemaException;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.exception.SchemaCompatibilityException;
@@ -47,10 +49,12 @@
import org.apache.avro.Conversions;
import org.apache.avro.Conversions.DecimalConversion;
import org.apache.avro.JsonProperties;
+import org.apache.avro.LogicalType;
import org.apache.avro.LogicalTypes;
import org.apache.avro.LogicalTypes.Decimal;
import org.apache.avro.Schema;
import org.apache.avro.Schema.Field;
+import org.apache.avro.Schema.Field.Order;
import org.apache.avro.generic.GenericData;
import org.apache.avro.generic.GenericData.Record;
import org.apache.avro.generic.GenericDatumReader;
@@ -104,10 +108,10 @@
import static org.apache.avro.Schema.Type.UNION;
import static org.apache.hudi.avro.AvroSchemaUtils.createNullableSchema;
import static org.apache.hudi.avro.AvroSchemaUtils.isNullable;
-import static org.apache.hudi.avro.AvroSchemaUtils.resolveNullableSchema;
import static org.apache.hudi.avro.AvroSchemaUtils.resolveUnionSchema;
import static org.apache.hudi.common.util.DateTimeUtils.instantToMicros;
import static org.apache.hudi.common.util.DateTimeUtils.microsToInstant;
+import static org.apache.hudi.avro.AvroSchemaUtils.getNonNullTypeFromUnion;
import static org.apache.hudi.common.util.ValidationUtils.checkState;
import static org.apache.hudi.metadata.HoodieTableMetadataUtil.tryUpcastDecimal;
@@ -157,6 +161,19 @@ public class HoodieAvroUtils {
private static final Pattern INVALID_AVRO_FIRST_CHAR_IN_NAMES_PATTERN = Pattern.compile("[^A-Za-z_]");
private static final String MASK_FOR_INVALID_CHARS_IN_NAMES = "__";
+ // Cached Avro logical type classes (null if not available, e.g., Avro 1.8.2).
+ // Loaded once at class init to avoid repeated Class.forName() and reflection cost per record.
+ private static final Class> LOCAL_TIMESTAMP_MILLIS_CLASS = loadClassSafe("org.apache.avro.LogicalTypes$LocalTimestampMillis");
+ private static final Class> LOCAL_TIMESTAMP_MICROS_CLASS = loadClassSafe("org.apache.avro.LogicalTypes$LocalTimestampMicros");
+
+ private static Class> loadClassSafe(String name) {
+ try {
+ return Class.forName(name);
+ } catch (ClassNotFoundException e) {
+ return null;
+ }
+ }
+
// All metadata fields are optional strings.
public static final Schema METADATA_FIELD_SCHEMA = createNullableSchema(Schema.Type.STRING);
@@ -653,7 +670,7 @@ public static Schema getNestedFieldSchemaFromRecord(GenericRecord record, String
Object val = valueNode.get(part);
if (i == parts.length - 1) {
- return resolveNullableSchema(valueNode.getSchema().getField(part).schema());
+ return getNonNullTypeFromUnion(valueNode.getSchema().getField(part).schema());
} else {
if (!(val instanceof GenericRecord)) {
throw new HoodieException("Cannot find a record at part value :" + part);
@@ -674,13 +691,20 @@ public static Schema getNestedFieldSchemaFromRecord(GenericRecord record, String
*/
public static Schema getNestedFieldSchemaFromWriteSchema(Schema writeSchema, String fieldName) {
String[] parts = fieldName.split("\\.");
+ Schema currentSchema = writeSchema;
int i = 0;
for (; i < parts.length; i++) {
String part = parts[i];
- Schema schema = writeSchema.getField(part).schema();
+ try {
+ // Resolve nullable/union schema to the actual schema
+ currentSchema = getNonNullTypeFromUnion(currentSchema.getField(part).schema());
- if (i == parts.length - 1) {
- return resolveNullableSchema(schema);
+ if (i == parts.length - 1) {
+ // Return the schema for the final part
+ return getNonNullTypeFromUnion(currentSchema);
+ }
+ } catch (Exception e) {
+ throw new HoodieException("Failed to get schema. Not a valid field name: " + fieldName);
}
}
throw new HoodieException("Failed to get schema. Not a valid field name: " + fieldName);
@@ -718,7 +742,7 @@ public static Object convertValueForSpecificDataTypes(Schema fieldSchema,
return null;
}
- return convertValueForAvroLogicalTypes(resolveNullableSchema(fieldSchema), fieldValue, consistentLogicalTimestampEnabled);
+ return convertValueForAvroLogicalTypes(getNonNullTypeFromUnion(fieldSchema), fieldValue, consistentLogicalTimestampEnabled);
}
/**
@@ -968,12 +992,35 @@ private static Object rewritePrimaryType(Object oldValue, Schema oldSchema, Sche
case NULL:
case BOOLEAN:
case INT:
- case LONG:
case FLOAT:
case DOUBLE:
case BYTES:
case STRING:
return oldValue;
+ case LONG:
+ if (oldSchema.getLogicalType() != newSchema.getLogicalType()) {
+ if (oldSchema.getLogicalType() == null || newSchema.getLogicalType() == null) {
+ return oldValue;
+ } else if (oldSchema.getLogicalType() instanceof LogicalTypes.TimestampMillis) {
+ if (newSchema.getLogicalType() instanceof LogicalTypes.TimestampMicros) {
+ return DateTimeUtils.millisToMicros((Long) oldValue);
+ }
+ } else if (oldSchema.getLogicalType() instanceof LogicalTypes.TimestampMicros) {
+ if (newSchema.getLogicalType() instanceof LogicalTypes.TimestampMillis) {
+ return DateTimeUtils.microsToMillis((Long) oldValue);
+ }
+ } else if (isLocalTimestampMillis(oldSchema.getLogicalType())) {
+ if (isLocalTimestampMicros(newSchema.getLogicalType())) {
+ return DateTimeUtils.millisToMicros((Long) oldValue);
+ }
+ } else if (isLocalTimestampMicros(oldSchema.getLogicalType())) {
+ if (isLocalTimestampMillis(newSchema.getLogicalType())) {
+ return DateTimeUtils.microsToMillis((Long) oldValue);
+ }
+ }
+ throw new HoodieAvroSchemaException("Long type logical change from " + oldSchema.getLogicalType() + " to " + newSchema.getLogicalType() + " is not supported");
+ }
+ return oldValue;
case FIXED:
if (oldSchema.getFixedSize() != newSchema.getFixedSize()) {
// Check whether this is a [[Decimal]]'s precision change
@@ -1271,6 +1318,10 @@ public static boolean gteqAvro1_10() {
return VersionUtil.compareVersions(AVRO_VERSION, "1.10") >= 0;
}
+ public static boolean gteqAvro1_12() {
+ return VersionUtil.compareVersions(AVRO_VERSION, "1.12") >= 0;
+ }
+
/**
* Wraps a value into Avro type wrapper.
*
@@ -1364,4 +1415,95 @@ public static Comparable> unwrapAvroValueWrapper(Object avroValueWrapper) {
}
}
+ /**
+ * Checks if a logical type is an instance of LocalTimestampMillis using reflection.
+ * Returns false if the class doesn't exist (e.g., in Avro 1.8.2).
+ */
+ public static boolean isLocalTimestampMillis(LogicalType logicalType) {
+ if (logicalType == null || LOCAL_TIMESTAMP_MILLIS_CLASS == null) {
+ return false;
+ }
+ return LOCAL_TIMESTAMP_MILLIS_CLASS.isInstance(logicalType);
+ }
+
+ /**
+ * Checks if a logical type is an instance of LocalTimestampMicros using reflection.
+ * Returns false if the class doesn't exist (e.g., in Avro 1.8.2).
+ */
+ public static boolean isLocalTimestampMicros(LogicalType logicalType) {
+ if (logicalType == null || LOCAL_TIMESTAMP_MICROS_CLASS == null) {
+ return false;
+ }
+ return LOCAL_TIMESTAMP_MICROS_CLASS.isInstance(logicalType);
+ }
+
+ private static Object convertDefaultValueForAvroCompatibility(Object defaultValue) {
+ if (gteqAvro1_12() && defaultValue instanceof byte[]) {
+ // For Avro 1.12.0 compatibility, we need to convert the default value in byte array
+ // to String so that correct JsonNode is used for the default value for validation,
+ // instead of directly relying on Avro's JacksonUtils.toJsonNode which is called
+ // by `Schema.Field` constructor
+ // The logic of getting the String value is copied from JacksonUtils.toJsonNode in Avro 1.11.4
+ return new String((byte[]) defaultValue, StandardCharsets.ISO_8859_1);
+ }
+ return defaultValue;
+ }
+
+ /**
+ * Creates a new Avro Schema.Field from an existing field, with special handling for
+ * default values to ensure compatibility with Avro 1.12.0 and later versions.
+ *
+ * @param field the original Schema.Field to create a new field from
+ * @return a new Schema.Field with the same properties but properly formatted default value
+ */
+ public static Schema.Field createNewSchemaField(Schema.Field field) {
+ return createNewSchemaField(field.name(), field.schema(), field.doc(), field.defaultVal());
+ }
+
+ /**
+ * Creates a new Avro Schema.Field with special handling for default values to ensure
+ * compatibility with Avro 1.12.0 and later versions.
+ *
+ *
In Avro 1.12.0+, the validation of default values for bytes fields is stricter.
+ * When the default value is a byte array, it needs to be converted to a String using
+ * ISO-8859-1 encoding so that the correct JsonNode type (TextNode) is used for validation,
+ * rather than BinaryNode which would fail validation. Changes in Avro 1.12.0 that
+ * lead to this behavior: [AVRO-3876] https://github.com/apache/avro/pull/2529
+ *
+ *
This conversion ensures that schemas with bytes fields having default values
+ * can be properly constructed without AvroTypeException in Avro 1.12.0+.
+ *
+ * @param name the name of the field
+ * @param schema the schema of the field
+ * @param doc the documentation for the field (can be null)
+ * @param defaultValue the default value for the field (can be null)
+ * @return a new Schema.Field with properly formatted default value for Avro 1.12.0+ compatibility
+ */
+ public static Schema.Field createNewSchemaField(String name, Schema schema, String doc, Object defaultValue) {
+ return new Schema.Field(name, schema, doc, convertDefaultValueForAvroCompatibility(defaultValue));
+ }
+
+ /**
+ * Creates a new Avro Schema.Field with special handling for default values to ensure
+ * compatibility with Avro 1.12.0 and later versions.
+ *
+ *
In Avro 1.12.0+, the validation of default values for bytes fields is stricter.
+ * When the default value is a byte array, it needs to be converted to a String using
+ * ISO-8859-1 encoding so that the correct JsonNode type (TextNode) is used for validation,
+ * rather than BinaryNode which would fail validation. Changes in Avro 1.12.0 that
+ * lead to this behavior: [AVRO-3876] https://github.com/apache/avro/pull/2529
+ *
+ *
This conversion ensures that schemas with bytes fields having default values
+ * can be properly constructed without AvroTypeException in Avro 1.12.0+.
+ *
+ * @param name the name of the field
+ * @param schema the schema of the field
+ * @param doc the documentation for the field (can be null)
+ * @param defaultValue the default value for the field (can be null)
+ * @param order the sort order for this field (can be null, defaults to ascending)
+ * @return a new Schema.Field with properly formatted default value for Avro 1.12.0+ compatibility
+ */
+ public static Schema.Field createNewSchemaField(String name, Schema schema, String doc, Object defaultValue, Order order) {
+ return new Schema.Field(name, schema, doc, convertDefaultValueForAvroCompatibility(defaultValue), order);
+ }
}
diff --git a/hudi-common/src/main/java/org/apache/hudi/avro/MercifulJsonConverter.java b/hudi-common/src/main/java/org/apache/hudi/avro/MercifulJsonConverter.java
index cdf0f15d80deb..cce3016aed859 100644
--- a/hudi-common/src/main/java/org/apache/hudi/avro/MercifulJsonConverter.java
+++ b/hudi-common/src/main/java/org/apache/hudi/avro/MercifulJsonConverter.java
@@ -299,12 +299,27 @@ private static JsonToAvroFieldProcessor generateFixedTypeHandler() {
return new JsonToAvroFieldProcessor() {
@Override
public Pair convert(Object value, String name, Schema schema, boolean shouldSanitize, String invalidCharMask) {
+ byte[] src;
// The ObjectMapper use List to represent FixedType
// eg: "decimal_val": [0, 0, 14, -63, -52] will convert to ArrayList
- List converval = (List) value;
- byte[] src = new byte[converval.size()];
- for (int i = 0; i < converval.size(); i++) {
- src[i] = converval.get(i).byteValue();
+ if (value instanceof List) {
+ List converval = (List) value;
+ src = new byte[converval.size()];
+ for (int i = 0; i < converval.size(); i++) {
+ src[i] = converval.get(i).byteValue();
+ }
+ } else if (value instanceof ByteBuffer) {
+ // Handle ByteBuffer when reading from existing records
+ ByteBuffer buffer = (ByteBuffer) value;
+ int start = buffer.position();
+ int length = buffer.limit() - start;
+ src = new byte[length];
+ buffer.get(src, 0, length);
+ buffer.position(start);
+ } else if (value instanceof byte[]) {
+ src = (byte[]) value;
+ } else {
+ return Pair.of(false, null);
}
byte[] dst = new byte[schema.getFixedSize()];
System.arraycopy(src, 0, dst, 0, Math.min(schema.getFixedSize(), src.length));
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieStorageConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieStorageConfig.java
index 2660b0b22c835..cf26afe226467 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieStorageConfig.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieStorageConfig.java
@@ -220,6 +220,15 @@ public class HoodieStorageConfig extends HoodieConfig {
+ "and it is loaded at runtime. This is only required when trying to "
+ "override the existing write context.");
+ public static final ConfigProperty HOODIE_PARQUET_SPARK_ROW_WRITE_SUPPORT_CLASS = ConfigProperty
+ .key("hoodie.parquet.spark.row.write.support.class")
+ .defaultValue("org.apache.hudi.io.storage.row.HoodieRowParquetWriteSupport")
+ .markAdvanced()
+ .sinceVersion("0.14.2")
+ .withDocumentation("Provided write support class should extend HoodieRowParquetWriteSupport class "
+ + "and it is loaded at runtime. This is only required when trying to "
+ + "override the existing write context when `hoodie.datasource.write.row.writer.enable=true`.");
+
/**
* @deprecated Use {@link #PARQUET_MAX_FILE_SIZE} and its methods instead
*/
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/config/TimestampKeyGeneratorConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/config/TimestampKeyGeneratorConfig.java
index 7098c076279b0..aa97b10c2c83c 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/config/TimestampKeyGeneratorConfig.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/config/TimestampKeyGeneratorConfig.java
@@ -42,7 +42,8 @@ public class TimestampKeyGeneratorConfig {
.withAlternatives(OLD_TIMESTAMP_KEYGEN_CONFIG_PREFIX + "timestamp.type")
.markAdvanced()
.withDocumentation("Timestamp type of the field, which should be one of the timestamp types "
- + "supported: `UNIX_TIMESTAMP`, `DATE_STRING`, `MIXED`, `EPOCHMILLISECONDS`, `SCALAR`.");
+ + "supported: `UNIX_TIMESTAMP`, `DATE_STRING`, `MIXED`, `EPOCHMILLISECONDS`,"
+ + " `EPOCHMICROSECONDS`, `SCALAR`.");
public static final ConfigProperty INPUT_TIME_UNIT = ConfigProperty
.key(TIMESTAMP_KEYGEN_CONFIG_PREFIX + "timestamp.scalar.time.unit")
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java
index f8a6b2d64afae..b0ac3bbda37a6 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java
@@ -47,6 +47,7 @@
import org.apache.hudi.exception.TableNotFoundException;
import org.apache.hudi.hadoop.CachingPath;
import org.apache.hudi.hadoop.SerializablePath;
+import org.apache.hudi.metadata.HoodieTableMetadata;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
@@ -302,6 +303,10 @@ public TimelineLayoutVersion getTimelineLayoutVersion() {
return timelineLayoutVersion;
}
+ public boolean isMetadataTable() {
+ return HoodieTableMetadata.isMetadataTable(getBasePath());
+ }
+
/**
* Get the FS implementation for this table.
*/
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/ParquetTableSchemaResolver.java b/hudi-common/src/main/java/org/apache/hudi/common/table/ParquetTableSchemaResolver.java
new file mode 100644
index 0000000000000..ea2bf2fe043a9
--- /dev/null
+++ b/hudi-common/src/main/java/org/apache/hudi/common/table/ParquetTableSchemaResolver.java
@@ -0,0 +1,64 @@
+/*
+ * 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.common.table;
+
+import org.apache.avro.Schema;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.parquet.schema.MessageType;
+
+import static org.apache.parquet.avro.HoodieAvroParquetSchemaConverter.getAvroSchemaConverter;
+
+public class ParquetTableSchemaResolver extends TableSchemaResolver {
+
+ public ParquetTableSchemaResolver(HoodieTableMetaClient metaClient) {
+ super(metaClient);
+ }
+
+ public static MessageType convertAvroSchemaToParquet(Schema schema, Configuration hadoopConf) {
+ return getAvroSchemaConverter(hadoopConf).convert(schema);
+ }
+
+ private Schema convertParquetSchemaToAvro(MessageType parquetSchema) {
+ return getAvroSchemaConverter(metaClient.getHadoopConf()).convert(parquetSchema);
+ }
+
+ private MessageType convertAvroSchemaToParquet(Schema schema) {
+ return getAvroSchemaConverter(metaClient.getHadoopConf()).convert(schema);
+ }
+
+ /**
+ * Gets full schema (user + metadata) for a hoodie table in Parquet format.
+ *
+ * @return Parquet schema for the table
+ */
+ public MessageType getTableParquetSchema() throws Exception {
+ return convertAvroSchemaToParquet(getTableAvroSchema(true));
+ }
+
+ /**
+ * Gets users data schema for a hoodie table in Parquet format.
+ *
+ * @return Parquet schema for the table
+ */
+ public MessageType getTableParquetSchema(boolean includeMetadataField) throws Exception {
+ return convertAvroSchemaToParquet(getTableAvroSchema(includeMetadataField));
+ }
+
+}
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/TableSchemaResolver.java b/hudi-common/src/main/java/org/apache/hudi/common/table/TableSchemaResolver.java
index 02b1ef352515b..e0295080be7f7 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/table/TableSchemaResolver.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/table/TableSchemaResolver.java
@@ -82,7 +82,7 @@ public class TableSchemaResolver {
private static final Logger LOG = LoggerFactory.getLogger(TableSchemaResolver.class);
- private final HoodieTableMetaClient metaClient;
+ protected final HoodieTableMetaClient metaClient;
/**
* Signals whether suite of the meta-fields should have additional field designating
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordReader.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordReader.java
index 3678efe786252..7466b31dc1b60 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordReader.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordReader.java
@@ -18,6 +18,7 @@
package org.apache.hudi.common.table.log;
+import org.apache.hudi.avro.AvroSchemaUtils;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.model.DeleteRecord;
import org.apache.hudi.common.model.HoodieLogFile;
@@ -40,7 +41,6 @@
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieIOException;
-import org.apache.hudi.hadoop.CachingPath;
import org.apache.hudi.internal.schema.InternalSchema;
import org.apache.hudi.internal.schema.action.InternalSchemaMerger;
import org.apache.hudi.internal.schema.convert.AvroInternalSchemaConverter;
@@ -111,6 +111,8 @@ public abstract class AbstractHoodieLogRecordReader {
private final TypedProperties payloadProps;
// Log File Paths
protected final List logFilePaths;
+ // Read block lazily - when true, reads block content on demand
+ private final boolean readBlocksLazily;
// Reverse reader - Not implemented yet (NA -> Why do we need ?)
// but present here for plumbing for future implementation
private final boolean reverseReader;
@@ -148,6 +150,8 @@ public abstract class AbstractHoodieLogRecordReader {
private final List validBlockInstants = new ArrayList<>();
// Use scanV2 method.
private final boolean enableOptimizedLogBlocksScan;
+ // Enable logical timestamp field repair for Avro log blocks (computed once from reader schema).
+ private final boolean enableLogicalTimestampFieldRepair;
protected AbstractHoodieLogRecordReader(FileSystem fs, String basePath, List logFilePaths,
Schema readerSchema, String latestInstantTime, boolean readBlocksLazily,
@@ -174,6 +178,7 @@ protected AbstractHoodieLogRecordReader(FileSystem fs, String basePath, List readerSchema != null && AvroSchemaUtils.hasTimestampMillisField(readerSchema));
if (keyFieldOverride.isPresent()) {
// NOTE: This branch specifically is leveraged handling Metadata Table
@@ -241,8 +248,11 @@ private void scanInternalV1(Option keySpecOpt) {
try {
// Iterate over the paths
logFormatReaderWrapper = new HoodieLogFormatReader(fs,
- logFilePaths.stream().map(logFile -> new HoodieLogFile(new CachingPath(logFile))).collect(Collectors.toList()),
- readerSchema, true, reverseReader, bufferSize, shouldLookupRecords(), recordKeyField, internalSchema);
+ logFilePaths.stream()
+ .map(filePath -> new HoodieLogFile(filePath))
+ .collect(Collectors.toList()),
+ readerSchema, readBlocksLazily, reverseReader, bufferSize, shouldLookupRecords(), recordKeyField, internalSchema,
+ enableLogicalTimestampFieldRepair);
Set scannedLogFiles = new HashSet<>();
while (logFormatReaderWrapper.hasNext()) {
@@ -554,8 +564,11 @@ private void scanInternalV2(Option keySpecOption, boolean skipProcessin
try {
// Iterate over the paths
logFormatReaderWrapper = new HoodieLogFormatReader(fs,
- logFilePaths.stream().map(logFile -> new HoodieLogFile(new CachingPath(logFile))).collect(Collectors.toList()),
- readerSchema, true, reverseReader, bufferSize, shouldLookupRecords(), recordKeyField, internalSchema);
+ logFilePaths.stream()
+ .map(logFile -> new HoodieLogFile(logFile))
+ .collect(Collectors.toList()),
+ readerSchema, readBlocksLazily, reverseReader, bufferSize, shouldLookupRecords(), recordKeyField, internalSchema,
+ enableLogicalTimestampFieldRepair);
/**
* Scanning log blocks and placing the compacted blocks at the right place require two traversals.
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFileReader.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFileReader.java
index cf21ef5f42c81..15cd90f844031 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFileReader.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFileReader.java
@@ -87,6 +87,7 @@ public class HoodieLogFileReader implements HoodieLogFormat.Reader {
private long lastReverseLogFilePosition;
private final boolean reverseReader;
private final boolean enableRecordLookups;
+ private final boolean enableLogicalTimestampFieldRepair;
private boolean closed = false;
private FSDataInputStream inputStream;
@@ -98,18 +99,16 @@ public HoodieLogFileReader(FileSystem fs, HoodieLogFile logFile, Schema readerSc
public HoodieLogFileReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema, int bufferSize,
boolean readBlockLazily, boolean reverseReader) throws IOException {
this(fs, logFile, readerSchema, bufferSize, readBlockLazily, reverseReader, false,
- HoodieRecord.RECORD_KEY_METADATA_FIELD);
- }
-
- public HoodieLogFileReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema, int bufferSize,
- boolean readBlockLazily, boolean reverseReader, boolean enableRecordLookups,
- String keyField) throws IOException {
- this(fs, logFile, readerSchema, bufferSize, readBlockLazily, reverseReader, enableRecordLookups, keyField, InternalSchema.getEmptyInternalSchema());
+ HoodieRecord.RECORD_KEY_METADATA_FIELD, InternalSchema.getEmptyInternalSchema(), false);
}
+ /**
+ * Constructor with full options for use by HoodieLogFormatReader (FileSystem-based, no storage abstraction).
+ */
public HoodieLogFileReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema, int bufferSize,
boolean readBlockLazily, boolean reverseReader, boolean enableRecordLookups,
- String keyField, InternalSchema internalSchema) throws IOException {
+ String keyField, InternalSchema internalSchema,
+ boolean enableLogicalTimestampFieldRepair) throws IOException {
this.fs = fs;
this.hadoopConf = fs.getConf();
// NOTE: We repackage {@code HoodieLogFile} here to make sure that the provided path
@@ -125,6 +124,7 @@ public HoodieLogFileReader(FileSystem fs, HoodieLogFile logFile, Schema readerSc
this.enableRecordLookups = enableRecordLookups;
this.keyField = keyField;
this.internalSchema = internalSchema == null ? InternalSchema.getEmptyInternalSchema() : internalSchema;
+ this.enableLogicalTimestampFieldRepair = enableLogicalTimestampFieldRepair;
if (this.reverseReader) {
this.reverseLogFilePosition = this.lastReverseLogFilePosition = this.logFile.getFileSize();
}
@@ -200,8 +200,8 @@ private HoodieLogBlock readBlock() throws IOException {
if (nextBlockVersion.getVersion() == HoodieLogFormatVersion.DEFAULT_VERSION) {
return HoodieAvroDataBlock.getBlock(content.get(), readerSchema, internalSchema);
} else {
- return new HoodieAvroDataBlock(() -> getFSDataInputStream(fs, this.logFile, bufferSize), content, readBlockLazily, logBlockContentLoc,
- getTargetReaderSchemaForBlock(), header, footer, keyField);
+ return new HoodieAvroDataBlock(() -> getFSDataInputStream(fs, this.logFile, bufferSize), content, true, logBlockContentLoc,
+ getTargetReaderSchemaForBlock(), header, footer, keyField, enableLogicalTimestampFieldRepair);
}
case HFILE_DATA_BLOCK:
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFormatReader.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFormatReader.java
index 955f5485ed459..ed27e503fb891 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFormatReader.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFormatReader.java
@@ -45,12 +45,14 @@ public class HoodieLogFormatReader implements HoodieLogFormat.Reader {
private final String recordKeyField;
private final boolean enableInlineReading;
private final int bufferSize;
+ private final boolean enableLogicalTimestampFieldRepair;
private static final Logger LOG = LoggerFactory.getLogger(HoodieLogFormatReader.class);
HoodieLogFormatReader(FileSystem fs, List logFiles, Schema readerSchema, boolean readBlocksLazily,
boolean reverseLogReader, int bufferSize, boolean enableRecordLookups,
- String recordKeyField, InternalSchema internalSchema) throws IOException {
+ String recordKeyField, InternalSchema internalSchema,
+ boolean enableLogicalTimestampFieldRepair) throws IOException {
this.logFiles = logFiles;
this.fs = fs;
this.readerSchema = readerSchema;
@@ -59,10 +61,11 @@ public class HoodieLogFormatReader implements HoodieLogFormat.Reader {
this.recordKeyField = recordKeyField;
this.enableInlineReading = enableRecordLookups;
this.internalSchema = internalSchema == null ? InternalSchema.getEmptyInternalSchema() : internalSchema;
- if (logFiles.size() > 0) {
+ this.enableLogicalTimestampFieldRepair = enableLogicalTimestampFieldRepair;
+ if (!logFiles.isEmpty()) {
HoodieLogFile nextLogFile = logFiles.remove(0);
this.currentReader = new HoodieLogFileReader(fs, nextLogFile, readerSchema, bufferSize, readBlocksLazily, false,
- enableRecordLookups, recordKeyField, internalSchema);
+ enableRecordLookups, recordKeyField, internalSchema, enableLogicalTimestampFieldRepair);
}
}
@@ -87,7 +90,7 @@ public boolean hasNext() {
HoodieLogFile nextLogFile = logFiles.remove(0);
this.currentReader.close();
this.currentReader = new HoodieLogFileReader(fs, nextLogFile, readerSchema, bufferSize, readBlocksLazily, false,
- enableInlineReading, recordKeyField, internalSchema);
+ enableInlineReading, recordKeyField, internalSchema, enableLogicalTimestampFieldRepair);
} catch (IOException io) {
throw new HoodieIOException("unable to initialize read with log file ", io);
}
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieAvroDataBlock.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieAvroDataBlock.java
index bdcd0ac690fd2..d7ec98758fbbc 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieAvroDataBlock.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieAvroDataBlock.java
@@ -62,6 +62,7 @@
import java.util.zip.DeflaterOutputStream;
import java.util.zip.InflaterInputStream;
+import static org.apache.hudi.avro.AvroSchemaUtils.getRepairedSchema;
import static org.apache.hudi.avro.HoodieAvroUtils.recordNeedsRewriteForExtendedAvroTypePromotion;
import static org.apache.hudi.common.util.ValidationUtils.checkArgument;
import static org.apache.hudi.common.util.ValidationUtils.checkState;
@@ -72,6 +73,7 @@
public class HoodieAvroDataBlock extends HoodieDataBlock {
private final ThreadLocal encoderCache = new ThreadLocal<>();
+ private final boolean enableLogicalTimestampFieldRepair;
public HoodieAvroDataBlock(Supplier inputStreamSupplier,
Option content,
@@ -80,8 +82,10 @@ public HoodieAvroDataBlock(Supplier inputStreamSupplier,
Option readerSchema,
Map header,
Map footer,
- String keyField) {
+ String keyField,
+ boolean enableLogicalTimestampFieldRepair) {
super(content, inputStreamSupplier, readBlockLazily, Option.of(logBlockContentLocation), readerSchema, header, footer, keyField, false);
+ this.enableLogicalTimestampFieldRepair = enableLogicalTimestampFieldRepair;
}
public HoodieAvroDataBlock(@Nonnull List records,
@@ -89,6 +93,7 @@ public HoodieAvroDataBlock(@Nonnull List records,
@Nonnull String keyField
) {
super(records, header, new HashMap<>(), keyField);
+ this.enableLogicalTimestampFieldRepair = false;
}
@Override
@@ -142,7 +147,7 @@ protected ClosableIterator> deserializeRecords(byte[] conten
checkState(this.readerSchema != null, "Reader's schema has to be non-null");
checkArgument(type != HoodieRecordType.SPARK, "Not support read avro to spark record");
// TODO AvroSparkReader need
- RecordIterator iterator = RecordIterator.getInstance(this, content);
+ RecordIterator iterator = RecordIterator.getInstance(this, content, enableLogicalTimestampFieldRepair);
return new CloseableMappingIterator<>(iterator, data -> (HoodieRecord) new HoodieAvroIndexedRecord(data));
}
@@ -155,7 +160,7 @@ private static class RecordIterator implements ClosableIterator {
private int totalRecords = 0;
private int readRecords = 0;
- private RecordIterator(Schema readerSchema, Schema writerSchema, byte[] content) throws IOException {
+ private RecordIterator(Schema readerSchema, Schema writerSchema, byte[] content, boolean enableLogicalTimestampFieldRepair) throws IOException {
this.content = content;
this.dis = new SizeAwareDataInputStream(new DataInputStream(new ByteArrayInputStream(this.content)));
@@ -166,16 +171,21 @@ private RecordIterator(Schema readerSchema, Schema writerSchema, byte[] content)
this.totalRecords = this.dis.readInt();
}
- if (recordNeedsRewriteForExtendedAvroTypePromotion(writerSchema, readerSchema)) {
- this.reader = new GenericDatumReader<>(writerSchema, writerSchema);
+ // writer schema could refer to table schema.
+ // avoid this for MDT for sure.
+ // and for tables having no logical ts column.
+ Schema repairedWriterSchema = enableLogicalTimestampFieldRepair
+ ? getRepairedSchema(writerSchema, readerSchema) : writerSchema;
+ if (recordNeedsRewriteForExtendedAvroTypePromotion(repairedWriterSchema, readerSchema)) {
+ this.reader = new GenericDatumReader<>(repairedWriterSchema, repairedWriterSchema);
this.promotedSchema = Option.of(readerSchema);
} else {
- this.reader = new GenericDatumReader<>(writerSchema, readerSchema);
+ this.reader = new GenericDatumReader<>(repairedWriterSchema, readerSchema);
}
}
- public static RecordIterator getInstance(HoodieAvroDataBlock dataBlock, byte[] content) throws IOException {
- return new RecordIterator(dataBlock.readerSchema, dataBlock.getSchemaFromHeader(), content);
+ public static RecordIterator getInstance(HoodieAvroDataBlock dataBlock, byte[] content, boolean enableLogicalTimestampFieldRepair) throws IOException {
+ return new RecordIterator(dataBlock.readerSchema, dataBlock.getSchemaFromHeader(), content, enableLogicalTimestampFieldRepair);
}
@Override
@@ -228,6 +238,7 @@ public IndexedRecord next() {
@Deprecated
public HoodieAvroDataBlock(List records, Schema schema) {
super(records, Collections.singletonMap(HeaderMetadataType.SCHEMA, schema.toString()), new HashMap<>(), HoodieRecord.RECORD_KEY_METADATA_FIELD);
+ this.enableLogicalTimestampFieldRepair = false;
}
public static HoodieAvroDataBlock getBlock(byte[] content, Schema readerSchema) throws IOException {
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieCDCDataBlock.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieCDCDataBlock.java
index 8f2cd8c644786..76527fc2e874b 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieCDCDataBlock.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieCDCDataBlock.java
@@ -43,7 +43,7 @@ public HoodieCDCDataBlock(
Map header,
String keyField) {
super(inputStreamSupplier, content, readBlockLazily, logBlockContentLocation,
- Option.of(readerSchema), header, new HashMap<>(), keyField);
+ Option.of(readerSchema), header, new HashMap<>(), keyField, false);
}
public HoodieCDCDataBlock(List records,
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java
index d051b33b74b5c..191ecebd9c2c9 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java
@@ -635,6 +635,19 @@ public final List getPartitionPaths() {
}
}
+ public final List getPartitionNames() {
+ try {
+ readLock.lock();
+ return fetchAllStoredFileGroups()
+ .filter(fg -> !isFileGroupReplaced(fg))
+ .map(HoodieFileGroup::getPartitionPath)
+ .distinct()
+ .collect(Collectors.toList());
+ } finally {
+ readLock.unlock();
+ }
+ }
+
@Override
public final Stream> getPendingLogCompactionOperations() {
try {
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/DateTimeUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/DateTimeUtils.java
index 9dde7727806c2..aa26db8272eca 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/util/DateTimeUtils.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/util/DateTimeUtils.java
@@ -71,6 +71,45 @@ public static long instantToMicros(Instant instant) {
}
}
+ /**
+ * This is based off instantToMicros above.
+ * */
+ public static long instantToNanos(Instant instant) {
+ long seconds = instant.getEpochSecond();
+ int nanos = instant.getNano();
+
+ if (seconds < 0 && nanos > 0) {
+ // Shift seconds by +1, then subtract a full second in nanos
+ long totalNanos = Math.multiplyExact(seconds + 1, 1_000_000_000L);
+ long adjustment = nanos - 1_000_000_000L;
+ return Math.addExact(totalNanos, adjustment);
+ } else {
+ long totalNanos = Math.multiplyExact(seconds, 1_000_000_000L);
+ return Math.addExact(totalNanos, nanos);
+ }
+ }
+
+ public static final long MICROS_PER_MILLIS = 1000L;
+
+ /**
+ * Converts the timestamp to milliseconds since epoch. In Spark timestamp values have microseconds
+ * precision, so this conversion is lossy.
+ */
+ public static Long microsToMillis(Long micros) {
+ // When the timestamp is negative i.e before 1970, we need to adjust the milliseconds 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).
+ return Math.floorDiv(micros, MICROS_PER_MILLIS);
+ }
+
+ /**
+ * Converts milliseconds since the epoch to microseconds.
+ */
+ public static Long millisToMicros(Long millis) {
+ return Math.multiplyExact(millis, MICROS_PER_MILLIS);
+ }
+
+
/**
* Parse input String to a {@link java.time.Instant}.
*
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/ExternalFilePathUtil.java b/hudi-common/src/main/java/org/apache/hudi/common/util/ExternalFilePathUtil.java
index 223ae8abc42b9..50032af9feffb 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/util/ExternalFilePathUtil.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/util/ExternalFilePathUtil.java
@@ -43,4 +43,25 @@ public static String appendCommitTimeAndExternalFileMarker(String filePath, Stri
public static boolean isExternallyCreatedFile(String fileName) {
return fileName.endsWith(EXTERNAL_FILE_SUFFIX);
}
+
+ /**
+ * Removes the appended commit time and external file marker from the file path, returning the original file path.
+ *
+ * @param filePath The file path with commit time and external file marker appended
+ */
+ public static String removeCommitTimeAndExternalFileMarker(String filePath) {
+ if (!isExternallyCreatedFile(filePath)) {
+ return filePath;
+ }
+ // Remove the suffix
+ String pathWithoutSuffix = filePath.substring(0, filePath.length() - EXTERNAL_FILE_SUFFIX.length());
+ // Find the last underscore which separates the commit time
+ int lastUnderscoreIndex = pathWithoutSuffix.lastIndexOf('_');
+ if (lastUnderscoreIndex == -1) {
+ // No underscore found, return as is
+ return filePath;
+ }
+ // Return the path without the commit time and suffix
+ return pathWithoutSuffix.substring(0, lastUnderscoreIndex);
+ }
}
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/HoodieAvroParquetReaderIterator.java b/hudi-common/src/main/java/org/apache/hudi/common/util/HoodieAvroParquetReaderIterator.java
new file mode 100644
index 0000000000000..b4eedac7ce2a7
--- /dev/null
+++ b/hudi-common/src/main/java/org/apache/hudi/common/util/HoodieAvroParquetReaderIterator.java
@@ -0,0 +1,39 @@
+/*
+ * 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.common.util;
+
+import org.apache.hudi.avro.HoodieAvroUtils;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.parquet.hadoop.ParquetReader;
+
+public class HoodieAvroParquetReaderIterator extends ParquetReaderIterator {
+ private final Schema promotedSchema;
+ public HoodieAvroParquetReaderIterator(ParquetReader parquetReader, Schema promotedSchema) {
+ super(parquetReader);
+ this.promotedSchema = promotedSchema;
+ }
+
+ @Override
+ public IndexedRecord next() {
+ return HoodieAvroUtils.rewriteRecordWithNewSchema(super.next(), this.promotedSchema);
+ }
+}
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/JsonUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/JsonUtils.java
index b35f8a1c18ccb..820f564f1cb1f 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/util/JsonUtils.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/util/JsonUtils.java
@@ -44,6 +44,8 @@ public class JsonUtils {
MAPPER.setVisibility(PropertyAccessor.IS_GETTER, JsonAutoDetect.Visibility.NONE);
MAPPER.setVisibility(PropertyAccessor.SETTER, JsonAutoDetect.Visibility.NONE);
MAPPER.setVisibility(PropertyAccessor.CREATOR, JsonAutoDetect.Visibility.NONE);
+ // NOTE: Registering [[JavaTimeModule]] is required for Jackson >= 2.11 (Spark >= 3.3)
+ MAPPER.registerModule(new JavaTimeModule());
}
public static ObjectMapper getObjectMapper() {
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/ParquetUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/ParquetUtils.java
index de5572523c1eb..c15ea5fbc34ec 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/util/ParquetUtils.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/util/ParquetUtils.java
@@ -35,7 +35,6 @@
import org.apache.hadoop.fs.Path;
import org.apache.parquet.avro.AvroParquetReader;
import org.apache.parquet.avro.AvroReadSupport;
-import org.apache.parquet.avro.AvroSchemaConverter;
import org.apache.parquet.column.statistics.Statistics;
import org.apache.parquet.hadoop.ParquetFileReader;
import org.apache.parquet.hadoop.ParquetReader;
@@ -65,6 +64,8 @@
import java.util.stream.Collectors;
import java.util.stream.Stream;
+import static org.apache.parquet.avro.HoodieAvroParquetSchemaConverter.getAvroSchemaConverter;
+
/**
* Utility functions involving with parquet.
*/
@@ -228,7 +229,7 @@ public Map readFooter(Configuration configuration, boolean requi
@Override
public Schema readAvroSchema(Configuration conf, Path parquetFilePath) {
MessageType parquetSchema = readSchema(conf, parquetFilePath);
- return new AvroSchemaConverter(conf).convert(parquetSchema);
+ return getAvroSchemaConverter(conf).convert(parquetSchema);
}
@Override
@@ -311,10 +312,12 @@ public List> readRangeFromParquetMetadata(
Collectors.groupingBy(HoodieColumnRangeMetadata::getColumnName);
// Collect stats from all individual Parquet blocks
- Map>> columnToStatsListMap =
- (Map>>) metadata.getBlocks().stream().sequential()
- .flatMap(blockMetaData ->
- blockMetaData.getColumns().stream()
+ // NOTE: Explicit cast on inner stream helps Java with type inference
+ @SuppressWarnings("unchecked")
+ Stream> blockStream = metadata.getBlocks().stream().sequential()
+ .flatMap(blockMetaData -> {
+ Stream> columnStream =
+ (Stream>) (Stream>) blockMetaData.getColumns().stream()
.filter(f -> cols.contains(f.getPath().toDotString()))
.map(columnChunkMetaData -> {
Statistics stats = columnChunkMetaData.getStatistics();
@@ -334,9 +337,11 @@ public List> readRangeFromParquetMetadata(
columnChunkMetaData.getValueCount(),
columnChunkMetaData.getTotalSize(),
columnChunkMetaData.getTotalUncompressedSize());
- })
- )
- .collect(groupingByCollector);
+ });
+ return columnStream;
+ });
+
+ Map>> columnToStatsListMap = blockStream.collect(groupingByCollector);
// Combine those into file-level statistics
// NOTE: Inlining this var makes javac (1.8) upset (due to its inability to infer
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/ReflectionUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/ReflectionUtils.java
index 21d91a8a3344f..babe35c6e6028 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/util/ReflectionUtils.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/util/ReflectionUtils.java
@@ -53,6 +53,8 @@ public static Class> getClass(String clazzName) {
return Class.forName(c);
} catch (ClassNotFoundException e) {
throw new HoodieException("Unable to load class", e);
+ } catch (NoClassDefFoundError e) {
+ throw new HoodieException("Unable to load class due to missing dependency", e);
}
});
}
@@ -112,6 +114,15 @@ public static boolean hasConstructor(String clazz, Class>[] constructorArgType
LOG.warn(message, e);
}
return false;
+ } catch (HoodieException e) {
+ // Class cannot be loaded (e.g., ClassNotFoundException or NoClassDefFoundError)
+ String message = "Unable to load class " + clazz;
+ if (silenceWarning) {
+ LOG.debug(message, e);
+ } else {
+ LOG.warn(message, e);
+ }
+ return false;
}
}
diff --git a/hudi-common/src/main/java/org/apache/hudi/exception/HoodieAvroSchemaException.java b/hudi-common/src/main/java/org/apache/hudi/exception/HoodieAvroSchemaException.java
new file mode 100644
index 0000000000000..7e9822ba31170
--- /dev/null
+++ b/hudi-common/src/main/java/org/apache/hudi/exception/HoodieAvroSchemaException.java
@@ -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.exception;
+
+/**
+ * Thrown when we detect in Hudi code that a record schema
+ * violates Avro rules. This can happen even when using Spark
+ * because we use Avro schema internally
+ */
+public class HoodieAvroSchemaException extends SchemaCompatibilityException {
+ public HoodieAvroSchemaException(String message) {
+ super(message);
+ }
+
+ public HoodieAvroSchemaException(String message, Throwable cause) {
+ super(message, cause);
+ }
+}
diff --git a/hudi-common/src/main/java/org/apache/hudi/internal/schema/Type.java b/hudi-common/src/main/java/org/apache/hudi/internal/schema/Type.java
index bc8b89004d695..95c6504446b95 100644
--- a/hudi-common/src/main/java/org/apache/hudi/internal/schema/Type.java
+++ b/hudi-common/src/main/java/org/apache/hudi/internal/schema/Type.java
@@ -64,7 +64,12 @@ enum TypeID {
TIME(Long.class),
TIMESTAMP(Long.class),
DECIMAL(BigDecimal.class),
- UUID(UUID.class);
+ UUID(UUID.class),
+ TIME_MILLIS(Integer.class),
+ TIMESTAMP_MILLIS(Long.class),
+ LOCAL_TIMESTAMP_MILLIS(Long.class),
+ LOCAL_TIMESTAMP_MICROS(Long.class);
+
private final String name;
private final Class> classTag;
diff --git a/hudi-common/src/main/java/org/apache/hudi/internal/schema/Types.java b/hudi-common/src/main/java/org/apache/hudi/internal/schema/Types.java
index ed03a7349cb72..86e39959c5178 100644
--- a/hudi-common/src/main/java/org/apache/hudi/internal/schema/Types.java
+++ b/hudi-common/src/main/java/org/apache/hudi/internal/schema/Types.java
@@ -383,6 +383,78 @@ public int hashCode() {
}
}
+ public static class TimeMillisType extends PrimitiveType {
+ private static final TimeMillisType INSTANCE = new TimeMillisType();
+
+ public static TimeMillisType get() {
+ return INSTANCE;
+ }
+
+ @Override
+ public TypeID typeId() {
+ return TypeID.TIME_MILLIS;
+ }
+
+ @Override
+ public String toString() {
+ return "time-millis";
+ }
+ }
+
+ public static class TimestampMillisType extends PrimitiveType {
+ private static final TimestampMillisType INSTANCE = new TimestampMillisType();
+
+ public static TimestampMillisType get() {
+ return INSTANCE;
+ }
+
+ @Override
+ public TypeID typeId() {
+ return TypeID.TIMESTAMP_MILLIS;
+ }
+
+ @Override
+ public String toString() {
+ return "timestamp-millis";
+ }
+ }
+
+ public static class LocalTimestampMillisType extends PrimitiveType {
+ private static final LocalTimestampMillisType INSTANCE = new LocalTimestampMillisType();
+
+ public static LocalTimestampMillisType get() {
+ return INSTANCE;
+ }
+
+ @Override
+ public TypeID typeId() {
+ return TypeID.LOCAL_TIMESTAMP_MILLIS;
+ }
+
+ @Override
+ public String toString() {
+ return "local-timestamp-millis";
+ }
+ }
+
+ public static class LocalTimestampMicrosType extends PrimitiveType {
+ private static final LocalTimestampMicrosType INSTANCE = new LocalTimestampMicrosType();
+
+ public static LocalTimestampMicrosType get() {
+ return INSTANCE;
+ }
+
+ @Override
+ public TypeID typeId() {
+ return TypeID.LOCAL_TIMESTAMP_MICROS;
+ }
+
+ @Override
+ public String toString() {
+ return "local-timestamp-micros";
+ }
+ }
+
/**
* UUID primitive type.
*/
diff --git a/hudi-common/src/main/java/org/apache/hudi/internal/schema/convert/AvroInternalSchemaConverter.java b/hudi-common/src/main/java/org/apache/hudi/internal/schema/convert/AvroInternalSchemaConverter.java
index 786ac538271a2..aa2a64e11d770 100644
--- a/hudi-common/src/main/java/org/apache/hudi/internal/schema/convert/AvroInternalSchemaConverter.java
+++ b/hudi-common/src/main/java/org/apache/hudi/internal/schema/convert/AvroInternalSchemaConverter.java
@@ -18,6 +18,7 @@
package org.apache.hudi.internal.schema.convert;
+import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.internal.schema.HoodieSchemaException;
import org.apache.hudi.internal.schema.InternalSchema;
@@ -231,15 +232,18 @@ private static Type visitAvroPrimitiveToBuildInternalType(Schema primitive) {
} else if (logical instanceof LogicalTypes.Date) {
return Types.DateType.get();
- } else if (
- logical instanceof LogicalTypes.TimeMillis
- || logical instanceof LogicalTypes.TimeMicros) {
+ } else if (logical instanceof LogicalTypes.TimeMillis) {
+ return Types.TimeMillisType.get();
+ } else if (logical instanceof LogicalTypes.TimeMicros) {
return Types.TimeType.get();
-
- } else if (
- logical instanceof LogicalTypes.TimestampMillis
- || logical instanceof LogicalTypes.TimestampMicros) {
+ } else if (logical instanceof LogicalTypes.TimestampMillis) {
+ return Types.TimestampMillisType.get();
+ } else if (logical instanceof LogicalTypes.TimestampMicros) {
return Types.TimestampType.get();
+ } else if (HoodieAvroUtils.isLocalTimestampMillis(logical)) {
+ return Types.LocalTimestampMillisType.get();
+ } else if (HoodieAvroUtils.isLocalTimestampMicros(logical)) {
+ return Types.LocalTimestampMicrosType.get();
} else if (LogicalTypes.uuid().getName().equals(name)) {
return Types.UUIDType.get();
}
@@ -428,9 +432,21 @@ private static Schema visitInternalPrimitiveToBuildAvroPrimitiveType(Type.Primit
case TIME:
return LogicalTypes.timeMicros().addToSchema(Schema.create(Schema.Type.LONG));
+ case TIME_MILLIS:
+ return LogicalTypes.timeMillis().addToSchema(Schema.create(Schema.Type.INT));
+
case TIMESTAMP:
return LogicalTypes.timestampMicros().addToSchema(Schema.create(Schema.Type.LONG));
+ case TIMESTAMP_MILLIS:
+ return LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG));
+
+ case LOCAL_TIMESTAMP_MICROS:
+ return createLocalTimestampMicrosSchema();
+
+ case LOCAL_TIMESTAMP_MILLIS:
+ return createLocalTimestampMillisSchema();
+
case STRING:
return Schema.create(Schema.Type.STRING);
@@ -481,4 +497,34 @@ private static int computeMinBytesForPrecision(int precision) {
}
return numBytes;
}
+
+ /**
+ * Creates a LocalTimestampMicros schema using reflection.
+ * Returns null if the class doesn't exist (e.g., in Avro 1.8.2).
+ */
+ private static Schema createLocalTimestampMicrosSchema() {
+ try {
+ java.lang.reflect.Method method = LogicalTypes.class.getMethod("localTimestampMicros");
+ LogicalType logicalType = (LogicalType) method.invoke(null);
+ return logicalType.addToSchema(Schema.create(Schema.Type.LONG));
+ } catch (Exception e) {
+ // Method doesn't exist (e.g., Avro 1.8.2)
+ throw new UnsupportedOperationException("LocalTimestampMicros is not supported in this Avro version", e);
+ }
+ }
+
+ /**
+ * Creates a LocalTimestampMillis schema using reflection.
+ * Returns null if the class doesn't exist (e.g., in Avro 1.8.2).
+ */
+ private static Schema createLocalTimestampMillisSchema() {
+ try {
+ java.lang.reflect.Method method = LogicalTypes.class.getMethod("localTimestampMillis");
+ LogicalType logicalType = (LogicalType) method.invoke(null);
+ return logicalType.addToSchema(Schema.create(Schema.Type.LONG));
+ } catch (Exception e) {
+ // Method doesn't exist (e.g., Avro 1.8.2)
+ throw new UnsupportedOperationException("LocalTimestampMillis is not supported in this Avro version", e);
+ }
+ }
}
diff --git a/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/SerDeHelper.java b/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/SerDeHelper.java
index f47d7f8da517b..6d009fbe55aac 100644
--- a/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/SerDeHelper.java
+++ b/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/SerDeHelper.java
@@ -217,8 +217,16 @@ private static Type parseTypeFromJson(JsonNode jsonNode) {
return Types.DateType.get();
case TIME:
return Types.TimeType.get();
+ case TIME_MILLIS:
+ return Types.TimeMillisType.get();
case TIMESTAMP:
return Types.TimestampType.get();
+ case TIMESTAMP_MILLIS:
+ return Types.TimestampMillisType.get();
+ case LOCAL_TIMESTAMP_MICROS:
+ return Types.LocalTimestampMicrosType.get();
+ case LOCAL_TIMESTAMP_MILLIS:
+ return Types.LocalTimestampMillisType.get();
case STRING:
return Types.StringType.get();
case UUID:
diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroFileWriterFactory.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroFileWriterFactory.java
index 8ed597ed920df..380a7527a42f9 100644
--- a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroFileWriterFactory.java
+++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroFileWriterFactory.java
@@ -33,7 +33,6 @@
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.io.compress.Compression;
import org.apache.orc.CompressionKind;
-import org.apache.parquet.avro.AvroSchemaConverter;
import org.apache.parquet.hadoop.metadata.CompressionCodecName;
import org.apache.parquet.schema.MessageType;
@@ -45,6 +44,8 @@
import static org.apache.hudi.io.storage.HoodieHFileConfig.HFILE_COMPARATOR;
import static org.apache.hudi.io.storage.HoodieHFileConfig.PREFETCH_ON_OPEN;
+import static org.apache.parquet.avro.HoodieAvroParquetSchemaConverter.getAvroSchemaConverter;
+
public class HoodieAvroFileWriterFactory extends HoodieFileWriterFactory {
protected HoodieFileWriter newParquetFileWriter(
@@ -113,6 +114,6 @@ private HoodieAvroWriteSupport getHoodieAvroWriteSupport(Configuration conf, Sch
return (HoodieAvroWriteSupport) ReflectionUtils.loadClass(
config.getStringOrDefault(HoodieStorageConfig.HOODIE_AVRO_WRITE_SUPPORT_CLASS),
new Class>[] {MessageType.class, Schema.class, Option.class, Properties.class},
- new AvroSchemaConverter(conf).convert(schema), schema, filter, config.getProps());
+ getAvroSchemaConverter(conf).convert(schema), schema, filter, config.getProps());
}
}
diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroParquetReader.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroParquetReader.java
index ad4d1f16a60ce..58c11ef2c656b 100644
--- a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroParquetReader.java
+++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroParquetReader.java
@@ -18,12 +18,14 @@
package org.apache.hudi.io.storage;
+import org.apache.hudi.avro.AvroSchemaUtils;
import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.common.bloom.BloomFilter;
import org.apache.hudi.common.model.HoodieAvroIndexedRecord;
import org.apache.hudi.common.model.HoodieFileFormat;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.util.BaseFileUtils;
+import org.apache.hudi.common.util.HoodieAvroParquetReaderIterator;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.ParquetReaderIterator;
import org.apache.hudi.common.util.collection.ClosableIterator;
@@ -43,8 +45,11 @@
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
+import java.util.Map;
import java.util.Set;
+import java.util.stream.Collectors;
+import static org.apache.hudi.avro.AvroSchemaUtils.getRepairedSchema;
import static org.apache.hudi.common.util.TypeUtils.unsafeCast;
/**
@@ -95,8 +100,8 @@ protected ClosableIterator getIndexedRecordIterator(Schema schema
}
@Override
- protected ClosableIterator getIndexedRecordIterator(Schema readerSchema, Schema requestedSchema) throws IOException {
- return getIndexedRecordIteratorInternal(readerSchema, Option.of(requestedSchema));
+ public ClosableIterator getIndexedRecordIterator(Schema readerSchema, Schema requestedSchema) throws IOException {
+ return getIndexedRecordIteratorInternal(requestedSchema, Option.empty());
}
@Override
@@ -154,23 +159,76 @@ private static Configuration tryOverrideDefaultConfigs(Configuration conf) {
return conf;
}
- private ClosableIterator getIndexedRecordIteratorInternal(Schema schema, Option requestedSchema) throws IOException {
+ private ClosableIterator getIndexedRecordIteratorInternal(Schema schema, Option renamedColumns) throws IOException {
// NOTE: We have to set both Avro read-schema and projection schema to make
// sure that in case the file-schema is not equal to read-schema we'd still
// be able to read that file (in case projection is a proper one)
- if (!requestedSchema.isPresent()) {
+ Schema repairedFileSchema;
+ if (AvroSchemaUtils.isLogicalTimestampRepairNeeded(conf, () -> true)) {
+ repairedFileSchema = getRepairedSchema(getSchema(), schema);
+ } else {
+ repairedFileSchema = schema;
+ }
+ Option promotedSchema = Option.empty();
+ if (!renamedColumns.isPresent() || HoodieAvroUtils.recordNeedsRewriteForExtendedAvroTypePromotion(repairedFileSchema, schema)) {
+ AvroReadSupport.setAvroReadSchema(conf, repairedFileSchema);
+ Schema projectionSchema = computeSafeProjection(repairedFileSchema, schema);
+ AvroReadSupport.setRequestedProjection(conf, projectionSchema);
+ promotedSchema = Option.of(schema);
+ } else {
AvroReadSupport.setAvroReadSchema(conf, schema);
AvroReadSupport.setRequestedProjection(conf, schema);
- } else {
- AvroReadSupport.setAvroReadSchema(conf, requestedSchema.get());
- AvroReadSupport.setRequestedProjection(conf, requestedSchema.get());
}
- ParquetReader reader = new HoodieAvroParquetReaderBuilder(path).withConf(conf).build();
- ParquetReaderIterator parquetReaderIterator = new ParquetReaderIterator<>(reader);
+ ParquetReader reader =
+ new HoodieAvroParquetReaderBuilder(path,
+ AvroSchemaUtils.isLogicalTimestampRepairNeeded(conf, () -> schema == null || AvroSchemaUtils.hasTimestampMillisField(schema)))
+ .withTableSchema(schema)
+ .withConf(conf)
+ .set(AvroSchemaConverter.ADD_LIST_ELEMENT_RECORDS, conf.get(AvroSchemaConverter.ADD_LIST_ELEMENT_RECORDS))
+ .set(ParquetInputFormat.STRICT_TYPE_CHECKING, conf.get(ParquetInputFormat.STRICT_TYPE_CHECKING))
+ .build();
+ ParquetReaderIterator parquetReaderIterator = promotedSchema.isPresent()
+ ? new HoodieAvroParquetReaderIterator(reader, promotedSchema.get())
+ : new ParquetReaderIterator<>(reader);
readerIterators.add(parquetReaderIterator);
return parquetReaderIterator;
}
+ /**
+ * Computes a safe projection schema by intersecting the requested schema with the file schema.
+ * This ensures we only request fields that actually exist in the file, enabling column pruning
+ * while avoiding "field not found" errors.
+ *
+ * @param fileSchema The schema from the file (with repaired types)
+ * @param requestedSchema The schema we'd like to read
+ * @return A projection schema containing only fields that exist in both schemas
+ */
+ private Schema computeSafeProjection(Schema fileSchema, Schema requestedSchema) {
+ Map fileFields = fileSchema.getFields().stream()
+ .collect(Collectors.toMap(Schema.Field::name, f -> f));
+
+ List projectedFields = requestedSchema.getFields().stream()
+ .filter(field -> fileFields.containsKey(field.name()))
+ .map(field -> {
+ Schema.Field fileField = fileFields.get(field.name());
+ return new Schema.Field(fileField.name(), fileField.schema(), fileField.doc(), fileField.defaultVal());
+ })
+ .collect(Collectors.toList());
+
+ if (projectedFields.isEmpty()) {
+ return fileSchema;
+ }
+
+ Schema projectedSchema = Schema.createRecord(
+ fileSchema.getName(),
+ fileSchema.getDoc(),
+ fileSchema.getNamespace(),
+ fileSchema.isError()
+ );
+ projectedSchema.setFields(projectedFields);
+ return projectedSchema;
+ }
+
@Override
public ClosableIterator getRecordKeyIterator() throws IOException {
ClosableIterator recordKeyIterator = getIndexedRecordIterator(HoodieAvroUtils.getRecordKeySchema());
diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieFileReader.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieFileReader.java
index 00fff9a220c64..b6a2b8d60a074 100644
--- a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieFileReader.java
+++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieFileReader.java
@@ -41,6 +41,8 @@
*/
public interface HoodieFileReader extends AutoCloseable {
+ String ENABLE_LOGICAL_TIMESTAMP_REPAIR = "spark.hudi.logicalTimestampField.repair.enable";
+
String[] readMinMaxRecordKeys();
BloomFilter readBloomFilter();
diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java
index 553e9d2e38aca..2be2cbb94060b 100644
--- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java
+++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java
@@ -73,6 +73,7 @@
import org.apache.hudi.util.Lazy;
import org.apache.avro.AvroTypeException;
+import org.apache.avro.LogicalType;
import org.apache.avro.LogicalTypes;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
@@ -109,7 +110,7 @@
import java.util.stream.Stream;
import static java.util.stream.Collectors.toList;
-import static org.apache.hudi.avro.AvroSchemaUtils.resolveNullableSchema;
+import static org.apache.hudi.avro.AvroSchemaUtils.getNonNullTypeFromUnion;
import static org.apache.hudi.avro.HoodieAvroUtils.addMetadataFields;
import static org.apache.hudi.avro.HoodieAvroUtils.convertValueForSpecificDataTypes;
import static org.apache.hudi.avro.HoodieAvroUtils.getNestedFieldSchemaFromWriteSchema;
@@ -215,9 +216,8 @@ class ColumnStats {
ColumnStats colStats = allColumnStats.computeIfAbsent(field.name(), (ignored) -> new ColumnStats());
GenericRecord genericRecord = (GenericRecord) record;
-
final Object fieldVal = convertValueForSpecificDataTypes(field.schema(), genericRecord.get(field.name()), false);
- final Schema fieldSchema = getNestedFieldSchemaFromWriteSchema(genericRecord.getSchema(), field.name());
+ final Schema fieldSchema = getNonNullTypeFromUnion(getNestedFieldSchemaFromWriteSchema(genericRecord.getSchema(), field.name()));
colStats.valueCount++;
@@ -241,7 +241,7 @@ class ColumnStats {
Collector, ?, Map>> collector =
Collectors.toMap(colRangeMetadata -> colRangeMetadata.getColumnName(), Function.identity());
- return (Map>) targetFields.stream()
+ Stream> stream = targetFields.stream()
.map(field -> {
ColumnStats colStats = allColumnStats.get(field.name());
return HoodieColumnRangeMetadata.create(
@@ -257,8 +257,9 @@ class ColumnStats {
0,
0
);
- })
- .collect(collector);
+ });
+
+ return stream.collect(collector);
}
/**
@@ -1060,18 +1061,47 @@ private static List getColumnsToIndex(MetadataRecordsGenerationParams re
List targetColumns = recordsGenParams.getTargetColumnsForColumnStatsIndex();
if (!targetColumns.isEmpty()) {
- return targetColumns;
+ // Filter out timestamp-millis columns from the explicitly specified columns
+ Option writerSchemaOpt = lazyWriterSchemaOpt.get();
+ return writerSchemaOpt
+ .map(writerSchema ->
+ targetColumns.stream()
+ .filter(colName -> {
+ Schema.Field field = writerSchema.getField(colName);
+ return field != null && !isTimestampMillisField(field.schema());
+ })
+ .collect(Collectors.toList()))
+ .orElse(targetColumns);
}
Option writerSchemaOpt = lazyWriterSchemaOpt.get();
return writerSchemaOpt
.map(writerSchema ->
writerSchema.getFields().stream()
+ .filter(field -> !isTimestampMillisField(field.schema()))
.map(Schema.Field::name)
.collect(Collectors.toList()))
.orElse(Collections.emptyList());
}
+ /**
+ * Checks if a schema field is of type timestamp-millis (timestamp-millis or local-timestamp-millis).
+ *
+ * @param fieldSchema The schema of the field to check
+ * @return true if the field is of type timestamp-millis, false otherwise
+ */
+ private static boolean isTimestampMillisField(Schema fieldSchema) {
+ Schema nonNullableSchema = getNonNullTypeFromUnion(fieldSchema);
+ if (nonNullableSchema.getType() == Schema.Type.LONG) {
+ LogicalType logicalType = nonNullableSchema.getLogicalType();
+ if (logicalType != null) {
+ String logicalTypeName = logicalType.getName();
+ return logicalTypeName.equals("timestamp-millis") || logicalTypeName.equals("local-timestamp-millis");
+ }
+ }
+ return false;
+ }
+
private static Stream translateWriteStatToColumnStats(HoodieWriteStat writeStat,
HoodieTableMetaClient datasetMetaClient,
List columnsToIndex) {
@@ -1191,7 +1221,7 @@ private static Comparable> coerceToComparable(Schema schema, Object val) {
switch (schema.getType()) {
case UNION:
// TODO we need to handle unions in general case as well
- return coerceToComparable(resolveNullableSchema(schema), val);
+ return coerceToComparable(getNonNullTypeFromUnion(schema), val);
case FIXED:
case BYTES:
diff --git a/hudi-common/src/main/java/org/apache/parquet/avro/HoodieAvroParquetReaderBuilder.java b/hudi-common/src/main/java/org/apache/parquet/avro/HoodieAvroParquetReaderBuilder.java
index d6179ea1aacd2..02b3803a012d9 100644
--- a/hudi-common/src/main/java/org/apache/parquet/avro/HoodieAvroParquetReaderBuilder.java
+++ b/hudi-common/src/main/java/org/apache/parquet/avro/HoodieAvroParquetReaderBuilder.java
@@ -18,6 +18,9 @@
package org.apache.parquet.avro;
+import org.apache.hudi.common.util.Option;
+
+import org.apache.avro.Schema;
import org.apache.avro.generic.GenericData;
import org.apache.avro.specific.SpecificData;
import org.apache.hadoop.fs.Path;
@@ -25,6 +28,8 @@
import org.apache.parquet.hadoop.api.ReadSupport;
import org.apache.parquet.io.InputFile;
+import static org.apache.parquet.avro.HoodieAvroParquetSchemaConverter.getAvroSchemaConverter;
+
/**
* Copy from org.apache.parquet.avro.AvroParquetReader.Builder.
* We use HoodieAvroParquetReaderBuilder to build HoodieAvroReadSupport
@@ -35,10 +40,13 @@ public class HoodieAvroParquetReaderBuilder extends ParquetReader.Builder
private GenericData model = null;
private boolean enableCompatibility = true;
private boolean isReflect = true;
+ private boolean isLogicalTimestampRepairNeeded;
+ private Schema tableSchema = null;
@Deprecated
- public HoodieAvroParquetReaderBuilder(Path path) {
+ public HoodieAvroParquetReaderBuilder(Path path, boolean isLogicalTimestampRepairNeeded) {
super(path);
+ this.isLogicalTimestampRepairNeeded = isLogicalTimestampRepairNeeded;
}
public HoodieAvroParquetReaderBuilder(InputFile file) {
@@ -67,6 +75,11 @@ public HoodieAvroParquetReaderBuilder withCompatibility(boolean enableCompati
return this;
}
+ public HoodieAvroParquetReaderBuilder withTableSchema(Schema tableSchema) {
+ this.tableSchema = tableSchema;
+ return this;
+ }
+
@Override
protected ReadSupport getReadSupport() {
if (isReflect) {
@@ -74,6 +87,7 @@ protected ReadSupport getReadSupport() {
} else {
conf.setBoolean(AvroReadSupport.AVRO_COMPATIBILITY, enableCompatibility);
}
- return new HoodieAvroReadSupport<>(model);
+ return new HoodieAvroReadSupport<>(model, Option.ofNullable(tableSchema).map(schema -> getAvroSchemaConverter(conf).convert(schema)),
+ isLogicalTimestampRepairNeeded);
}
}
diff --git a/hudi-common/src/main/java/org/apache/parquet/avro/HoodieAvroParquetSchemaConverter.java b/hudi-common/src/main/java/org/apache/parquet/avro/HoodieAvroParquetSchemaConverter.java
new file mode 100644
index 0000000000000..2a748eb247f77
--- /dev/null
+++ b/hudi-common/src/main/java/org/apache/parquet/avro/HoodieAvroParquetSchemaConverter.java
@@ -0,0 +1,56 @@
+/*
+ * 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.parquet.avro;
+
+import org.apache.hudi.common.util.ReflectionUtils;
+
+import org.apache.avro.Schema;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.parquet.schema.MessageType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Parquet-Java AvroSchemaConverter doesn't support local timestamp types until version 1.14
+ * for this reason we use a modified version of the AvroSchemaConverter that adds support for local timestamp types
+ * Parquet-Java still supports local timestamp types from version 1.11.0, just that the AvroSchemaConverter
+ * doesn't work.
+ *
+ * However, for versions < 1.11.0, local timestamp is not supported at all. Therefore, we just use the
+ * library AvroSchemaConverter in this case.
+ *
+ */
+public abstract class HoodieAvroParquetSchemaConverter {
+ private static final Logger LOG = LoggerFactory.getLogger(HoodieAvroParquetSchemaConverter.class);
+ public static HoodieAvroParquetSchemaConverter getAvroSchemaConverter(Configuration configuration) {
+ try {
+ return (HoodieAvroParquetSchemaConverter) ReflectionUtils.loadClass("org.apache.parquet.avro.AvroSchemaConverterWithTimestampNTZ",
+ new Class>[] {Configuration.class}, configuration);
+ } catch (Throwable t) {
+ LOG.debug("Failed to load AvroSchemaConverterWithTimestampNTZ, using NativeAvroSchemaConverter instead", t);
+ return (HoodieAvroParquetSchemaConverter) ReflectionUtils.loadClass("org.apache.parquet.avro.NativeAvroSchemaConverter",
+ new Class>[] {Configuration.class}, configuration);
+ }
+ }
+
+ public abstract MessageType convert(Schema schema);
+
+ public abstract Schema convert(MessageType schema);
+}
diff --git a/hudi-common/src/main/java/org/apache/parquet/avro/HoodieAvroReadSupport.java b/hudi-common/src/main/java/org/apache/parquet/avro/HoodieAvroReadSupport.java
index 326accb66b2c2..1ed4cc993a97d 100644
--- a/hudi-common/src/main/java/org/apache/parquet/avro/HoodieAvroReadSupport.java
+++ b/hudi-common/src/main/java/org/apache/parquet/avro/HoodieAvroReadSupport.java
@@ -18,6 +18,8 @@
package org.apache.parquet.avro;
+import org.apache.hudi.common.util.Option;
+
import org.apache.avro.generic.GenericData;
import org.apache.hadoop.conf.Configuration;
import org.apache.parquet.schema.GroupType;
@@ -25,6 +27,8 @@
import org.apache.parquet.schema.OriginalType;
import org.apache.parquet.schema.Type;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
@@ -36,11 +40,17 @@
*/
public class HoodieAvroReadSupport extends AvroReadSupport {
- public HoodieAvroReadSupport(GenericData model) {
+ private Option tableSchema;
+ private boolean hasTimestampMillisField;
+
+ public HoodieAvroReadSupport(GenericData model, Option tableSchema, boolean hasTimestampMillisField) {
super(model);
+ this.tableSchema = tableSchema;
+ this.hasTimestampMillisField = hasTimestampMillisField;
}
public HoodieAvroReadSupport() {
+ tableSchema = Option.empty();
}
@Override
@@ -52,7 +62,12 @@ public ReadContext init(Configuration configuration, Map keyValu
"false", "support reading avro from non-legacy map/list in parquet file");
}
ReadContext readContext = super.init(configuration, keyValueMetaData, fileSchema);
- MessageType requestedSchema = readContext.getRequestedSchema();
+ MessageType requestedSchema;
+ if (hasTimestampMillisField) {
+ requestedSchema = repairLogicalTypes(readContext.getRequestedSchema(), tableSchema);
+ } else {
+ requestedSchema = readContext.getRequestedSchema();
+ }
// support non-legacy map. Convert non-legacy map to legacy map
// Because there is no AvroWriteSupport.WRITE_OLD_MAP_STRUCTURE
// according to AvroWriteSupport.WRITE_OLD_LIST_STRUCTURE
@@ -130,4 +145,16 @@ private List convertLegacyMap(List oldTypes) {
}
return newTypes;
}
+
+ private MessageType repairLogicalTypes(MessageType fileSchema, Option tableSchemaOpt) {
+ try {
+ Class> repairClass = Class.forName("org.apache.parquet.schema.SchemaRepair");
+ Method repairMethod = repairClass.getDeclaredMethod(
+ "repairLogicalTypes", MessageType.class, Option.class);
+ MessageType repaired = (MessageType) repairMethod.invoke(null, fileSchema, tableSchemaOpt);
+ return repaired != null ? repaired : fileSchema;
+ } catch (ClassNotFoundException | NoSuchMethodException | IllegalAccessException | InvocationTargetException e) {
+ return fileSchema;
+ }
+ }
}
diff --git a/hudi-common/src/main/java/org/apache/parquet/avro/NativeAvroSchemaConverter.java b/hudi-common/src/main/java/org/apache/parquet/avro/NativeAvroSchemaConverter.java
new file mode 100644
index 0000000000000..509a8afb3ea83
--- /dev/null
+++ b/hudi-common/src/main/java/org/apache/parquet/avro/NativeAvroSchemaConverter.java
@@ -0,0 +1,46 @@
+/*
+ * 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.parquet.avro;
+
+import org.apache.avro.Schema;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.parquet.schema.MessageType;
+
+/**
+ * uses the native avro schema converter from parquet java
+ */
+public class NativeAvroSchemaConverter extends HoodieAvroParquetSchemaConverter {
+
+ private final AvroSchemaConverter avroSchemaConverter;
+
+ public NativeAvroSchemaConverter(Configuration configuration) {
+ this.avroSchemaConverter = new AvroSchemaConverter(configuration);
+ }
+
+ @Override
+ public MessageType convert(Schema schema) {
+ return avroSchemaConverter.convert(schema);
+ }
+
+ @Override
+ public Schema convert(MessageType schema) {
+ return avroSchemaConverter.convert(schema);
+ }
+}
diff --git a/hudi-common/src/parquet/java/org/apache/parquet/schema/SchemaRepair.java b/hudi-common/src/parquet/java/org/apache/parquet/schema/SchemaRepair.java
new file mode 100644
index 0000000000000..09ef59ea50bdb
--- /dev/null
+++ b/hudi-common/src/parquet/java/org/apache/parquet/schema/SchemaRepair.java
@@ -0,0 +1,163 @@
+/*
+ * 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.parquet.schema;
+
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieException;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class SchemaRepair {
+
+ public static MessageType repairLogicalTypes(MessageType fileSchema, Option tableSchema) {
+ if (!tableSchema.isPresent()) {
+ return fileSchema;
+ }
+ return repairLogicalTypes(fileSchema, tableSchema.get());
+ }
+
+ static MessageType repairLogicalTypes(MessageType fileSchema, MessageType tableSchema) {
+ List repairedFields = repairFields(fileSchema.getFields(), tableSchema);
+
+ // If nothing changed, return the original schema
+ if (repairedFields == null) {
+ return fileSchema;
+ }
+
+ return new MessageType(fileSchema.getName(), repairedFields);
+ }
+
+ /**
+ * Repairs a list of fields against a table schema (MessageType or GroupType).
+ * Returns null if no changes were made, otherwise returns the repaired field list.
+ */
+ private static List repairFields(List fileSchemaFields, GroupType tableSchema) {
+ // First pass: find the first field that changes
+ int firstChangedIndex = -1;
+ Type firstRepairedField = null;
+
+ for (int i = 0; i < fileSchemaFields.size(); i++) {
+ Type requestedField = fileSchemaFields.get(i);
+ if (tableSchema.containsField(requestedField.getName())) {
+ Type tableField = tableSchema.getType(requestedField.getName());
+ Type repaired = repairField(requestedField, tableField);
+ if (repaired != requestedField) {
+ firstChangedIndex = i;
+ firstRepairedField = repaired;
+ break;
+ }
+ }
+ }
+
+ // If nothing changed, return null
+ if (firstChangedIndex == -1) {
+ return null;
+ }
+
+ // Second pass: build the new field list with repaired fields
+ List repairedFields = new ArrayList<>(fileSchemaFields.size());
+
+ // Copy all fields before the first changed field
+ for (int i = 0; i < firstChangedIndex; i++) {
+ repairedFields.add(fileSchemaFields.get(i));
+ }
+
+ // Add the first changed field (using cached repaired field)
+ repairedFields.add(firstRepairedField);
+
+ // Process remaining fields
+ for (int i = firstChangedIndex + 1; i < fileSchemaFields.size(); i++) {
+ Type fileSchemaField = fileSchemaFields.get(i);
+ Type repaired = fileSchemaField;
+ if (tableSchema.containsField(fileSchemaField.getName())) {
+ Type tableSchemaField = tableSchema.getType(fileSchemaField.getName());
+ repaired = repairField(fileSchemaField, tableSchemaField);
+ }
+ repairedFields.add(repaired);
+ }
+
+ return repairedFields;
+ }
+
+ private static Type repairField(Type fileSchemaFieldType, Type tableSchemaFieldType) {
+ if (fileSchemaFieldType.isPrimitive() && tableSchemaFieldType.isPrimitive()) {
+ return repairPrimitiveType(fileSchemaFieldType.asPrimitiveType(), tableSchemaFieldType.asPrimitiveType());
+ } else if (!fileSchemaFieldType.isPrimitive() && !tableSchemaFieldType.isPrimitive()) {
+ // recurse into nested structs
+ GroupType reqGroup = fileSchemaFieldType.asGroupType();
+ GroupType tblGroup = tableSchemaFieldType.asGroupType();
+
+ // Repair fields directly without creating MessageType intermediaries
+ List repairedFields = repairFields(reqGroup.getFields(), tblGroup);
+
+ // If nothing changed, return the original field
+ if (repairedFields == null) {
+ return fileSchemaFieldType;
+ }
+
+ return new GroupType(
+ reqGroup.getRepetition(),
+ reqGroup.getName(),
+ reqGroup.getLogicalTypeAnnotation(),
+ repairedFields
+ );
+ } else {
+ // fallback: keep requested
+ return fileSchemaFieldType;
+ }
+ }
+
+ private static PrimitiveType repairPrimitiveType(PrimitiveType fileSchemaPrimitiveType, PrimitiveType tableSchemaPrimitiveType) {
+ // Quick check if repair is needed (no allocations)
+ if (needsLogicalTypeRepair(fileSchemaPrimitiveType, tableSchemaPrimitiveType)) {
+ return Types.primitive(tableSchemaPrimitiveType.getPrimitiveTypeName(), fileSchemaPrimitiveType.getRepetition())
+ .as(tableSchemaPrimitiveType.getLogicalTypeAnnotation())
+ .named(fileSchemaPrimitiveType.getName());
+ }
+ return fileSchemaPrimitiveType;
+ }
+
+ /**
+ * Quick check if a logical type repair is needed (no allocations).
+ */
+ private static boolean needsLogicalTypeRepair(PrimitiveType fileSchemaPrimitiveType, PrimitiveType tableSchemaPrimitiveType) {
+ if (fileSchemaPrimitiveType.getPrimitiveTypeName() != PrimitiveType.PrimitiveTypeName.INT64
+ || tableSchemaPrimitiveType.getPrimitiveTypeName() != PrimitiveType.PrimitiveTypeName.INT64) {
+ return false;
+ }
+ LogicalTypeAnnotation fileLogicalTypeAnnotation = fileSchemaPrimitiveType.getLogicalTypeAnnotation();
+ LogicalTypeAnnotation tableLogicalTypeAnnotation = tableSchemaPrimitiveType.getLogicalTypeAnnotation();
+
+ // if requested has no logical type, and the table has a local timestamp, then we need to repair
+ if (fileLogicalTypeAnnotation == null) {
+ return tableLogicalTypeAnnotation instanceof LogicalTypeAnnotation.TimestampLogicalTypeAnnotation
+ && !((LogicalTypeAnnotation.TimestampLogicalTypeAnnotation) tableLogicalTypeAnnotation).isAdjustedToUTC();
+ }
+
+ // if requested is timestamp-micros and table is timestamp-millis then we need to repair
+ return fileLogicalTypeAnnotation instanceof LogicalTypeAnnotation.TimestampLogicalTypeAnnotation
+ && tableLogicalTypeAnnotation instanceof LogicalTypeAnnotation.TimestampLogicalTypeAnnotation
+ && ((LogicalTypeAnnotation.TimestampLogicalTypeAnnotation) fileLogicalTypeAnnotation).getUnit() == LogicalTypeAnnotation.TimeUnit.MICROS
+ && ((LogicalTypeAnnotation.TimestampLogicalTypeAnnotation) tableLogicalTypeAnnotation).getUnit() == LogicalTypeAnnotation.TimeUnit.MILLIS
+ && ((LogicalTypeAnnotation.TimestampLogicalTypeAnnotation) fileLogicalTypeAnnotation).isAdjustedToUTC()
+ && ((LogicalTypeAnnotation.TimestampLogicalTypeAnnotation) tableLogicalTypeAnnotation).isAdjustedToUTC();
+ }
+}
diff --git a/hudi-common/src/parquet/test/org/apache/parquet/avro/TestAvroSchemaConverter.java b/hudi-common/src/parquet/test/org/apache/parquet/avro/TestAvroSchemaConverter.java
new file mode 100644
index 0000000000000..1787a0b83628c
--- /dev/null
+++ b/hudi-common/src/parquet/test/org/apache/parquet/avro/TestAvroSchemaConverter.java
@@ -0,0 +1,954 @@
+/*
+ * 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.parquet.avro;
+
+import org.apache.avro.JsonProperties;
+import org.apache.avro.LogicalTypes;
+import org.apache.avro.Schema;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.parquet.schema.LogicalTypeAnnotation;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.MessageTypeParser;
+import org.apache.parquet.schema.PrimitiveType;
+import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName;
+import org.apache.parquet.schema.Type;
+import org.apache.parquet.schema.Types;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+
+import static org.apache.avro.Schema.Type.INT;
+import static org.apache.avro.Schema.Type.LONG;
+import static org.apache.avro.Schema.Type.STRING;
+import static org.apache.avro.SchemaCompatibility.SchemaCompatibilityType.COMPATIBLE;
+import static org.apache.avro.SchemaCompatibility.checkReaderWriterCompatibility;
+import static org.apache.hudi.common.testutils.SchemaTestUtil.getSchemaFromResource;
+import static org.apache.parquet.avro.HoodieAvroParquetSchemaConverter.getAvroSchemaConverter;
+import static org.apache.parquet.schema.OriginalType.DATE;
+import static org.apache.parquet.schema.OriginalType.TIMESTAMP_MICROS;
+import static org.apache.parquet.schema.OriginalType.TIMESTAMP_MILLIS;
+import static org.apache.parquet.schema.OriginalType.TIME_MICROS;
+import static org.apache.parquet.schema.OriginalType.TIME_MILLIS;
+import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.BINARY;
+import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.BOOLEAN;
+import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.DOUBLE;
+import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY;
+import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.FLOAT;
+import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT32;
+import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT64;
+import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT96;
+import static org.apache.parquet.schema.Type.Repetition.REQUIRED;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.fail;
+
+public class TestAvroSchemaConverter {
+
+ private static final Configuration NEW_BEHAVIOR = new Configuration(false);
+
+ @BeforeAll
+ public static void setupConf() {
+ NEW_BEHAVIOR.setBoolean("parquet.avro.add-list-element-records", false);
+ NEW_BEHAVIOR.setBoolean("parquet.avro.write-old-list-structure", false);
+ }
+
+ public static final String ALL_PARQUET_SCHEMA = "message org.apache.parquet.avro.myrecord {\n"
+ + " required boolean myboolean;\n"
+ + " required int32 myint;\n"
+ + " required int64 mylong;\n"
+ + " required float myfloat;\n"
+ + " required double mydouble;\n"
+ + " required binary mybytes;\n"
+ + " required binary mystring (UTF8);\n"
+ + " required group mynestedrecord {\n"
+ + " required int32 mynestedint;\n"
+ + " }\n"
+ + " required binary myenum (ENUM);\n"
+ + " required group myarray (LIST) {\n"
+ + " repeated int32 array;\n"
+ + " }\n"
+ + " optional group myoptionalarray (LIST) {\n"
+ + " repeated int32 array;\n"
+ + " }\n"
+ + " required group myarrayofoptional (LIST) {\n"
+ + " repeated group list {\n"
+ + " optional int32 element;\n"
+ + " }\n"
+ + " }\n"
+ + " required group myrecordarray (LIST) {\n"
+ + " repeated group array {\n"
+ + " required int32 a;\n"
+ + " required int32 b;\n"
+ + " }\n"
+ + " }\n"
+ + " required group mymap (MAP) {\n"
+ + " repeated group map (MAP_KEY_VALUE) {\n"
+ + " required binary key (UTF8);\n"
+ + " required int32 value;\n"
+ + " }\n"
+ + " }\n"
+ + " required fixed_len_byte_array(1) myfixed;\n"
+ + "}\n";
+
+ private void testAvroToParquetConversion(Schema avroSchema, String schemaString) throws Exception {
+ testAvroToParquetConversion(new Configuration(false), avroSchema, schemaString);
+ }
+
+ private void testAvroToParquetConversion(Configuration conf, Schema avroSchema, String schemaString)
+ throws Exception {
+ HoodieAvroParquetSchemaConverter avroSchemaConverter = getAvroSchemaConverter(conf);
+ MessageType schema = avroSchemaConverter.convert(avroSchema);
+ MessageType expectedMT = MessageTypeParser.parseMessageType(schemaString);
+ assertEquals(expectedMT.toString(), schema.toString());
+ }
+
+ private void testParquetToAvroConversion(Schema avroSchema, String schemaString) throws Exception {
+ testParquetToAvroConversion(new Configuration(false), avroSchema, schemaString);
+ }
+
+ private void testParquetToAvroConversion(Configuration conf, Schema avroSchema, String schemaString)
+ throws Exception {
+ HoodieAvroParquetSchemaConverter avroSchemaConverter = getAvroSchemaConverter(conf);
+ Schema schema = avroSchemaConverter.convert(MessageTypeParser.parseMessageType(schemaString));
+ assertEquals(avroSchema.toString(), schema.toString());
+ }
+
+ private void testRoundTripConversion(Schema avroSchema, String schemaString) throws Exception {
+ testRoundTripConversion(new Configuration(), avroSchema, schemaString);
+ }
+
+ private void testRoundTripConversion(Configuration conf, Schema avroSchema, String schemaString) throws Exception {
+ HoodieAvroParquetSchemaConverter avroSchemaConverter = getAvroSchemaConverter(conf);
+ MessageType schema = avroSchemaConverter.convert(avroSchema);
+ MessageType expectedMT = MessageTypeParser.parseMessageType(schemaString);
+ assertEquals(expectedMT.toString(), schema.toString());
+ Schema convertedAvroSchema = avroSchemaConverter.convert(expectedMT);
+ assertEquals(avroSchema.toString(), convertedAvroSchema.toString());
+ }
+
+ @Test()
+ public void testTopLevelMustBeARecord() {
+ assertThrows("expected to throw", IllegalArgumentException.class, () -> getAvroSchemaConverter(new Configuration()).convert(Schema.create(INT)));
+ }
+
+ @Test
+ public void testAllTypes() throws Exception {
+ Schema schema = getSchemaFromResource(TestAvroSchemaConverter.class, "/parquet-java/all.avsc");
+ testAvroToParquetConversion(
+ NEW_BEHAVIOR,
+ schema,
+ "message org.apache.parquet.avro.myrecord {\n"
+ // Avro nulls are not encoded, unless they are null unions
+ + " required boolean myboolean;\n"
+ + " required int32 myint;\n"
+ + " required int64 mylong;\n"
+ + " required float myfloat;\n"
+ + " required double mydouble;\n"
+ + " required binary mybytes;\n"
+ + " required binary mystring (UTF8);\n"
+ + " required group mynestedrecord {\n"
+ + " required int32 mynestedint;\n"
+ + " }\n"
+ + " required binary myenum (ENUM);\n"
+ + " required group myarray (LIST) {\n"
+ + " repeated group list {\n"
+ + " required int32 element;\n"
+ + " }\n"
+ + " }\n"
+ + " required group myemptyarray (LIST) {\n"
+ + " repeated group list {\n"
+ + " required int32 element;\n"
+ + " }\n"
+ + " }\n"
+ + " optional group myoptionalarray (LIST) {\n"
+ + " repeated group list {\n"
+ + " required int32 element;\n"
+ + " }\n"
+ + " }\n"
+ + " required group myarrayofoptional (LIST) {\n"
+ + " repeated group list {\n"
+ + " optional int32 element;\n"
+ + " }\n"
+ + " }\n"
+ + " required group mymap (MAP) {\n"
+ + " repeated group key_value (MAP_KEY_VALUE) {\n"
+ + " required binary key (UTF8);\n"
+ + " required int32 value;\n"
+ + " }\n"
+ + " }\n"
+ + " required group myemptymap (MAP) {\n"
+ + " repeated group key_value (MAP_KEY_VALUE) {\n"
+ + " required binary key (UTF8);\n"
+ + " required int32 value;\n"
+ + " }\n"
+ + " }\n"
+ + " required fixed_len_byte_array(1) myfixed;\n"
+ + "}\n");
+ }
+
+ @Test
+ public void testAllTypesOldListBehavior() throws Exception {
+ Schema schema = getSchemaFromResource(TestAvroSchemaConverter.class, "/parquet-java/all.avsc");
+ testAvroToParquetConversion(
+ schema,
+ "message org.apache.parquet.avro.myrecord {\n"
+ // Avro nulls are not encoded, unless they are null unions
+ + " required boolean myboolean;\n"
+ + " required int32 myint;\n"
+ + " required int64 mylong;\n"
+ + " required float myfloat;\n"
+ + " required double mydouble;\n"
+ + " required binary mybytes;\n"
+ + " required binary mystring (UTF8);\n"
+ + " required group mynestedrecord {\n"
+ + " required int32 mynestedint;\n"
+ + " }\n"
+ + " required binary myenum (ENUM);\n"
+ + " required group myarray (LIST) {\n"
+ + " repeated int32 array;\n"
+ + " }\n"
+ + " required group myemptyarray (LIST) {\n"
+ + " repeated int32 array;\n"
+ + " }\n"
+ + " optional group myoptionalarray (LIST) {\n"
+ + " repeated int32 array;\n"
+ + " }\n"
+ + " required group myarrayofoptional (LIST) {\n"
+ + " repeated int32 array;\n"
+ + " }\n"
+ + " required group mymap (MAP) {\n"
+ + " repeated group key_value (MAP_KEY_VALUE) {\n"
+ + " required binary key (UTF8);\n"
+ + " required int32 value;\n"
+ + " }\n"
+ + " }\n"
+ + " required group myemptymap (MAP) {\n"
+ + " repeated group key_value (MAP_KEY_VALUE) {\n"
+ + " required binary key (UTF8);\n"
+ + " required int32 value;\n"
+ + " }\n"
+ + " }\n"
+ + " required fixed_len_byte_array(1) myfixed;\n"
+ + "}\n");
+ }
+
+ @Test
+ public void testAllTypesParquetToAvro() throws Exception {
+ Schema schema = getSchemaFromResource(TestAvroSchemaConverter.class, "/parquet-java/allFromParquetNewBehavior.avsc");
+ // Cannot use round-trip assertion because enum is lost
+ testParquetToAvroConversion(NEW_BEHAVIOR, schema, ALL_PARQUET_SCHEMA);
+ }
+
+ @Test
+ public void testAllTypesParquetToAvroOldBehavior() throws Exception {
+ Schema schema = getSchemaFromResource(TestAvroSchemaConverter.class, "/parquet-java/allFromParquetOldBehavior.avsc");
+ // Cannot use round-trip assertion because enum is lost
+ testParquetToAvroConversion(schema, ALL_PARQUET_SCHEMA);
+ }
+
+ @Test
+ public void testParquetMapWithNonStringKeyFails() throws Exception {
+ MessageType parquetSchema =
+ MessageTypeParser.parseMessageType("message myrecord {\n" + " required group mymap (MAP) {\n"
+ + " repeated group map (MAP_KEY_VALUE) {\n"
+ + " required int32 key;\n"
+ + " required int32 value;\n"
+ + " }\n"
+ + " }\n"
+ + "}\n");
+ assertThrows("expected to throw", IllegalArgumentException.class, () -> getAvroSchemaConverter(new Configuration()).convert(parquetSchema));
+ }
+
+ @Test
+ public void testOptionalFields() throws Exception {
+ Schema schema = Schema.createRecord("record1", null, null, false);
+ Schema optionalInt = optional(Schema.create(INT));
+ schema.setFields(
+ Collections.singletonList(new Schema.Field("myint", optionalInt, null, JsonProperties.NULL_VALUE)));
+ testRoundTripConversion(schema, "message record1 {\n" + " optional int32 myint;\n" + "}\n");
+ }
+
+ @Test
+ public void testOptionalMapValue() throws Exception {
+ Schema schema = Schema.createRecord("record1", null, null, false);
+ Schema optionalIntMap = Schema.createMap(optional(Schema.create(INT)));
+ schema.setFields(Arrays.asList(new Schema.Field("myintmap", optionalIntMap, null, null)));
+ testRoundTripConversion(
+ schema,
+ "message record1 {\n" + " required group myintmap (MAP) {\n"
+ + " repeated group key_value (MAP_KEY_VALUE) {\n"
+ + " required binary key (UTF8);\n"
+ + " optional int32 value;\n"
+ + " }\n"
+ + " }\n"
+ + "}\n");
+ }
+
+ @Test
+ public void testOptionalArrayElement() throws Exception {
+ Schema schema = Schema.createRecord("record1", null, null, false);
+ Schema optionalIntArray = Schema.createArray(optional(Schema.create(INT)));
+ schema.setFields(Arrays.asList(new Schema.Field("myintarray", optionalIntArray, null, null)));
+ testRoundTripConversion(
+ NEW_BEHAVIOR,
+ schema,
+ "message record1 {\n" + " required group myintarray (LIST) {\n"
+ + " repeated group list {\n"
+ + " optional int32 element;\n"
+ + " }\n"
+ + " }\n"
+ + "}\n");
+ }
+
+ @Test
+ public void testUnionOfTwoTypes() throws Exception {
+ Schema schema = Schema.createRecord("record2", null, null, false);
+ Schema multipleTypes = Schema.createUnion(
+ Arrays.asList(Schema.create(Schema.Type.NULL), Schema.create(INT), Schema.create(Schema.Type.FLOAT)));
+ schema.setFields(Arrays.asList(new Schema.Field("myunion", multipleTypes, null, JsonProperties.NULL_VALUE)));
+
+ // Avro union is modelled using optional data members of the different
+ // types. This does not translate back into an Avro union
+ testAvroToParquetConversion(
+ schema,
+ "message record2 {\n" + " optional group myunion {\n"
+ + " optional int32 member0;\n"
+ + " optional float member1;\n"
+ + " }\n"
+ + "}\n");
+ }
+
+ @Test
+ public void testArrayOfOptionalRecords() throws Exception {
+ Schema innerRecord = Schema.createRecord("element", null, null, false);
+ Schema optionalString = optional(Schema.create(Schema.Type.STRING));
+ innerRecord.setFields(Arrays.asList(
+ new Schema.Field("s1", optionalString, null, JsonProperties.NULL_VALUE),
+ new Schema.Field("s2", optionalString, null, JsonProperties.NULL_VALUE)));
+ Schema schema = Schema.createRecord("HasArray", null, null, false);
+ schema.setFields(
+ Arrays.asList(new Schema.Field("myarray", Schema.createArray(optional(innerRecord)), null, null)));
+ System.err.println("Avro schema: " + schema.toString(true));
+
+ testRoundTripConversion(
+ NEW_BEHAVIOR,
+ schema,
+ "message HasArray {\n" + " required group myarray (LIST) {\n"
+ + " repeated group list {\n"
+ + " optional group element {\n"
+ + " optional binary s1 (UTF8);\n"
+ + " optional binary s2 (UTF8);\n"
+ + " }\n"
+ + " }\n"
+ + " }\n"
+ + "}\n");
+ }
+
+ @Test
+ public void testArrayOfOptionalRecordsOldBehavior() throws Exception {
+ Schema innerRecord = Schema.createRecord("InnerRecord", null, null, false);
+ Schema optionalString = optional(Schema.create(Schema.Type.STRING));
+ innerRecord.setFields(Arrays.asList(
+ new Schema.Field("s1", optionalString, null, JsonProperties.NULL_VALUE),
+ new Schema.Field("s2", optionalString, null, JsonProperties.NULL_VALUE)));
+ Schema schema = Schema.createRecord("HasArray", null, null, false);
+ schema.setFields(
+ Arrays.asList(new Schema.Field("myarray", Schema.createArray(optional(innerRecord)), null, null)));
+ System.err.println("Avro schema: " + schema.toString(true));
+
+ // Cannot use round-trip assertion because InnerRecord optional is removed
+ testAvroToParquetConversion(
+ schema,
+ "message HasArray {\n" + " required group myarray (LIST) {\n"
+ + " repeated group array {\n"
+ + " optional binary s1 (UTF8);\n"
+ + " optional binary s2 (UTF8);\n"
+ + " }\n"
+ + " }\n"
+ + "}\n");
+ }
+
+ @Test
+ public void testOldAvroListOfLists() throws Exception {
+ Schema listOfLists = optional(Schema.createArray(Schema.createArray(Schema.create(INT))));
+ Schema schema = Schema.createRecord("AvroCompatListInList", null, null, false);
+ schema.setFields(
+ Arrays.asList(new Schema.Field("listOfLists", listOfLists, null, JsonProperties.NULL_VALUE)));
+ System.err.println("Avro schema: " + schema.toString(true));
+
+ testRoundTripConversion(
+ schema,
+ "message AvroCompatListInList {\n" + " optional group listOfLists (LIST) {\n"
+ + " repeated group array (LIST) {\n"
+ + " repeated int32 array;\n"
+ + " }\n"
+ + " }\n"
+ + "}");
+ // Cannot use round-trip assertion because 3-level representation is used
+ testParquetToAvroConversion(
+ NEW_BEHAVIOR,
+ schema,
+ "message AvroCompatListInList {\n" + " optional group listOfLists (LIST) {\n"
+ + " repeated group array (LIST) {\n"
+ + " repeated int32 array;\n"
+ + " }\n"
+ + " }\n"
+ + "}");
+ }
+
+ @Test
+ public void testOldThriftListOfLists() throws Exception {
+ Schema listOfLists = optional(Schema.createArray(Schema.createArray(Schema.create(INT))));
+ Schema schema = Schema.createRecord("ThriftCompatListInList", null, null, false);
+ schema.setFields(
+ Arrays.asList(new Schema.Field("listOfLists", listOfLists, null, JsonProperties.NULL_VALUE)));
+ System.err.println("Avro schema: " + schema.toString(true));
+
+ // Cannot use round-trip assertion because repeated group names differ
+ testParquetToAvroConversion(
+ schema,
+ "message ThriftCompatListInList {\n" + " optional group listOfLists (LIST) {\n"
+ + " repeated group listOfLists_tuple (LIST) {\n"
+ + " repeated int32 listOfLists_tuple_tuple;\n"
+ + " }\n"
+ + " }\n"
+ + "}");
+ // Cannot use round-trip assertion because 3-level representation is used
+ testParquetToAvroConversion(
+ NEW_BEHAVIOR,
+ schema,
+ "message ThriftCompatListInList {\n" + " optional group listOfLists (LIST) {\n"
+ + " repeated group listOfLists_tuple (LIST) {\n"
+ + " repeated int32 listOfLists_tuple_tuple;\n"
+ + " }\n"
+ + " }\n"
+ + "}");
+ }
+
+ @Test
+ public void testUnknownTwoLevelListOfLists() throws Exception {
+ // This tests the case where we don't detect a 2-level list by the repeated
+ // group's name, but it must be 2-level because the repeated group doesn't
+ // contain an optional or repeated element as required for 3-level lists
+ Schema listOfLists = optional(Schema.createArray(Schema.createArray(Schema.create(INT))));
+ Schema schema = Schema.createRecord("UnknownTwoLevelListInList", null, null, false);
+ schema.setFields(
+ Arrays.asList(new Schema.Field("listOfLists", listOfLists, null, JsonProperties.NULL_VALUE)));
+ System.err.println("Avro schema: " + schema.toString(true));
+
+ // Cannot use round-trip assertion because repeated group names differ
+ testParquetToAvroConversion(
+ schema,
+ "message UnknownTwoLevelListInList {\n" + " optional group listOfLists (LIST) {\n"
+ + " repeated group mylist (LIST) {\n"
+ + " repeated int32 innerlist;\n"
+ + " }\n"
+ + " }\n"
+ + "}");
+ // Cannot use round-trip assertion because 3-level representation is used
+ testParquetToAvroConversion(
+ NEW_BEHAVIOR,
+ schema,
+ "message UnknownTwoLevelListInList {\n" + " optional group listOfLists (LIST) {\n"
+ + " repeated group mylist (LIST) {\n"
+ + " repeated int32 innerlist;\n"
+ + " }\n"
+ + " }\n"
+ + "}");
+ }
+
+ @Test
+ public void testParquetMapWithoutMapKeyValueAnnotation() throws Exception {
+ Schema schema = Schema.createRecord("myrecord", null, null, false);
+ Schema map = Schema.createMap(Schema.create(INT));
+ schema.setFields(Collections.singletonList(new Schema.Field("mymap", map, null, null)));
+ String parquetSchema = "message myrecord {\n" + " required group mymap (MAP) {\n"
+ + " repeated group map {\n"
+ + " required binary key (UTF8);\n"
+ + " required int32 value;\n"
+ + " }\n"
+ + " }\n"
+ + "}\n";
+
+ testParquetToAvroConversion(schema, parquetSchema);
+ testParquetToAvroConversion(NEW_BEHAVIOR, schema, parquetSchema);
+ }
+
+ @Test
+ public void testDecimalBytesType() throws Exception {
+ Schema schema = Schema.createRecord("myrecord", null, null, false);
+ Schema decimal = LogicalTypes.decimal(9, 2).addToSchema(Schema.create(Schema.Type.BYTES));
+ schema.setFields(Collections.singletonList(new Schema.Field("dec", decimal, null, null)));
+
+ testRoundTripConversion(schema, "message myrecord {\n" + " required binary dec (DECIMAL(9,2));\n" + "}\n");
+ }
+
+ @Test
+ public void testDecimalFixedType() throws Exception {
+ Schema schema = Schema.createRecord("myrecord", null, null, false);
+ Schema decimal = LogicalTypes.decimal(9, 2).addToSchema(Schema.createFixed("dec", null, null, 8));
+ schema.setFields(Collections.singletonList(new Schema.Field("dec", decimal, null, null)));
+
+ testRoundTripConversion(
+ schema, "message myrecord {\n" + " required fixed_len_byte_array(8) dec (DECIMAL(9,2));\n" + "}\n");
+ }
+
+ @Test
+ public void testDecimalIntegerType() throws Exception {
+ Schema expected = Schema.createRecord(
+ "myrecord", null, null, false, Arrays.asList(new Schema.Field("dec", Schema.create(INT), null, null)));
+
+ // the decimal portion is lost because it isn't valid in Avro
+ testParquetToAvroConversion(
+ expected, "message myrecord {\n" + " required int32 dec (DECIMAL(9,2));\n" + "}\n");
+ }
+
+ @Test
+ public void testDecimalLongType() throws Exception {
+ Schema expected = Schema.createRecord(
+ "myrecord", null, null, false, Arrays.asList(new Schema.Field("dec", Schema.create(LONG), null, null)));
+
+ // the decimal portion is lost because it isn't valid in Avro
+ testParquetToAvroConversion(
+ expected, "message myrecord {\n" + " required int64 dec (DECIMAL(9,2));\n" + "}\n");
+ }
+
+ @Test
+ public void testParquetInt96AsFixed12AvroType() throws Exception {
+ Configuration enableInt96ReadingConfig = new Configuration();
+ enableInt96ReadingConfig.setBoolean(AvroReadSupport.READ_INT96_AS_FIXED, true);
+
+ Schema schema = Schema.createRecord("myrecord", null, null, false);
+ Schema int96schema = Schema.createFixed("INT96", "INT96 represented as byte[12]", null, 12);
+ schema.setFields(Collections.singletonList(
+ new Schema.Field("int96_field", int96schema, null, null)));
+
+ testParquetToAvroConversion(enableInt96ReadingConfig, schema, "message myrecord {\n"
+ + " required int96 int96_field;\n"
+ + "}\n");
+ }
+
+ @Test
+ public void testParquetInt96DefaultFail() throws Exception {
+ Schema schema = Schema.createRecord("myrecord", null, null, false);
+
+ MessageType parquetSchemaWithInt96 =
+ MessageTypeParser.parseMessageType("message myrecord {\n required int96 int96_field;\n}\n");
+
+ assertThrows(
+ "INT96 is deprecated. As interim enable READ_INT96_AS_FIXED flag to read as byte array.",
+ IllegalArgumentException.class,
+ () -> getAvroSchemaConverter(new Configuration()).convert(parquetSchemaWithInt96));
+ }
+
+ @Test
+ public void testDateType() throws Exception {
+ Schema date = LogicalTypes.date().addToSchema(Schema.create(INT));
+ Schema expected = Schema.createRecord(
+ "myrecord", null, null, false, Arrays.asList(new Schema.Field("date", date, null, null)));
+
+ testRoundTripConversion(expected, "message myrecord {\n" + " required int32 date (DATE);\n" + "}\n");
+
+ for (PrimitiveTypeName primitive :
+ new PrimitiveTypeName[] {INT64, INT96, FLOAT, DOUBLE, BOOLEAN, BINARY, FIXED_LEN_BYTE_ARRAY}) {
+ final PrimitiveType type;
+ if (primitive == FIXED_LEN_BYTE_ARRAY) {
+ type = new PrimitiveType(REQUIRED, primitive, 12, "test", DATE);
+ } else {
+ type = new PrimitiveType(REQUIRED, primitive, "test", DATE);
+ }
+
+ assertThrows(
+ "Should not allow TIME_MICROS with " + primitive,
+ IllegalArgumentException.class,
+ () -> getAvroSchemaConverter(new Configuration()).convert(message(type)));
+ }
+ }
+
+ @Test
+ public void testTimeMillisType() throws Exception {
+ Schema date = LogicalTypes.timeMillis().addToSchema(Schema.create(INT));
+ Schema expected = Schema.createRecord(
+ "myrecord", null, null, false, Arrays.asList(new Schema.Field("time", date, null, null)));
+
+ testRoundTripConversion(
+ expected, "message myrecord {\n" + " required int32 time (TIME(MILLIS,true));\n" + "}\n");
+
+ for (PrimitiveTypeName primitive :
+ new PrimitiveTypeName[] {INT64, INT96, FLOAT, DOUBLE, BOOLEAN, BINARY, FIXED_LEN_BYTE_ARRAY}) {
+ final PrimitiveType type;
+ if (primitive == FIXED_LEN_BYTE_ARRAY) {
+ type = new PrimitiveType(REQUIRED, primitive, 12, "test", TIME_MILLIS);
+ } else {
+ type = new PrimitiveType(REQUIRED, primitive, "test", TIME_MILLIS);
+ }
+
+ assertThrows(
+ "Should not allow TIME_MICROS with " + primitive,
+ IllegalArgumentException.class,
+ () -> getAvroSchemaConverter(new Configuration()).convert(message(type)));
+ }
+ }
+
+ @Test
+ public void testTimeMicrosType() throws Exception {
+ Schema date = LogicalTypes.timeMicros().addToSchema(Schema.create(LONG));
+ Schema expected = Schema.createRecord(
+ "myrecord", null, null, false, Arrays.asList(new Schema.Field("time", date, null, null)));
+
+ testRoundTripConversion(
+ expected, "message myrecord {\n" + " required int64 time (TIME(MICROS,true));\n" + "}\n");
+
+ for (PrimitiveTypeName primitive :
+ new PrimitiveTypeName[] {INT32, INT96, FLOAT, DOUBLE, BOOLEAN, BINARY, FIXED_LEN_BYTE_ARRAY}) {
+ final PrimitiveType type;
+ if (primitive == FIXED_LEN_BYTE_ARRAY) {
+ type = new PrimitiveType(REQUIRED, primitive, 12, "test", TIME_MICROS);
+ } else {
+ type = new PrimitiveType(REQUIRED, primitive, "test", TIME_MICROS);
+ }
+
+ assertThrows(
+ "Should not allow TIME_MICROS with " + primitive,
+ IllegalArgumentException.class,
+ () -> getAvroSchemaConverter(new Configuration()).convert(message(type)));
+ }
+ }
+
+ @Test
+ public void testTimestampMillisType() throws Exception {
+ Schema date = LogicalTypes.timestampMillis().addToSchema(Schema.create(LONG));
+ Schema expected = Schema.createRecord(
+ "myrecord", null, null, false, Arrays.asList(new Schema.Field("timestamp", date, null, null)));
+
+ testRoundTripConversion(
+ expected, "message myrecord {\n" + " required int64 timestamp (TIMESTAMP(MILLIS,true));\n" + "}\n");
+
+ final Schema converted = getAvroSchemaConverter(new Configuration())
+ .convert(Types.buildMessage()
+ .addField(Types.primitive(INT64, Type.Repetition.REQUIRED)
+ .as(LogicalTypeAnnotation.timestampType(
+ false, LogicalTypeAnnotation.TimeUnit.MILLIS))
+ .length(1)
+ .named("timestamp_type"))
+ .named("TestAvro"));
+ assertEquals(
+ "local-timestamp-millis",
+ converted
+ .getField("timestamp_type")
+ .schema()
+ .getLogicalType()
+ .getName());
+
+ for (PrimitiveTypeName primitive :
+ new PrimitiveTypeName[] {INT32, INT96, FLOAT, DOUBLE, BOOLEAN, BINARY, FIXED_LEN_BYTE_ARRAY}) {
+ final PrimitiveType type;
+ if (primitive == FIXED_LEN_BYTE_ARRAY) {
+ type = new PrimitiveType(REQUIRED, primitive, 12, "test", TIMESTAMP_MILLIS);
+ } else {
+ type = new PrimitiveType(REQUIRED, primitive, "test", TIMESTAMP_MILLIS);
+ }
+
+ assertThrows(
+ "Should not allow TIMESTAMP_MILLIS with " + primitive,
+ IllegalArgumentException.class,
+ () -> getAvroSchemaConverter(new Configuration()).convert(message(type)));
+ }
+ }
+
+ @Test
+ public void testLocalTimestampMillisType() throws Exception {
+ Schema date = LogicalTypes.localTimestampMillis().addToSchema(Schema.create(LONG));
+ Schema expected = Schema.createRecord(
+ "myrecord", null, null, false, Arrays.asList(new Schema.Field("timestamp", date, null, null)));
+
+ testRoundTripConversion(
+ expected, "message myrecord {\n" + " required int64 timestamp (TIMESTAMP(MILLIS,false));\n" + "}\n");
+
+ for (PrimitiveTypeName primitive :
+ new PrimitiveTypeName[] {INT32, INT96, FLOAT, DOUBLE, BOOLEAN, BINARY, FIXED_LEN_BYTE_ARRAY}) {
+ final PrimitiveType type;
+ if (primitive == FIXED_LEN_BYTE_ARRAY) {
+ type = new PrimitiveType(REQUIRED, primitive, 12, "test", TIMESTAMP_MILLIS);
+ } else {
+ type = new PrimitiveType(REQUIRED, primitive, "test", TIMESTAMP_MILLIS);
+ }
+
+ assertThrows(
+ "Should not allow TIMESTAMP_MILLIS with " + primitive,
+ IllegalArgumentException.class,
+ () -> getAvroSchemaConverter(new Configuration()).convert(message(type)));
+ }
+ }
+
+ @Test
+ public void testTimestampMicrosType() throws Exception {
+ Schema date = LogicalTypes.timestampMicros().addToSchema(Schema.create(LONG));
+ Schema expected = Schema.createRecord(
+ "myrecord", null, null, false, Arrays.asList(new Schema.Field("timestamp", date, null, null)));
+
+ testRoundTripConversion(
+ expected, "message myrecord {\n" + " required int64 timestamp (TIMESTAMP(MICROS,true));\n" + "}\n");
+
+ for (PrimitiveTypeName primitive :
+ new PrimitiveTypeName[] {INT32, INT96, FLOAT, DOUBLE, BOOLEAN, BINARY, FIXED_LEN_BYTE_ARRAY}) {
+ final PrimitiveType type;
+ if (primitive == FIXED_LEN_BYTE_ARRAY) {
+ type = new PrimitiveType(REQUIRED, primitive, 12, "test", TIMESTAMP_MICROS);
+ } else {
+ type = new PrimitiveType(REQUIRED, primitive, "test", TIMESTAMP_MICROS);
+ }
+
+ assertThrows(
+ "Should not allow TIMESTAMP_MICROS with " + primitive,
+ IllegalArgumentException.class,
+ () -> getAvroSchemaConverter(new Configuration()).convert(message(type)));
+ }
+
+ final Schema converted = getAvroSchemaConverter(new Configuration())
+ .convert(Types.buildMessage()
+ .addField(Types.primitive(INT64, Type.Repetition.REQUIRED)
+ .as(LogicalTypeAnnotation.timestampType(
+ false, LogicalTypeAnnotation.TimeUnit.MICROS))
+ .length(1)
+ .named("timestamp_type"))
+ .named("TestAvro"));
+
+ assertEquals(
+ "local-timestamp-micros",
+ converted
+ .getField("timestamp_type")
+ .schema()
+ .getLogicalType()
+ .getName());
+ }
+
+ @Test
+ public void testLocalTimestampMicrosType() throws Exception {
+ Schema date = LogicalTypes.localTimestampMicros().addToSchema(Schema.create(LONG));
+ Schema expected = Schema.createRecord(
+ "myrecord", null, null, false, Arrays.asList(new Schema.Field("timestamp", date, null, null)));
+
+ testRoundTripConversion(
+ expected, "message myrecord {\n" + " required int64 timestamp (TIMESTAMP(MICROS,false));\n" + "}\n");
+
+ for (PrimitiveTypeName primitive :
+ new PrimitiveTypeName[] {INT32, INT96, FLOAT, DOUBLE, BOOLEAN, BINARY, FIXED_LEN_BYTE_ARRAY}) {
+ final PrimitiveType type;
+ if (primitive == FIXED_LEN_BYTE_ARRAY) {
+ type = new PrimitiveType(REQUIRED, primitive, 12, "test", TIMESTAMP_MICROS);
+ } else {
+ type = new PrimitiveType(REQUIRED, primitive, "test", TIMESTAMP_MICROS);
+ }
+
+ assertThrows(
+ "Should not allow TIMESTAMP_MICROS with " + primitive,
+ IllegalArgumentException.class,
+ () -> getAvroSchemaConverter(new Configuration()).convert(message(type)));
+ }
+ }
+
+ @Test
+ public void testReuseNameInNestedStructure() throws Exception {
+ Schema innerA1 = record("a1", "a12", field("a4", primitive(Schema.Type.FLOAT)));
+
+ Schema outerA1 = record("a1", field("a2", primitive(Schema.Type.FLOAT)), optionalField("a1", innerA1));
+ Schema schema = record("Message", optionalField("a1", outerA1));
+
+ String parquetSchema = "message Message {\n"
+ + " optional group a1 {\n"
+ + " required float a2;\n"
+ + " optional group a1 {\n"
+ + " required float a4;\n"
+ + " }\n"
+ + " }\n"
+ + "}\n";
+
+ testParquetToAvroConversion(schema, parquetSchema);
+ testParquetToAvroConversion(NEW_BEHAVIOR, schema, parquetSchema);
+ }
+
+ @Test
+ public void testReuseNameInNestedStructureAtSameLevel() throws Exception {
+ Schema a2 = record("a2", field("a4", primitive(Schema.Type.FLOAT)));
+ Schema a22 = record(
+ "a2", "a22", field("a4", primitive(Schema.Type.FLOAT)), field("a5", primitive(Schema.Type.FLOAT)));
+
+ Schema a1 = record("a1", optionalField("a2", a2));
+ Schema a3 = record("a3", optionalField("a2", a22));
+
+ Schema schema = record("Message", optionalField("a1", a1), optionalField("a3", a3));
+
+ String parquetSchema = "message Message {\n"
+ + " optional group a1 {\n"
+ + " optional group a2 {\n"
+ + " required float a4;\n"
+ + " }\n"
+ + " }\n"
+ + " optional group a3 {\n"
+ + " optional group a2 {\n"
+ + " required float a4;\n"
+ + " required float a5;\n"
+ + " }\n"
+ + " }\n"
+ + "}\n";
+
+ testParquetToAvroConversion(schema, parquetSchema);
+ testParquetToAvroConversion(NEW_BEHAVIOR, schema, parquetSchema);
+ }
+
+ @Test
+ public void testUUIDType() throws Exception {
+ Schema fromAvro = Schema.createRecord(
+ "myrecord",
+ null,
+ null,
+ false,
+ Arrays.asList(
+ new Schema.Field("uuid", LogicalTypes.uuid().addToSchema(Schema.create(STRING)), null, null)));
+ String parquet = "message myrecord {\n" + " required binary uuid (STRING);\n" + "}\n";
+ Schema toAvro = Schema.createRecord(
+ "myrecord",
+ null,
+ null,
+ false,
+ Arrays.asList(new Schema.Field("uuid", Schema.create(STRING), null, null)));
+
+ testAvroToParquetConversion(fromAvro, parquet);
+ testParquetToAvroConversion(toAvro, parquet);
+
+ assertEquals(
+ COMPATIBLE, checkReaderWriterCompatibility(fromAvro, toAvro).getType());
+ }
+
+ @Test
+ public void testUUIDTypeWithParquetUUID() throws Exception {
+ Schema uuid = LogicalTypes.uuid().addToSchema(Schema.create(STRING));
+ Schema expected = Schema.createRecord(
+ "myrecord", null, null, false, Arrays.asList(new Schema.Field("uuid", uuid, null, null)));
+
+ testRoundTripConversion(
+ conf(AvroWriteSupport.WRITE_PARQUET_UUID, true),
+ expected,
+ "message myrecord {\n" + " required fixed_len_byte_array(16) uuid (UUID);\n" + "}\n");
+ }
+
+ @Test
+ public void testAvroFixed12AsParquetInt96Type() throws Exception {
+ Schema schema = getSchemaFromResource(TestAvroSchemaConverter.class, "/parquet-java/fixedToInt96.avsc");
+
+ Configuration conf = new Configuration();
+ conf.setStrings(
+ "parquet.avro.writeFixedAsInt96",
+ "int96",
+ "mynestedrecord.int96inrecord",
+ "mynestedrecord.myarrayofoptional",
+ "mynestedrecord.mymap");
+ testAvroToParquetConversion(
+ conf,
+ schema,
+ "message org.apache.parquet.avro.fixedToInt96 {\n"
+ + " required int96 int96;\n"
+ + " required fixed_len_byte_array(12) notanint96;\n"
+ + " required group mynestedrecord {\n"
+ + " required int96 int96inrecord;\n"
+ + " required group myarrayofoptional (LIST) {\n"
+ + " repeated int96 array;\n"
+ + " }\n"
+ + " required group mymap (MAP) {\n"
+ + " repeated group key_value (MAP_KEY_VALUE) {\n"
+ + " required binary key (STRING);\n"
+ + " required int96 value;\n"
+ + " }\n"
+ + " }\n"
+ + " }\n"
+ + " required fixed_len_byte_array(1) onebytefixed;\n"
+ + "}");
+
+ conf.setStrings("parquet.avro.writeFixedAsInt96", "onebytefixed");
+ assertThrows(
+ "Exception should be thrown for fixed types to be converted to INT96 where the size is not 12 bytes",
+ IllegalArgumentException.class,
+ () -> getAvroSchemaConverter(conf).convert(schema));
+ }
+
+ public static Schema optional(Schema original) {
+ return Schema.createUnion(Arrays.asList(Schema.create(Schema.Type.NULL), original));
+ }
+
+ public static MessageType message(PrimitiveType primitive) {
+ return Types.buildMessage().addField(primitive).named("myrecord");
+ }
+
+ /**
+ * A convenience method to avoid a large number of @Test(expected=...) tests
+ *
+ * @param message A String message to describe this assertion
+ * @param expected An Exception class that the Runnable should throw
+ * @param runnable A Runnable that is expected to throw the exception
+ */
+ public static void assertThrows(String message, Class extends Exception> expected, Runnable runnable) {
+ try {
+ runnable.run();
+ fail("No exception was thrown (" + message + "), expected: " + expected.getName());
+ } catch (Exception actual) {
+ try {
+ assertEquals(expected, actual.getClass(), message);
+ } catch (AssertionError e) {
+ e.addSuppressed(actual);
+ throw e;
+ }
+ }
+ }
+
+ public static Schema record(String name, String namespace, Schema.Field... fields) {
+ Schema record = Schema.createRecord(name, null, namespace, false);
+ record.setFields(Arrays.asList(fields));
+ return record;
+ }
+
+ public static Schema record(String name, Schema.Field... fields) {
+ return record(name, null, fields);
+ }
+
+ public static Schema.Field field(String name, Schema schema) {
+ return new Schema.Field(name, schema, null, null);
+ }
+
+ public static Schema.Field optionalField(String name, Schema schema) {
+ return new Schema.Field(name, optional(schema), null, JsonProperties.NULL_VALUE);
+ }
+
+ public static Schema array(Schema element) {
+ return Schema.createArray(element);
+ }
+
+ public static Schema primitive(Schema.Type type) {
+ return Schema.create(type);
+ }
+
+ public static Configuration conf(String name, boolean value) {
+ Configuration conf = new Configuration(false);
+ conf.setBoolean(name, value);
+ return conf;
+ }
+
+}
\ No newline at end of file
diff --git a/hudi-common/src/parquet/test/org/apache/parquet/schema/TestSchemaRepair.java b/hudi-common/src/parquet/test/org/apache/parquet/schema/TestSchemaRepair.java
new file mode 100644
index 0000000000000..b31d37c835dbd
--- /dev/null
+++ b/hudi-common/src/parquet/test/org/apache/parquet/schema/TestSchemaRepair.java
@@ -0,0 +1,600 @@
+/*
+ * 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.parquet.schema;
+
+import org.apache.hudi.common.util.Option;
+import org.junit.jupiter.api.Test;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotSame;
+import static org.junit.jupiter.api.Assertions.assertSame;
+
+/**
+ * Tests {@link SchemaRepair}.
+ */
+public class TestSchemaRepair {
+
+ @Test
+ public void testNoRepairNeededIdenticalSchemas() {
+ MessageType requestedSchema = new MessageType("TestSchema",
+ Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED)
+ .named("timestamp")
+ );
+ MessageType tableSchema = new MessageType("TestSchema",
+ Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED)
+ .named("timestamp")
+ );
+
+ MessageType result = SchemaRepair.repairLogicalTypes(requestedSchema, tableSchema);
+
+ assertSame(requestedSchema, result, "When schemas are identical, should return same instance");
+ }
+
+ @Test
+ public void testNoRepairNeededDifferentPrimitiveTypes() {
+ MessageType requestedSchema = new MessageType("TestSchema",
+ Types.primitive(PrimitiveType.PrimitiveTypeName.BINARY, Type.Repetition.REQUIRED)
+ .as(LogicalTypeAnnotation.stringType())
+ .named("name")
+ );
+ MessageType tableSchema = new MessageType("TestSchema",
+ Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.REQUIRED)
+ .named("id")
+ );
+
+ MessageType result = SchemaRepair.repairLogicalTypes(requestedSchema, tableSchema);
+
+ assertSame(requestedSchema, result, "When field names differ, should return original schema");
+ }
+
+ @Test
+ public void testRepairLongWithoutLogicalTypeToLocalTimestampMillis() {
+ MessageType requestedSchema = new MessageType("TestSchema",
+ Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED)
+ .named("timestamp")
+ );
+ MessageType tableSchema = new MessageType("TestSchema",
+ Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED)
+ .as(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MILLIS))
+ .named("timestamp")
+ );
+
+ MessageType result = SchemaRepair.repairLogicalTypes(requestedSchema, tableSchema);
+
+ assertNotSame(requestedSchema, result, "Should create a new schema with logical type");
+ PrimitiveType timestampField = result.getType("timestamp").asPrimitiveType();
+ assertEquals(PrimitiveType.PrimitiveTypeName.INT64, timestampField.getPrimitiveTypeName());
+ assertEquals(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MILLIS),
+ timestampField.getLogicalTypeAnnotation());
+ }
+
+ @Test
+ public void testRepairLongWithoutLogicalTypeToLocalTimestampMicros() {
+ MessageType requestedSchema = new MessageType("TestSchema",
+ Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED)
+ .named("timestamp")
+ );
+ MessageType tableSchema = new MessageType("TestSchema",
+ Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED)
+ .as(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MICROS))
+ .named("timestamp")
+ );
+
+ MessageType result = SchemaRepair.repairLogicalTypes(requestedSchema, tableSchema);
+
+ assertNotSame(requestedSchema, result, "Should create a new schema with logical type");
+ PrimitiveType timestampField = result.getType("timestamp").asPrimitiveType();
+ assertEquals(PrimitiveType.PrimitiveTypeName.INT64, timestampField.getPrimitiveTypeName());
+ assertEquals(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MICROS),
+ timestampField.getLogicalTypeAnnotation());
+ }
+
+ @Test
+ public void testRepairTimestampMicrosToTimestampMillis() {
+ MessageType requestedSchema = new MessageType("TestSchema",
+ Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED)
+ .as(LogicalTypeAnnotation.timestampType(true, LogicalTypeAnnotation.TimeUnit.MICROS))
+ .named("timestamp")
+ );
+ MessageType tableSchema = new MessageType("TestSchema",
+ Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED)
+ .as(LogicalTypeAnnotation.timestampType(true, LogicalTypeAnnotation.TimeUnit.MILLIS))
+ .named("timestamp")
+ );
+
+ MessageType result = SchemaRepair.repairLogicalTypes(requestedSchema, tableSchema);
+
+ assertNotSame(requestedSchema, result, "Should create a new schema with timestamp-millis");
+ PrimitiveType timestampField = result.getType("timestamp").asPrimitiveType();
+ assertEquals(PrimitiveType.PrimitiveTypeName.INT64, timestampField.getPrimitiveTypeName());
+ assertEquals(LogicalTypeAnnotation.timestampType(true, LogicalTypeAnnotation.TimeUnit.MILLIS),
+ timestampField.getLogicalTypeAnnotation());
+ }
+
+ @Test
+ public void testNoRepairNeededTimestampMillisToTimestampMicros() {
+ // This direction should NOT trigger repair
+ MessageType requestedSchema = new MessageType("TestSchema",
+ Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED)
+ .as(LogicalTypeAnnotation.timestampType(true, LogicalTypeAnnotation.TimeUnit.MILLIS))
+ .named("timestamp")
+ );
+ MessageType tableSchema = new MessageType("TestSchema",
+ Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED)
+ .as(LogicalTypeAnnotation.timestampType(true, LogicalTypeAnnotation.TimeUnit.MICROS))
+ .named("timestamp")
+ );
+
+ MessageType result = SchemaRepair.repairLogicalTypes(requestedSchema, tableSchema);
+
+ assertSame(requestedSchema, result, "Should not repair timestamp-millis to timestamp-micros");
+ }
+
+ @Test
+ public void testNoRepairNeededNonLongTypes() {
+ MessageType requestedSchema = new MessageType("TestSchema",
+ Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.REQUIRED)
+ .named("id")
+ );
+ MessageType tableSchema = new MessageType("TestSchema",
+ Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.REQUIRED)
+ .as(LogicalTypeAnnotation.dateType())
+ .named("id")
+ );
+
+ MessageType result = SchemaRepair.repairLogicalTypes(requestedSchema, tableSchema);
+
+ assertSame(requestedSchema, result, "Should not repair non-LONG types");
+ }
+
+ @Test
+ public void testRepairRecordSingleField() {
+ MessageType requestedSchema = new MessageType("TestRecord",
+ Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED)
+ .named("timestamp")
+ );
+
+ MessageType tableSchema = new MessageType("TestRecord",
+ Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED)
+ .as(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MILLIS))
+ .named("timestamp")
+ );
+
+ MessageType result = SchemaRepair.repairLogicalTypes(requestedSchema, tableSchema);
+
+ assertNotSame(requestedSchema, result, "Should create new record schema");
+ assertEquals(1, result.getFields().size());
+
+ PrimitiveType field = result.getType("timestamp").asPrimitiveType();
+ assertEquals(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MILLIS),
+ field.getLogicalTypeAnnotation());
+ }
+
+ @Test
+ public void testRepairRecordMultipleFieldsOnlyOneNeedsRepair() {
+ MessageType requestedSchema = new MessageType("TestRecord",
+ Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.REQUIRED)
+ .named("id"),
+ Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED)
+ .named("timestamp"),
+ Types.primitive(PrimitiveType.PrimitiveTypeName.BINARY, Type.Repetition.REQUIRED)
+ .as(LogicalTypeAnnotation.stringType())
+ .named("name")
+ );
+
+ MessageType tableSchema = new MessageType("TestRecord",
+ Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.REQUIRED)
+ .named("id"),
+ Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED)
+ .as(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MICROS))
+ .named("timestamp"),
+ Types.primitive(PrimitiveType.PrimitiveTypeName.BINARY, Type.Repetition.REQUIRED)
+ .as(LogicalTypeAnnotation.stringType())
+ .named("name")
+ );
+
+ MessageType result = SchemaRepair.repairLogicalTypes(requestedSchema, tableSchema);
+
+ assertNotSame(requestedSchema, result, "Should create new record schema");
+ assertEquals(3, result.getFields().size());
+
+ // Verify id field unchanged - should be same type instance
+ assertSame(requestedSchema.getType("id"), result.getType("id"));
+
+ // Verify timestamp field repaired
+ PrimitiveType timestampField = result.getType("timestamp").asPrimitiveType();
+ assertEquals(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MICROS),
+ timestampField.getLogicalTypeAnnotation());
+
+ // Verify name field unchanged - should be same type instance
+ assertSame(requestedSchema.getType("name"), result.getType("name"));
+ }
+
+ @Test
+ public void testRepairRecordNestedRecord() {
+ GroupType nestedRequestedSchema = new GroupType(Type.Repetition.REQUIRED, "nested",
+ Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED)
+ .named("timestamp")
+ );
+
+ GroupType nestedTableSchema = new GroupType(Type.Repetition.REQUIRED, "nested",
+ Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED)
+ .as(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MILLIS))
+ .named("timestamp")
+ );
+
+ MessageType requestedSchema = new MessageType("OuterRecord",
+ Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.REQUIRED)
+ .named("id"),
+ nestedRequestedSchema
+ );
+
+ MessageType tableSchema = new MessageType("OuterRecord",
+ Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.REQUIRED)
+ .named("id"),
+ nestedTableSchema
+ );
+
+ MessageType result = SchemaRepair.repairLogicalTypes(requestedSchema, tableSchema);
+
+ assertNotSame(requestedSchema, result, "Should create new schema for nested record");
+
+ // Verify id field unchanged - should be same type instance
+ assertSame(requestedSchema.getType("id"), result.getType("id"));
+
+ // Verify nested record was repaired
+ GroupType nestedResult = result.getType("nested").asGroupType();
+ PrimitiveType nestedTimestamp = nestedResult.getType("timestamp").asPrimitiveType();
+ assertEquals(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MILLIS),
+ nestedTimestamp.getLogicalTypeAnnotation());
+ }
+
+ @Test
+ public void testRepairRecordMissingFieldInTableSchema() {
+ // Requested schema has a field not present in table schema
+ MessageType requestedSchema = new MessageType("TestRecord",
+ Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.REQUIRED)
+ .named("id"),
+ Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED)
+ .named("newField")
+ );
+
+ MessageType tableSchema = new MessageType("TestRecord",
+ Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.REQUIRED)
+ .named("id")
+ );
+
+ MessageType result = SchemaRepair.repairLogicalTypes(requestedSchema, tableSchema);
+
+ // Should return original schema unchanged since newField doesn't exist in table schema
+ assertSame(requestedSchema, result, "Should return original when field missing in table schema");
+ }
+
+ @Test
+ public void testRepairRecordMultipleFieldsMissingInTableSchema() {
+ // Requested schema has multiple fields not present in table schema
+ MessageType requestedSchema = new MessageType("TestRecord",
+ Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.REQUIRED)
+ .named("id"),
+ Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED)
+ .named("newField1"),
+ Types.primitive(PrimitiveType.PrimitiveTypeName.BINARY, Type.Repetition.REQUIRED)
+ .as(LogicalTypeAnnotation.stringType())
+ .named("name"),
+ Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED)
+ .named("newField2")
+ );
+
+ MessageType tableSchema = new MessageType("TestRecord",
+ Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.REQUIRED)
+ .named("id"),
+ Types.primitive(PrimitiveType.PrimitiveTypeName.BINARY, Type.Repetition.REQUIRED)
+ .as(LogicalTypeAnnotation.stringType())
+ .named("name")
+ );
+
+ MessageType result = SchemaRepair.repairLogicalTypes(requestedSchema, tableSchema);
+
+ // Should return original schema unchanged since new fields don't exist in table schema
+ assertSame(requestedSchema, result, "Should return original when multiple fields missing in table schema");
+ }
+
+ @Test
+ public void testRepairRecordMixedMissingAndRepairableFields() {
+ // Requested schema has some fields missing in table, some needing repair, some unchanged
+ MessageType requestedSchema = new MessageType("TestRecord",
+ Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.REQUIRED)
+ .named("id"),
+ Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED)
+ .named("timestamp"),
+ Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED)
+ .named("newField"),
+ Types.primitive(PrimitiveType.PrimitiveTypeName.BINARY, Type.Repetition.REQUIRED)
+ .as(LogicalTypeAnnotation.stringType())
+ .named("name")
+ );
+
+ MessageType tableSchema = new MessageType("TestRecord",
+ Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.REQUIRED)
+ .named("id"),
+ Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED)
+ .as(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MILLIS))
+ .named("timestamp"),
+ Types.primitive(PrimitiveType.PrimitiveTypeName.BINARY, Type.Repetition.REQUIRED)
+ .as(LogicalTypeAnnotation.stringType())
+ .named("name")
+ );
+
+ MessageType result = SchemaRepair.repairLogicalTypes(requestedSchema, tableSchema);
+
+ // Should create new schema with timestamp repaired, but newField preserved from requested
+ assertNotSame(requestedSchema, result, "Should create new schema");
+ assertEquals(4, result.getFields().size());
+
+ // Verify id field unchanged
+ assertSame(requestedSchema.getType("id"), result.getType("id"));
+
+ // Verify timestamp field repaired
+ PrimitiveType timestampField = result.getType("timestamp").asPrimitiveType();
+ assertEquals(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MILLIS),
+ timestampField.getLogicalTypeAnnotation());
+
+ // Verify newField preserved from requested schema (not in table)
+ assertSame(requestedSchema.getType("newField"), result.getType("newField"));
+
+ // Verify name field unchanged
+ assertSame(requestedSchema.getType("name"), result.getType("name"));
+ }
+
+ @Test
+ public void testRepairNestedRecordFieldMissingInTableSchema() {
+ // Requested nested record has a field not present in table's nested record
+ GroupType nestedRequestedSchema = new GroupType(Type.Repetition.REQUIRED, "nested",
+ Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED)
+ .named("timestamp"),
+ Types.primitive(PrimitiveType.PrimitiveTypeName.BINARY, Type.Repetition.REQUIRED)
+ .as(LogicalTypeAnnotation.stringType())
+ .named("extraField")
+ );
+
+ GroupType nestedTableSchema = new GroupType(Type.Repetition.REQUIRED, "nested",
+ Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED)
+ .as(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MILLIS))
+ .named("timestamp")
+ );
+
+ MessageType requestedSchema = new MessageType("OuterRecord",
+ Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.REQUIRED)
+ .named("id"),
+ nestedRequestedSchema
+ );
+
+ MessageType tableSchema = new MessageType("OuterRecord",
+ Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.REQUIRED)
+ .named("id"),
+ nestedTableSchema
+ );
+
+ MessageType result = SchemaRepair.repairLogicalTypes(requestedSchema, tableSchema);
+
+ assertNotSame(requestedSchema, result, "Should create new schema");
+
+ // Verify id field unchanged
+ assertSame(requestedSchema.getType("id"), result.getType("id"));
+
+ // Verify nested record was repaired but still has extraField
+ GroupType nestedResult = result.getType("nested").asGroupType();
+ assertEquals(2, nestedResult.getFieldCount());
+
+ // Timestamp should be repaired
+ PrimitiveType timestampField = nestedResult.getType("timestamp").asPrimitiveType();
+ assertEquals(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MILLIS),
+ timestampField.getLogicalTypeAnnotation());
+
+ // extraField should be preserved from requested schema
+ assertSame(nestedRequestedSchema.getType("extraField"), nestedResult.getType("extraField"));
+ }
+
+ @Test
+ public void testRepairRecordWholeNestedRecordMissingInTableSchema() {
+ // Requested schema has a nested record field that doesn't exist in table schema
+ GroupType nestedRequestedSchema = new GroupType(Type.Repetition.REQUIRED, "newNested",
+ Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED)
+ .named("timestamp")
+ );
+
+ MessageType requestedSchema = new MessageType("OuterRecord",
+ Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.REQUIRED)
+ .named("id"),
+ nestedRequestedSchema
+ );
+
+ MessageType tableSchema = new MessageType("OuterRecord",
+ Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.REQUIRED)
+ .named("id")
+ );
+
+ MessageType result = SchemaRepair.repairLogicalTypes(requestedSchema, tableSchema);
+
+ // Should return original schema unchanged since newNested field doesn't exist in table
+ assertSame(requestedSchema, result, "Should return original when nested field missing in table schema");
+ }
+
+ @Test
+ public void testEdgeCaseEmptyRecord() {
+ MessageType requestedSchema = new MessageType("EmptyRecord");
+ MessageType tableSchema = new MessageType("EmptyRecord");
+
+ MessageType result = SchemaRepair.repairLogicalTypes(requestedSchema, tableSchema);
+
+ assertSame(requestedSchema, result, "Empty records should return same instance");
+ }
+
+ @Test
+ public void testRepairRecordFirstFieldChanged() {
+ // Test the optimization path where the first field needs repair
+ MessageType requestedSchema = new MessageType("TestRecord",
+ Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED)
+ .named("timestamp1"),
+ Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED)
+ .named("timestamp2")
+ );
+
+ MessageType tableSchema = new MessageType("TestRecord",
+ Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED)
+ .as(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MILLIS))
+ .named("timestamp1"),
+ Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED)
+ .as(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MICROS))
+ .named("timestamp2")
+ );
+
+ MessageType result = SchemaRepair.repairLogicalTypes(requestedSchema, tableSchema);
+
+ assertNotSame(requestedSchema, result);
+ PrimitiveType timestamp1 = result.getType("timestamp1").asPrimitiveType();
+ assertEquals(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MILLIS),
+ timestamp1.getLogicalTypeAnnotation());
+ PrimitiveType timestamp2 = result.getType("timestamp2").asPrimitiveType();
+ assertEquals(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MICROS),
+ timestamp2.getLogicalTypeAnnotation());
+ }
+
+ @Test
+ public void testRepairRecordLastFieldChanged() {
+ // Test the optimization path where only the last field needs repair
+ MessageType requestedSchema = new MessageType("TestRecord",
+ Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.REQUIRED)
+ .named("id"),
+ Types.primitive(PrimitiveType.PrimitiveTypeName.BINARY, Type.Repetition.REQUIRED)
+ .as(LogicalTypeAnnotation.stringType())
+ .named("name"),
+ Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED)
+ .named("timestamp")
+ );
+
+ MessageType tableSchema = new MessageType("TestRecord",
+ Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.REQUIRED)
+ .named("id"),
+ Types.primitive(PrimitiveType.PrimitiveTypeName.BINARY, Type.Repetition.REQUIRED)
+ .as(LogicalTypeAnnotation.stringType())
+ .named("name"),
+ Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED)
+ .as(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MILLIS))
+ .named("timestamp")
+ );
+
+ MessageType result = SchemaRepair.repairLogicalTypes(requestedSchema, tableSchema);
+
+ assertNotSame(requestedSchema, result);
+ // Verify id and name fields unchanged - should be same type instances
+ assertSame(requestedSchema.getType("id"), result.getType("id"));
+ assertSame(requestedSchema.getType("name"), result.getType("name"));
+ // Verify timestamp field repaired
+ PrimitiveType timestampField = result.getType("timestamp").asPrimitiveType();
+ assertEquals(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MILLIS),
+ timestampField.getLogicalTypeAnnotation());
+ }
+
+ @Test
+ public void testRepairLogicalTypesWithOptionEmpty() {
+ MessageType requestedSchema = new MessageType("TestSchema",
+ Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED)
+ .named("timestamp")
+ );
+
+ MessageType result = SchemaRepair.repairLogicalTypes(requestedSchema, Option.empty());
+
+ assertSame(requestedSchema, result, "Should return original when Option is empty");
+ }
+
+ @Test
+ public void testRepairLogicalTypesWithOptionPresent() {
+ MessageType requestedSchema = new MessageType("TestSchema",
+ Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED)
+ .named("timestamp")
+ );
+ MessageType tableSchema = new MessageType("TestSchema",
+ Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED)
+ .as(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MILLIS))
+ .named("timestamp")
+ );
+
+ MessageType result = SchemaRepair.repairLogicalTypes(requestedSchema, Option.of(tableSchema));
+
+ assertNotSame(requestedSchema, result, "Should repair when Option is present");
+ PrimitiveType timestampField = result.getType("timestamp").asPrimitiveType();
+ assertEquals(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MILLIS),
+ timestampField.getLogicalTypeAnnotation());
+ }
+
+ @Test
+ public void testRepairOptionalFieldRepetition() {
+ // Test that repair preserves the requested field's repetition (OPTIONAL vs REQUIRED)
+ MessageType requestedSchema = new MessageType("TestSchema",
+ Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.OPTIONAL)
+ .named("timestamp")
+ );
+ MessageType tableSchema = new MessageType("TestSchema",
+ Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED)
+ .as(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MILLIS))
+ .named("timestamp")
+ );
+
+ MessageType result = SchemaRepair.repairLogicalTypes(requestedSchema, tableSchema);
+
+ assertNotSame(requestedSchema, result);
+ PrimitiveType timestampField = result.getType("timestamp").asPrimitiveType();
+ assertEquals(Type.Repetition.OPTIONAL, timestampField.getRepetition(),
+ "Should preserve requested field's repetition");
+ assertEquals(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MILLIS),
+ timestampField.getLogicalTypeAnnotation());
+ }
+
+ @Test
+ public void testRepairNestedGroupPreservesLogicalType() {
+ // Test that repair preserves the group's logical type annotation
+ GroupType nestedRequestedSchema = new GroupType(Type.Repetition.REQUIRED, "nested",
+ LogicalTypeAnnotation.listType(),
+ Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED)
+ .named("timestamp")
+ );
+
+ GroupType nestedTableSchema = new GroupType(Type.Repetition.REQUIRED, "nested",
+ LogicalTypeAnnotation.listType(),
+ Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED)
+ .as(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MILLIS))
+ .named("timestamp")
+ );
+
+ MessageType requestedSchema = new MessageType("OuterRecord", nestedRequestedSchema);
+ MessageType tableSchema = new MessageType("OuterRecord", nestedTableSchema);
+
+ MessageType result = SchemaRepair.repairLogicalTypes(requestedSchema, tableSchema);
+
+ assertNotSame(requestedSchema, result);
+ GroupType nestedResult = result.getType("nested").asGroupType();
+ assertEquals(LogicalTypeAnnotation.listType(), nestedResult.getLogicalTypeAnnotation(),
+ "Should preserve group's logical type annotation");
+ PrimitiveType timestampField = nestedResult.getType("timestamp").asPrimitiveType();
+ assertEquals(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MILLIS),
+ timestampField.getLogicalTypeAnnotation());
+ }
+}
diff --git a/hudi-common/src/parquet/test/org/apache/parquet/schema/TestSchemaRepairEquivalence.java b/hudi-common/src/parquet/test/org/apache/parquet/schema/TestSchemaRepairEquivalence.java
new file mode 100644
index 0000000000000..75fe9ffde7d61
--- /dev/null
+++ b/hudi-common/src/parquet/test/org/apache/parquet/schema/TestSchemaRepairEquivalence.java
@@ -0,0 +1,481 @@
+/*
+ * 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.parquet.schema;
+
+import org.apache.avro.LogicalTypes;
+import org.apache.avro.Schema;
+import org.apache.avro.SchemaBuilder;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.parquet.avro.HoodieAvroParquetSchemaConverter;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+/**
+ * Tests equivalence between {@link SchemaRepair} and {@link AvroSchemaRepair}.
+ *
+ * This test class verifies that both repair implementations produce logically
+ * equivalent results when converting between Avro and Parquet schemas.
+ */
+public class TestSchemaRepairEquivalence {
+
+ private HoodieAvroParquetSchemaConverter converter;
+
+ @BeforeEach
+ public void setUp() {
+ converter = HoodieAvroParquetSchemaConverter.getAvroSchemaConverter(new Configuration());
+ }
+
+ /**
+ * Helper method to verify that AvroSchemaRepair and SchemaRepair produce equivalent results.
+ */
+ private void assertRepairEquivalence(Schema requestedAvro, Schema tableAvro) {
+ // Apply Avro repair
+ Schema repairedAvro = AvroSchemaRepair.repairLogicalTypes(requestedAvro, tableAvro);
+
+ // Convert to Parquet schemas
+ MessageType requestedParquet = converter.convert(requestedAvro);
+ MessageType tableParquet = converter.convert(tableAvro);
+
+ // Apply Parquet repair
+ MessageType repairedParquet = SchemaRepair.repairLogicalTypes(requestedParquet, tableParquet);
+
+ // Convert repaired Parquet back to Avro
+ Schema repairedParquetAsAvro = converter.convert(repairedParquet);
+
+ // Verify equivalence
+ assertEquals(repairedAvro, repairedParquetAsAvro,
+ "SchemaRepair and AvroSchemaRepair should produce equivalent results");
+ }
+
+ @Test
+ public void testEquivalenceNoRepairNeeded() {
+ Schema requestedSchema = SchemaBuilder.record("testrecord")
+ .fields()
+ .name("value").type().longType().noDefault()
+ .endRecord();
+ Schema tableSchema = SchemaBuilder.record("testrecord")
+ .fields()
+ .name("value").type().longType().noDefault()
+ .endRecord();
+
+ assertRepairEquivalence(requestedSchema, tableSchema);
+ }
+
+ @Test
+ public void testEquivalenceLongToLocalTimestampMillis() {
+ Schema requestedSchema = SchemaBuilder.record("testrecord")
+ .fields()
+ .name("timestamp").type().longType().noDefault()
+ .endRecord();
+ Schema tableSchema = SchemaBuilder.record("testrecord")
+ .fields()
+ .name("timestamp")
+ .type(LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG)))
+ .noDefault()
+ .endRecord();
+
+ assertRepairEquivalence(requestedSchema, tableSchema);
+ }
+
+ @Test
+ public void testEquivalenceLongToLocalTimestampMicros() {
+ Schema requestedSchema = SchemaBuilder.record("testrecord")
+ .fields()
+ .name("timestamp").type().longType().noDefault()
+ .endRecord();
+ Schema tableSchema = SchemaBuilder.record("testrecord")
+ .fields()
+ .name("timestamp")
+ .type(LogicalTypes.localTimestampMicros().addToSchema(Schema.create(Schema.Type.LONG)))
+ .noDefault()
+ .endRecord();
+
+ assertRepairEquivalence(requestedSchema, tableSchema);
+ }
+
+ @Test
+ public void testEquivalenceTimestampMicrosToMillis() {
+ Schema requestedSchema = SchemaBuilder.record("testrecord")
+ .fields()
+ .name("timestamp")
+ .type(LogicalTypes.timestampMicros().addToSchema(Schema.create(Schema.Type.LONG)))
+ .noDefault()
+ .endRecord();
+ Schema tableSchema = SchemaBuilder.record("testrecord")
+ .fields()
+ .name("timestamp")
+ .type(LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG)))
+ .noDefault()
+ .endRecord();
+
+ assertRepairEquivalence(requestedSchema, tableSchema);
+ }
+
+ @Test
+ public void testEquivalenceNoRepairTimestampMillisToMicros() {
+ Schema requestedSchema = SchemaBuilder.record("testrecord")
+ .fields()
+ .name("timestamp")
+ .type(LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG)))
+ .noDefault()
+ .endRecord();
+ Schema tableSchema = SchemaBuilder.record("testrecord")
+ .fields()
+ .name("timestamp")
+ .type(LogicalTypes.timestampMicros().addToSchema(Schema.create(Schema.Type.LONG)))
+ .noDefault()
+ .endRecord();
+
+ assertRepairEquivalence(requestedSchema, tableSchema);
+ }
+
+ @Test
+ public void testEquivalenceSimpleRecord() {
+ Schema requestedSchema = SchemaBuilder.record("testrecord")
+ .fields()
+ .name("timestamp").type().longType().noDefault()
+ .endRecord();
+
+ Schema tableSchema = SchemaBuilder.record("testrecord")
+ .fields()
+ .name("timestamp")
+ .type(LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG)))
+ .noDefault()
+ .endRecord();
+
+ assertRepairEquivalence(requestedSchema, tableSchema);
+ }
+
+ @Test
+ public void testEquivalenceRecordMultipleFields() {
+ Schema requestedSchema = SchemaBuilder.record("testrecord")
+ .fields()
+ .name("id").type().intType().noDefault()
+ .name("timestamp").type().longType().noDefault()
+ .name("name").type().stringType().noDefault()
+ .endRecord();
+
+ Schema tableSchema = SchemaBuilder.record("testrecord")
+ .fields()
+ .name("id").type().intType().noDefault()
+ .name("timestamp")
+ .type(LogicalTypes.localTimestampMicros().addToSchema(Schema.create(Schema.Type.LONG)))
+ .noDefault()
+ .name("name").type().stringType().noDefault()
+ .endRecord();
+
+ assertRepairEquivalence(requestedSchema, tableSchema);
+ }
+
+ @Test
+ public void testEquivalenceNestedRecord() {
+ Schema nestedRequestedSchema = SchemaBuilder.record("nestedrecord")
+ .fields()
+ .name("timestamp").type().longType().noDefault()
+ .endRecord();
+
+ Schema nestedTableSchema = SchemaBuilder.record("nestedrecord")
+ .fields()
+ .name("timestamp")
+ .type(LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG)))
+ .noDefault()
+ .endRecord();
+
+ Schema requestedSchema = SchemaBuilder.record("outerrecord")
+ .fields()
+ .name("id").type().intType().noDefault()
+ .name("nestedrecord").type(nestedRequestedSchema).noDefault()
+ .endRecord();
+
+ Schema tableSchema = SchemaBuilder.record("outerrecord")
+ .fields()
+ .name("id").type().intType().noDefault()
+ .name("nestedrecord").type(nestedTableSchema).noDefault()
+ .endRecord();
+
+ assertRepairEquivalence(requestedSchema, tableSchema);
+ }
+
+ @Test
+ public void testEquivalenceRecordWithExtraFieldInRequested() {
+ Schema requestedSchema = SchemaBuilder.record("testrecord")
+ .fields()
+ .name("id").type().intType().noDefault()
+ .name("timestamp").type().longType().noDefault()
+ .name("newfield").type().longType().noDefault()
+ .endRecord();
+
+ Schema tableSchema = SchemaBuilder.record("testrecord")
+ .fields()
+ .name("id").type().intType().noDefault()
+ .name("timestamp")
+ .type(LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG)))
+ .noDefault()
+ .endRecord();
+
+ assertRepairEquivalence(requestedSchema, tableSchema);
+ }
+
+ @Test
+ public void testEquivalenceRecordMixedFields() {
+ Schema requestedSchema = SchemaBuilder.record("testrecord")
+ .fields()
+ .name("id").type().intType().noDefault()
+ .name("timestamp").type().longType().noDefault()
+ .name("newfield").type().stringType().noDefault()
+ .name("name").type().stringType().noDefault()
+ .endRecord();
+
+ Schema tableSchema = SchemaBuilder.record("testrecord")
+ .fields()
+ .name("id").type().intType().noDefault()
+ .name("timestamp")
+ .type(LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG)))
+ .noDefault()
+ .name("name").type().stringType().noDefault()
+ .endRecord();
+
+ assertRepairEquivalence(requestedSchema, tableSchema);
+ }
+
+ @Test
+ public void testEquivalenceNestedRecordWithExtraField() {
+ Schema nestedRequestedSchema = SchemaBuilder.record("nestedrecord")
+ .fields()
+ .name("timestamp").type().longType().noDefault()
+ .name("extrafield").type().stringType().noDefault()
+ .endRecord();
+
+ Schema nestedTableSchema = SchemaBuilder.record("nestedrecord")
+ .fields()
+ .name("timestamp")
+ .type(LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG)))
+ .noDefault()
+ .endRecord();
+
+ Schema requestedSchema = SchemaBuilder.record("outerrecord")
+ .fields()
+ .name("id").type().intType().noDefault()
+ .name("nestedrecord").type(nestedRequestedSchema).noDefault()
+ .endRecord();
+
+ Schema tableSchema = SchemaBuilder.record("outerrecord")
+ .fields()
+ .name("id").type().intType().noDefault()
+ .name("nestedrecord").type(nestedTableSchema).noDefault()
+ .endRecord();
+
+ assertRepairEquivalence(requestedSchema, tableSchema);
+ }
+
+ @Test
+ public void testEquivalenceRecordFirstFieldChanged() {
+ Schema requestedSchema = SchemaBuilder.record("testrecord")
+ .fields()
+ .name("timestamp1").type().longType().noDefault()
+ .name("timestamp2").type().longType().noDefault()
+ .endRecord();
+
+ Schema tableSchema = SchemaBuilder.record("testrecord")
+ .fields()
+ .name("timestamp1")
+ .type(LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG)))
+ .noDefault()
+ .name("timestamp2")
+ .type(LogicalTypes.localTimestampMicros().addToSchema(Schema.create(Schema.Type.LONG)))
+ .noDefault()
+ .endRecord();
+
+ assertRepairEquivalence(requestedSchema, tableSchema);
+ }
+
+ @Test
+ public void testEquivalenceRecordLastFieldChanged() {
+ Schema requestedSchema = SchemaBuilder.record("testrecord")
+ .fields()
+ .name("id").type().intType().noDefault()
+ .name("name").type().stringType().noDefault()
+ .name("timestamp").type().longType().noDefault()
+ .endRecord();
+
+ Schema tableSchema = SchemaBuilder.record("testrecord")
+ .fields()
+ .name("id").type().intType().noDefault()
+ .name("name").type().stringType().noDefault()
+ .name("timestamp")
+ .type(LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG)))
+ .noDefault()
+ .endRecord();
+
+ assertRepairEquivalence(requestedSchema, tableSchema);
+ }
+
+ @Test
+ public void testEquivalenceComplexNestedStructure() {
+ Schema innerRecordRequested = SchemaBuilder.record("inner")
+ .fields()
+ .name("timestamp").type().longType().noDefault()
+ .name("value").type().intType().noDefault()
+ .endRecord();
+
+ Schema innerRecordTable = SchemaBuilder.record("inner")
+ .fields()
+ .name("timestamp")
+ .type(LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG)))
+ .noDefault()
+ .name("value").type().intType().noDefault()
+ .endRecord();
+
+ Schema middleRecordRequested = SchemaBuilder.record("middle")
+ .fields()
+ .name("inner").type(innerRecordRequested).noDefault()
+ .name("middletimestamp").type().longType().noDefault()
+ .endRecord();
+
+ Schema middleRecordTable = SchemaBuilder.record("middle")
+ .fields()
+ .name("inner").type(innerRecordTable).noDefault()
+ .name("middletimestamp")
+ .type(LogicalTypes.localTimestampMicros().addToSchema(Schema.create(Schema.Type.LONG)))
+ .noDefault()
+ .endRecord();
+
+ Schema requestedSchema = SchemaBuilder.record("outer")
+ .fields()
+ .name("id").type().intType().noDefault()
+ .name("middle").type(middleRecordRequested).noDefault()
+ .name("outertimestamp").type().longType().noDefault()
+ .endRecord();
+
+ Schema tableSchema = SchemaBuilder.record("outer")
+ .fields()
+ .name("id").type().intType().noDefault()
+ .name("middle").type(middleRecordTable).noDefault()
+ .name("outertimestamp")
+ .type(LogicalTypes.timestampMicros().addToSchema(Schema.create(Schema.Type.LONG)))
+ .noDefault()
+ .endRecord();
+
+ assertRepairEquivalence(requestedSchema, tableSchema);
+ }
+
+ @Test
+ public void testEquivalenceEmptyRecord() {
+ Schema requestedSchema = SchemaBuilder.record("emptyrecord").fields().endRecord();
+ Schema tableSchema = SchemaBuilder.record("emptyrecord").fields().endRecord();
+
+ assertRepairEquivalence(requestedSchema, tableSchema);
+ }
+
+ @Test
+ public void testEquivalenceRecordNoFieldsMatch() {
+ Schema requestedSchema = SchemaBuilder.record("testrecord")
+ .fields()
+ .name("field1").type().longType().noDefault()
+ .name("field2").type().stringType().noDefault()
+ .endRecord();
+
+ Schema tableSchema = SchemaBuilder.record("testrecord")
+ .fields()
+ .name("field3").type().intType().noDefault()
+ .name("field4")
+ .type(LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG)))
+ .noDefault()
+ .endRecord();
+
+ assertRepairEquivalence(requestedSchema, tableSchema);
+ }
+
+ @Test
+ public void testEquivalenceMultipleTimestampRepairs() {
+ Schema requestedSchema = SchemaBuilder.record("testrecord")
+ .fields()
+ .name("ts1").type().longType().noDefault()
+ .name("ts2").type().longType().noDefault()
+ .name("ts3").type(LogicalTypes.timestampMicros().addToSchema(Schema.create(Schema.Type.LONG))).noDefault()
+ .name("ts4").type().longType().noDefault()
+ .endRecord();
+
+ Schema tableSchema = SchemaBuilder.record("testrecord")
+ .fields()
+ .name("ts1")
+ .type(LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG)))
+ .noDefault()
+ .name("ts2")
+ .type(LogicalTypes.localTimestampMicros().addToSchema(Schema.create(Schema.Type.LONG)))
+ .noDefault()
+ .name("ts3")
+ .type(LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG)))
+ .noDefault()
+ .name("ts4").type().longType().noDefault()
+ .endRecord();
+
+ assertRepairEquivalence(requestedSchema, tableSchema);
+ }
+
+ @Test
+ public void testEquivalenceDeepNesting() {
+ Schema level3Requested = SchemaBuilder.record("level3")
+ .fields()
+ .name("timestamp").type().longType().noDefault()
+ .endRecord();
+
+ Schema level3Table = SchemaBuilder.record("level3")
+ .fields()
+ .name("timestamp")
+ .type(LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG)))
+ .noDefault()
+ .endRecord();
+
+ Schema level2Requested = SchemaBuilder.record("level2")
+ .fields()
+ .name("level3").type(level3Requested).noDefault()
+ .endRecord();
+
+ Schema level2Table = SchemaBuilder.record("level2")
+ .fields()
+ .name("level3").type(level3Table).noDefault()
+ .endRecord();
+
+ Schema level1Requested = SchemaBuilder.record("level1")
+ .fields()
+ .name("level2").type(level2Requested).noDefault()
+ .endRecord();
+
+ Schema level1Table = SchemaBuilder.record("level1")
+ .fields()
+ .name("level2").type(level2Table).noDefault()
+ .endRecord();
+
+ Schema requestedSchema = SchemaBuilder.record("level0")
+ .fields()
+ .name("level1").type(level1Requested).noDefault()
+ .endRecord();
+
+ Schema tableSchema = SchemaBuilder.record("level0")
+ .fields()
+ .name("level1").type(level1Table).noDefault()
+ .endRecord();
+
+ assertRepairEquivalence(requestedSchema, tableSchema);
+ }
+}
diff --git a/hudi-common/src/test/java/org/apache/hudi/common/functional/TestHoodieLogFormat.java b/hudi-common/src/test/java/org/apache/hudi/common/functional/TestHoodieLogFormat.java
index 601f83101c9b7..129bb5287736e 100755
--- a/hudi-common/src/test/java/org/apache/hudi/common/functional/TestHoodieLogFormat.java
+++ b/hudi-common/src/test/java/org/apache/hudi/common/functional/TestHoodieLogFormat.java
@@ -425,7 +425,7 @@ public void testHugeLogFileWrite() throws IOException, URISyntaxException, Inter
byte[] dataBlockContentBytes = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records, header).getContentBytes();
HoodieLogBlock.HoodieLogBlockContentLocation logBlockContentLoc = new HoodieLogBlock.HoodieLogBlockContentLocation(new Configuration(), null, 0, dataBlockContentBytes.length, 0);
HoodieDataBlock reusableDataBlock = new HoodieAvroDataBlock(null, Option.ofNullable(dataBlockContentBytes), false,
- logBlockContentLoc, Option.ofNullable(getSimpleSchema()), header, new HashMap<>(), HoodieRecord.RECORD_KEY_METADATA_FIELD);
+ logBlockContentLoc, Option.ofNullable(getSimpleSchema()), header, new HashMap<>(), HoodieRecord.RECORD_KEY_METADATA_FIELD, false);
long writtenSize = 0;
int logBlockWrittenNum = 0;
while (writtenSize < Integer.MAX_VALUE) {
diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestDataGenerator.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestDataGenerator.java
index cd3755d26c81f..32ca10777aaf2 100644
--- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestDataGenerator.java
+++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestDataGenerator.java
@@ -19,9 +19,11 @@
package org.apache.hudi.common.testutils;
+import org.apache.hudi.avro.AvroSchemaUtils;
import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.avro.model.HoodieCompactionPlan;
import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.HoodieAvroIndexedRecord;
import org.apache.hudi.common.model.HoodieAvroPayload;
import org.apache.hudi.common.model.HoodieAvroRecord;
import org.apache.hudi.common.model.HoodieCommitMetadata;
@@ -35,10 +37,13 @@
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.table.timeline.TimelineMetadataUtils;
import org.apache.hudi.common.util.AvroOrcUtils;
+import org.apache.hudi.common.util.CollectionUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieIOException;
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.avro.Conversions;
import org.apache.avro.LogicalTypes;
import org.apache.avro.Schema;
@@ -62,8 +67,10 @@
import java.nio.ByteBuffer;
import java.nio.charset.StandardCharsets;
import java.time.Instant;
+import java.time.LocalDate;
import java.time.LocalDateTime;
import java.time.ZoneOffset;
+import java.time.ZonedDateTime;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
@@ -72,15 +79,20 @@
import java.util.List;
import java.util.Locale;
import java.util.Map;
+import java.util.Objects;
import java.util.Random;
import java.util.Set;
import java.util.UUID;
+import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Predicate;
import java.util.function.Supplier;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
import java.util.stream.Stream;
+import static org.apache.hudi.avro.HoodieAvroUtils.createNewSchemaField;
+import static org.apache.hudi.common.util.StringUtils.getUTF8Bytes;
import static org.apache.hudi.common.util.ValidationUtils.checkState;
/**
@@ -131,6 +143,21 @@ public class HoodieTestDataGenerator implements AutoCloseable {
+ "{\"name\":\"current_date\",\"type\": {\"type\": \"int\", \"logicalType\": \"date\"}},"
+ "{\"name\":\"current_ts\",\"type\": {\"type\": \"long\"}},"
+ "{\"name\":\"height\",\"type\":{\"type\":\"fixed\",\"name\":\"abc\",\"size\":5,\"logicalType\":\"decimal\",\"precision\":10,\"scale\":6}},";
+ public static final String EXTENDED_LOGICAL_TYPES_SCHEMA_V6 = "{\"name\":\"ts_millis\",\"type\":{\"type\":\"long\",\"logicalType\":\"timestamp-millis\"}},"
+ + "{\"name\":\"ts_micros\",\"type\":{\"type\":\"long\",\"logicalType\":\"timestamp-micros\"}},"
+ + "{\"name\":\"local_ts_millis\",\"type\":{\"type\":\"long\",\"logicalType\":\"local-timestamp-millis\"}},"
+ + "{\"name\":\"local_ts_micros\",\"type\":{\"type\":\"long\",\"logicalType\":\"local-timestamp-micros\"}},"
+ + "{\"name\":\"event_date\",\"type\":{\"type\":\"int\",\"logicalType\":\"date\"}},";
+ public static final String EXTENDED_LOGICAL_TYPES_SCHEMA = "{\"name\":\"ts_millis\",\"type\":{\"type\":\"long\",\"logicalType\":\"timestamp-millis\"}},"
+ + "{\"name\":\"ts_micros\",\"type\":{\"type\":\"long\",\"logicalType\":\"timestamp-micros\"}},"
+ + "{\"name\":\"local_ts_millis\",\"type\":{\"type\":\"long\",\"logicalType\":\"local-timestamp-millis\"}},"
+ + "{\"name\":\"local_ts_micros\",\"type\":{\"type\":\"long\",\"logicalType\":\"local-timestamp-micros\"}},"
+ + "{\"name\":\"event_date\",\"type\":{\"type\":\"int\",\"logicalType\":\"date\"}},";
+
+ // LTS = Local Timestamp
+ public static final String EXTENDED_LOGICAL_TYPES_SCHEMA_NO_LTS = "{\"name\":\"ts_millis\",\"type\":{\"type\":\"long\",\"logicalType\":\"timestamp-millis\"}},"
+ + "{\"name\":\"ts_micros\",\"type\":{\"type\":\"long\",\"logicalType\":\"timestamp-micros\"}},"
+ + "{\"name\":\"event_date\",\"type\":{\"type\":\"int\",\"logicalType\":\"date\"}},";
public static final String TRIP_EXAMPLE_SCHEMA =
TRIP_SCHEMA_PREFIX + EXTRA_TYPE_SCHEMA + MAP_TYPE_SCHEMA + FARE_NESTED_SCHEMA + TIP_NESTED_SCHEMA + TRIP_SCHEMA_SUFFIX;
@@ -139,6 +166,13 @@ public class HoodieTestDataGenerator implements AutoCloseable {
public static final String TRIP_NESTED_EXAMPLE_SCHEMA =
TRIP_SCHEMA_PREFIX + FARE_NESTED_SCHEMA + TRIP_SCHEMA_SUFFIX;
+ public static final String TRIP_LOGICAL_TYPES_SCHEMA_V6 =
+ TRIP_SCHEMA_PREFIX + EXTENDED_LOGICAL_TYPES_SCHEMA_V6 + TRIP_SCHEMA_SUFFIX;
+ public static final String TRIP_LOGICAL_TYPES_SCHEMA =
+ TRIP_SCHEMA_PREFIX + EXTENDED_LOGICAL_TYPES_SCHEMA + TRIP_SCHEMA_SUFFIX;
+ // LTS = Local Timestamp
+ public static final String TRIP_LOGICAL_TYPES_SCHEMA_NO_LTS =
+ TRIP_SCHEMA_PREFIX + EXTENDED_LOGICAL_TYPES_SCHEMA_NO_LTS + TRIP_SCHEMA_SUFFIX;
public static final String TRIP_SCHEMA = "{\"type\":\"record\",\"name\":\"tripUberRec\",\"fields\":["
+ "{\"name\":\"timestamp\",\"type\":\"long\"},{\"name\":\"_row_key\",\"type\":\"string\"},{\"name\":\"rider\",\"type\":\"string\"},"
@@ -151,13 +185,15 @@ public class HoodieTestDataGenerator implements AutoCloseable {
public static final String TRIP_HIVE_COLUMN_TYPES = "bigint,string,string,string,string,string,double,double,double,double,int,bigint,float,binary,int,bigint,decimal(10,6),"
+ "map,struct,array>,boolean";
-
public static final Schema AVRO_SCHEMA = new Schema.Parser().parse(TRIP_EXAMPLE_SCHEMA);
public static final Schema NESTED_AVRO_SCHEMA = new Schema.Parser().parse(TRIP_NESTED_EXAMPLE_SCHEMA);
public static final TypeDescription ORC_SCHEMA = AvroOrcUtils.createOrcSchema(new Schema.Parser().parse(TRIP_EXAMPLE_SCHEMA));
public static final Schema AVRO_SCHEMA_WITH_METADATA_FIELDS =
HoodieAvroUtils.addMetadataFields(AVRO_SCHEMA);
public static final Schema AVRO_SHORT_TRIP_SCHEMA = new Schema.Parser().parse(SHORT_TRIP_SCHEMA);
+ public static final Schema AVRO_TRIP_LOGICAL_TYPES_SCHEMA = new Schema.Parser().parse(TRIP_LOGICAL_TYPES_SCHEMA);
+ public static final Schema AVRO_TRIP_LOGICAL_TYPES_SCHEMA_V6 = new Schema.Parser().parse(TRIP_LOGICAL_TYPES_SCHEMA_V6);
+ public static final Schema AVRO_TRIP_LOGICAL_TYPES_SCHEMA_NO_LTS = new Schema.Parser().parse(TRIP_LOGICAL_TYPES_SCHEMA_NO_LTS);
public static final Schema AVRO_TRIP_SCHEMA = new Schema.Parser().parse(TRIP_SCHEMA);
public static final TypeDescription ORC_TRIP_SCHEMA = AvroOrcUtils.createOrcSchema(new Schema.Parser().parse(TRIP_SCHEMA));
public static final Schema FLATTENED_AVRO_SCHEMA = new Schema.Parser().parse(TRIP_FLATTENED_SCHEMA);
@@ -169,6 +205,7 @@ public class HoodieTestDataGenerator implements AutoCloseable {
private final String[] partitionPaths;
//maintains the count of existing keys schema wise
private Map numKeysBySchema;
+ private Option extendedSchema = Option.empty();
public HoodieTestDataGenerator(long seed) {
this(seed, DEFAULT_PARTITION_PATHS, new HashMap<>());
@@ -258,16 +295,40 @@ public int getEstimatedFileSizeInBytes(int numOfRecords) {
}
public RawTripTestPayload generateRandomValueAsPerSchema(String schemaStr, HoodieKey key, String commitTime, boolean isFlattened) throws IOException {
- if (TRIP_EXAMPLE_SCHEMA.equals(schemaStr)) {
- return generateRandomValue(key, commitTime, isFlattened);
- } else if (TRIP_SCHEMA.equals(schemaStr)) {
- return generatePayloadForTripSchema(key, commitTime);
- } else if (SHORT_TRIP_SCHEMA.equals(schemaStr)) {
- return generatePayloadForShortTripSchema(key, commitTime);
- } else if (TRIP_NESTED_EXAMPLE_SCHEMA.equals(schemaStr)) {
- return generateNestedExampleRandomValue(key, commitTime);
+ return generateRandomValueAsPerSchema(schemaStr, key, commitTime, isFlattened, false, 0L);
+ }
+
+ public RawTripTestPayload generateRandomValueAsPerSchema(String schemaStr, HoodieKey key, String commitTime,
+ boolean isFlattened, boolean isDelete, long timestamp) throws IOException {
+ if (!isDelete) {
+ if (TRIP_FLATTENED_SCHEMA.equals(schemaStr)) {
+ return generateRandomValue(key, commitTime, true, timestamp);
+ } else if (TRIP_EXAMPLE_SCHEMA.equals(schemaStr)) {
+ return generateRandomValue(key, commitTime, isFlattened, timestamp);
+ } else if (TRIP_SCHEMA.equals(schemaStr)) {
+ return generatePayloadForTripSchema(key, commitTime);
+ } else if (SHORT_TRIP_SCHEMA.equals(schemaStr)) {
+ return generatePayloadForShortTripSchema(key, commitTime);
+ } else if (TRIP_NESTED_EXAMPLE_SCHEMA.equals(schemaStr)) {
+ return generateNestedExampleRandomValue(key, commitTime);
+ } else if (TRIP_LOGICAL_TYPES_SCHEMA.equals(schemaStr)) {
+ return generatePayloadForLogicalTypesSchema(key, commitTime, false, timestamp);
+ } else if (TRIP_LOGICAL_TYPES_SCHEMA_V6.equals(schemaStr)) {
+ return generatePayloadForLogicalTypesSchemaV6(key, commitTime, false, timestamp);
+ } else if (TRIP_LOGICAL_TYPES_SCHEMA_NO_LTS.equals(schemaStr)) {
+ return generatePayloadForLogicalTypesSchemaNoLTS(key, commitTime, false, timestamp);
+ }
+ } else {
+ if (TRIP_EXAMPLE_SCHEMA.equals(schemaStr)) {
+ return generateRandomDeleteValue(key, commitTime);
+ } else if (TRIP_LOGICAL_TYPES_SCHEMA.equals(schemaStr)) {
+ return generatePayloadForLogicalTypesSchema(key, commitTime, true, timestamp);
+ } else if (TRIP_LOGICAL_TYPES_SCHEMA_V6.equals(schemaStr)) {
+ return generatePayloadForLogicalTypesSchemaV6(key, commitTime, true, timestamp);
+ } else if (TRIP_LOGICAL_TYPES_SCHEMA_NO_LTS.equals(schemaStr)) {
+ return generatePayloadForLogicalTypesSchemaNoLTS(key, commitTime, true, timestamp);
+ }
}
-
return null;
}
@@ -336,9 +397,17 @@ public RawTripTestPayload generatePayloadForShortTripSchema(HoodieKey key, Strin
* Generates a new avro record of the above schema format for a delete.
*/
private RawTripTestPayload generateRandomDeleteValue(HoodieKey key, String instantTime) throws IOException {
+ return generateRandomDeleteValue(key, instantTime, TRIP_EXAMPLE_SCHEMA);
+ }
+
+ private RawTripTestPayload generateRandomDeleteValue(HoodieKey key, String instantTime, String schemaStr) throws IOException {
GenericRecord rec = generateGenericRecord(key.getRecordKey(), key.getPartitionPath(), "rider-" + instantTime, "driver-" + instantTime, 0,
true, false);
- return new RawTripTestPayload(Option.of(rec.toString()), key.getRecordKey(), key.getPartitionPath(), TRIP_EXAMPLE_SCHEMA, true, 0L);
+ return new RawTripTestPayload(Option.of(rec.toString()), key.getRecordKey(), key.getPartitionPath(), schemaStr, true, 0L);
+ }
+
+ private RawTripTestPayload generateRandomDeleteValuePerSchema(HoodieKey key, String instantTime, String schemaStr) throws IOException {
+ return generateRandomValueAsPerSchema(schemaStr, key, instantTime, false, true, 0L);
}
/**
@@ -354,6 +423,20 @@ public GenericRecord generateGenericRecord(String rowKey, String partitionPath,
return generateGenericRecord(rowKey, partitionPath, riderName, driverName, timestamp, false, false);
}
+ /**
+ * LTS = Local Timestamp
+ */
+ public RawTripTestPayload generatePayloadForLogicalTypesSchemaNoLTS(HoodieKey key, String commitTime, boolean isDelete, long timestamp) throws IOException {
+ return generateRecordForTripLogicalTypesSchema(key, "rider-" + commitTime, "driver-" + commitTime, timestamp, isDelete, false, false);
+ }
+
+ public RawTripTestPayload generatePayloadForLogicalTypesSchema(HoodieKey key, String commitTime, boolean isDelete, long timestamp) throws IOException {
+ return generateRecordForTripLogicalTypesSchema(key, "rider-" + commitTime, "driver-" + commitTime, timestamp, isDelete, false, true);
+ }
+
+ public RawTripTestPayload generatePayloadForLogicalTypesSchemaV6(HoodieKey key, String commitTime, boolean isDelete, long timestamp) throws IOException {
+ return generateRecordForTripLogicalTypesSchema(key, "rider-" + commitTime, "driver-" + commitTime, timestamp, isDelete, true, true);
+ }
/**
* Populate rec with values for TRIP_SCHEMA_PREFIX
@@ -442,13 +525,11 @@ private void generateTripSuffixValues(GenericRecord rec, boolean isDeleteRecord)
}
}
-
/**
* Generate record conforming to TRIP_EXAMPLE_SCHEMA or TRIP_FLATTENED_SCHEMA if isFlattened is true
*/
public GenericRecord generateGenericRecord(String rowKey, String partitionPath, String riderName, String driverName,
- long timestamp, boolean isDeleteRecord,
- boolean isFlattened) {
+ long timestamp, boolean isDeleteRecord, boolean isFlattened) {
GenericRecord rec = new GenericData.Record(isFlattened ? FLATTENED_AVRO_SCHEMA : AVRO_SCHEMA);
generateTripPrefixValues(rec, rowKey, partitionPath, riderName, driverName, timestamp);
if (isFlattened) {
@@ -500,6 +581,65 @@ public GenericRecord generateRecordForShortTripSchema(String rowKey, String ride
return rec;
}
+ public RawTripTestPayload generateRecordForTripLogicalTypesSchema(HoodieKey key, String riderName, String driverName,
+ long timestamp, boolean isDeleteRecord, boolean v6,
+ boolean hasLTS) throws IOException {
+ GenericRecord rec;
+ if (!hasLTS) {
+ // LTS = Local Timestamp
+ rec = new GenericData.Record(AVRO_TRIP_LOGICAL_TYPES_SCHEMA_NO_LTS);
+ } else if (v6) {
+ rec = new GenericData.Record(AVRO_TRIP_LOGICAL_TYPES_SCHEMA_V6);
+ } else {
+ rec = new GenericData.Record(AVRO_TRIP_LOGICAL_TYPES_SCHEMA);
+ }
+ generateTripPrefixValues(rec, key.getRecordKey(), key.getPartitionPath(), riderName, driverName, timestamp);
+
+ int hash = key.getRecordKey().hashCode();
+ boolean above = (hash & 1) == 0; // half above, half below threshold
+
+ // -------------------
+ // Threshold definitions
+ // -------------------
+ Instant tsMillisThreshold = Instant.parse("2020-01-01T00:00:00Z");
+ Instant tsMicrosThreshold = Instant.parse("2020-06-01T12:00:00Z");
+
+ Instant localTsMillisThreshold = ZonedDateTime.of(
+ 2015, 5, 20, 12, 34, 56, 0, ZoneOffset.UTC).toInstant();
+ Instant localTsMicrosThreshold = ZonedDateTime.of(
+ 2017, 7, 7, 7, 7, 7, 0, ZoneOffset.UTC).toInstant();
+
+ LocalDate dateThreshold = LocalDate.of(2000, 1, 1);
+
+ // -------------------
+ // Assign edge values
+ // -------------------
+
+ // ts_millis
+ long tsMillisBase = tsMillisThreshold.toEpochMilli();
+ rec.put("ts_millis", above ? tsMillisBase + 1 : tsMillisBase - 1);
+
+ // ts_micros
+ long tsMicrosBase = TimeUnit.SECONDS.toMicros(tsMicrosThreshold.getEpochSecond()) + tsMicrosThreshold.getNano() / 1_000L;
+ rec.put("ts_micros", above ? tsMicrosBase + 1 : tsMicrosBase - 1);
+
+ if (hasLTS) {
+ // local_ts_millis
+ long localTsMillisBase = localTsMillisThreshold.toEpochMilli();
+ rec.put("local_ts_millis", above ? localTsMillisBase + 1 : localTsMillisBase - 1);
+
+ // local_ts_micros
+ long localTsMicrosBase = TimeUnit.SECONDS.toMicros(localTsMicrosThreshold.getEpochSecond()) + localTsMicrosThreshold.getNano() / 1_000L;
+ rec.put("local_ts_micros", above ? localTsMicrosBase + 1 : localTsMicrosBase - 1);
+ }
+
+ // event_date
+ int eventDateBase = (int) dateThreshold.toEpochDay();
+ rec.put("event_date", above ? eventDateBase + 1 : eventDateBase - 1);
+ generateTripSuffixValues(rec, isDeleteRecord);
+ return new RawTripTestPayload(rec.toString(), key.getRecordKey(), key.getPartitionPath(), rec.getSchema().toString());
+ }
+
public static void createRequestedCommitFile(String basePath, String instantTime, Configuration configuration) throws IOException {
Path pendingRequestedFile = new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/"
+ HoodieTimeline.makeRequestedCommitFileName(instantTime));
@@ -939,9 +1079,13 @@ public Stream generateUniqueUpdatesStream(String instantTime, Inte
* @return stream of hoodie record updates
*/
public Stream generateUniqueDeleteStream(Integer n) {
+ return generateUniqueDeleteStream(n, TRIP_EXAMPLE_SCHEMA);
+ }
+
+ public Stream generateUniqueDeleteStream(Integer n, String streamStr) {
final Set used = new HashSet<>();
- Map existingKeys = existingKeysBySchema.get(TRIP_EXAMPLE_SCHEMA);
- Integer numExistingKeys = numKeysBySchema.get(TRIP_EXAMPLE_SCHEMA);
+ Map existingKeys = existingKeysBySchema.get(streamStr);
+ Integer numExistingKeys = numKeysBySchema.get(streamStr);
if (n > numExistingKeys) {
throw new IllegalArgumentException("Requested unique deletes is greater than number of available keys");
}
@@ -959,7 +1103,7 @@ public Stream generateUniqueDeleteStream(Integer n) {
used.add(kp);
result.add(kp.key);
}
- numKeysBySchema.put(TRIP_EXAMPLE_SCHEMA, numExistingKeys);
+ numKeysBySchema.put(streamStr, numExistingKeys);
return result.stream();
}
@@ -971,9 +1115,13 @@ public Stream generateUniqueDeleteStream(Integer n) {
* @return stream of hoodie records for delete
*/
public Stream generateUniqueDeleteRecordStream(String instantTime, Integer n) {
+ return generateUniqueDeleteRecordStream(instantTime, n, TRIP_EXAMPLE_SCHEMA);
+ }
+
+ public Stream generateUniqueDeleteRecordStream(String instantTime, Integer n, String schemaStr) {
final Set used = new HashSet<>();
- Map existingKeys = existingKeysBySchema.get(TRIP_EXAMPLE_SCHEMA);
- Integer numExistingKeys = numKeysBySchema.get(TRIP_EXAMPLE_SCHEMA);
+ Map existingKeys = existingKeysBySchema.get(schemaStr);
+ Integer numExistingKeys = numKeysBySchema.get(schemaStr);
if (n > numExistingKeys) {
throw new IllegalArgumentException("Requested unique deletes is greater than number of available keys");
}
@@ -991,12 +1139,12 @@ public Stream generateUniqueDeleteRecordStream(String instantTime,
numExistingKeys--;
used.add(kp);
try {
- result.add(new HoodieAvroRecord(kp.key, generateRandomDeleteValue(kp.key, instantTime)));
+ result.add(new HoodieAvroRecord(kp.key, generateRandomDeleteValuePerSchema(kp.key, instantTime, schemaStr)));
} catch (IOException e) {
throw new HoodieIOException(e.getMessage(), e);
}
}
- numKeysBySchema.put(TRIP_EXAMPLE_SCHEMA, numExistingKeys);
+ numKeysBySchema.put(schemaStr, numExistingKeys);
return result.stream();
}
@@ -1062,7 +1210,7 @@ public void close() {
private static long genRandomTimeMillis(Random r) {
// Fri Feb 13 15:31:30 PST 2009
- long anchorTs = 1234567890L;
+ long anchorTs = 1234567890000L;
// NOTE: To provide for certainty and not generate overly random dates, we will limit
// dispersion to be w/in +/- 3 days from the anchor date
return anchorTs + r.nextLong() % 259200000L;
@@ -1086,4 +1234,311 @@ public static UUID genPseudoRandomUUID(Random r) {
throw new HoodieException(e);
}
}
+
+ /**
+ * Used for equality checks between the expected and actual records for generated by the HoodieTestDataGenerator.
+ * The fields identify the record with the combination of the recordKey and partitionPath and assert that the proper
+ * value is present with the orderingVal and the riderValue, which is updated as part of the update utility methods.
+ */
+ public static class RecordIdentifier {
+ private final String recordKey;
+ private final String orderingVal;
+ private final String partitionPath;
+ private final String riderValue;
+
+ @JsonCreator
+ public RecordIdentifier(@JsonProperty("recordKey") String recordKey,
+ @JsonProperty("partitionPath") String partitionPath,
+ @JsonProperty("orderingVal") String orderingVal,
+ @JsonProperty("riderValue") String riderValue) {
+ this.recordKey = recordKey;
+ this.orderingVal = orderingVal;
+ this.partitionPath = partitionPath;
+ this.riderValue = riderValue;
+ }
+
+ public static RecordIdentifier clone(RecordIdentifier toClone, String orderingVal) {
+ return new RecordIdentifier(toClone.recordKey, toClone.partitionPath, orderingVal, toClone.riderValue);
+ }
+
+ public static RecordIdentifier fromTripTestPayload(HoodieAvroIndexedRecord record, String[] orderingFields) {
+ String recordKey = record.getRecordKey();
+ String partitionPath = record.getPartitionPath();
+ Comparable orderingValue = record.getOrderingValue(record.getData().getSchema(), CollectionUtils.emptyProps());
+ String orderingValStr = orderingValue.toString();
+ GenericRecord data = (GenericRecord) record.getData();
+ String riderValue = data.getSchema().getField("rider") != null ? data.get("rider").toString() : "";
+ return new RecordIdentifier(recordKey, partitionPath, orderingValStr, riderValue);
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ RecordIdentifier that = (RecordIdentifier) o;
+ return Objects.equals(recordKey, that.recordKey)
+ && Objects.equals(orderingVal, that.orderingVal)
+ && Objects.equals(partitionPath, that.partitionPath)
+ && Objects.equals(riderValue, that.riderValue);
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(recordKey, orderingVal, partitionPath, riderValue);
+ }
+
+ public String getRecordKey() {
+ return recordKey;
+ }
+
+ public String getOrderingVal() {
+ return orderingVal;
+ }
+
+ public String getPartitionPath() {
+ return partitionPath;
+ }
+
+ public String getRiderValue() {
+ return riderValue;
+ }
+
+ @Override
+ public String toString() {
+ return "RowKey: " + recordKey + ", PartitionPath: " + partitionPath
+ + ", OrderingVal: " + orderingVal + ", RiderValue: " + riderValue;
+ }
+ }
+
+ public static class SchemaEvolutionConfigs {
+ public Schema schema = AVRO_SCHEMA;
+ public boolean nestedSupport = true;
+ public boolean mapSupport = true;
+ public boolean arraySupport = true;
+ public boolean addNewFieldSupport = true;
+ // TODO: [HUDI-9603] Flink 1.18 array values incorrect in fg reader test
+ public boolean anyArraySupport = true;
+
+ // Int
+ public boolean intToLongSupport = true;
+ public boolean intToFloatSupport = true;
+ public boolean intToDoubleSupport = true;
+ public boolean intToStringSupport = true;
+
+ // Long
+ public boolean longToFloatSupport = true;
+ public boolean longToDoubleSupport = true;
+ public boolean longToStringSupport = true;
+
+ // Float
+ public boolean floatToDoubleSupport = true;
+ public boolean floatToStringSupport = true;
+
+ // Double
+ public boolean doubleToStringSupport = true;
+
+ // String
+ public boolean stringToBytesSupport = true;
+
+ // Bytes
+ public boolean bytesToStringSupport = true;
+ }
+
+ private enum SchemaEvolutionTypePromotionCase {
+ INT_TO_INT(Schema.Type.INT, Schema.Type.INT, config -> true),
+ INT_TO_LONG(Schema.Type.INT, Schema.Type.LONG, config -> config.intToLongSupport),
+ INT_TO_FLOAT(Schema.Type.INT, Schema.Type.FLOAT, config -> config.intToFloatSupport),
+ INT_TO_DOUBLE(Schema.Type.INT, Schema.Type.DOUBLE, config -> config.intToDoubleSupport),
+ INT_TO_STRING(Schema.Type.INT, Schema.Type.STRING, config -> config.intToStringSupport),
+ LONG_TO_LONG(Schema.Type.LONG, Schema.Type.LONG, config -> true),
+ LONG_TO_FLOAT(Schema.Type.LONG, Schema.Type.FLOAT, config -> config.longToFloatSupport),
+ LONG_TO_DOUBLE(Schema.Type.LONG, Schema.Type.DOUBLE, config -> config.longToDoubleSupport),
+ LONG_TO_STRING(Schema.Type.LONG, Schema.Type.STRING, config -> config.longToStringSupport),
+ FLOAT_TO_FLOAT(Schema.Type.FLOAT, Schema.Type.FLOAT, config -> true),
+ FLOAT_TO_DOUBLE(Schema.Type.FLOAT, Schema.Type.DOUBLE, config -> config.floatToDoubleSupport),
+ FLOAT_TO_STRING(Schema.Type.FLOAT, Schema.Type.STRING, config -> config.floatToStringSupport),
+ DOUBLE_TO_DOUBLE(Schema.Type.DOUBLE, Schema.Type.DOUBLE, config -> true),
+ DOUBLE_TO_STRING(Schema.Type.DOUBLE, Schema.Type.STRING, config -> config.doubleToStringSupport),
+ STRING_TO_STRING(Schema.Type.STRING, Schema.Type.STRING, config -> true),
+ STRING_TO_BYTES(Schema.Type.STRING, Schema.Type.BYTES, config -> config.stringToBytesSupport),
+ BYTES_TO_BYTES(Schema.Type.BYTES, Schema.Type.BYTES, config -> true),
+ BYTES_TO_STRING(Schema.Type.BYTES, Schema.Type.STRING, config -> config.bytesToStringSupport);
+
+ public final Schema.Type before;
+ public final Schema.Type after;
+ public final Predicate isEnabled;
+
+ SchemaEvolutionTypePromotionCase(Schema.Type before, Schema.Type after, Predicate isEnabled) {
+ this.before = before;
+ this.after = after;
+ this.isEnabled = isEnabled;
+ }
+ }
+
+ public void extendSchema(SchemaEvolutionConfigs configs, boolean isBefore) {
+ List baseFields = new ArrayList<>();
+ for (SchemaEvolutionTypePromotionCase evolution : SchemaEvolutionTypePromotionCase.values()) {
+ if (evolution.isEnabled.test(configs)) {
+ baseFields.add(isBefore ? evolution.before : evolution.after);
+ }
+ }
+
+ // Add new field if we are testing adding new fields
+ if (!isBefore && configs.addNewFieldSupport) {
+ baseFields.add(Schema.Type.BOOLEAN);
+ }
+
+ this.extendedSchema = Option.of(generateExtendedSchema(configs, new ArrayList<>(baseFields)));
+ }
+
+ public void extendSchemaBeforeEvolution(SchemaEvolutionConfigs configs) {
+ extendSchema(configs, true);
+ }
+
+ public void extendSchemaAfterEvolution(SchemaEvolutionConfigs configs) {
+ extendSchema(configs, false);
+ }
+
+ public Schema getExtendedSchema() {
+ return extendedSchema.orElseThrow(IllegalArgumentException::new);
+ }
+
+ private static Schema generateExtendedSchema(SchemaEvolutionConfigs configs, List baseFields) {
+ return generateExtendedSchema(configs.schema, configs, baseFields, "customField", true);
+ }
+
+ private static Schema generateExtendedSchema(Schema baseSchema, SchemaEvolutionConfigs configs, List baseFields, String fieldPrefix, boolean toplevel) {
+ List fields = baseSchema.getFields();
+ List finalFields = new ArrayList<>(fields.size() + baseFields.size());
+ boolean addedFields = false;
+ for (Schema.Field field : fields) {
+ if (configs.nestedSupport && field.name().equals("fare") && field.schema().getType() == Schema.Type.RECORD) {
+ finalFields.add(createNewSchemaField(field.name(), generateExtendedSchema(field.schema(), configs, baseFields, "customFare", false), field.doc(), field.defaultVal()));
+ } else if (configs.anyArraySupport || !field.name().equals("tip_history")) {
+ //TODO: [HUDI-9603] remove the if condition when the issue is fixed
+ if (field.name().equals("_hoodie_is_deleted")) {
+ addedFields = true;
+ addFields(configs, finalFields, baseFields, fieldPrefix, baseSchema.getNamespace(), toplevel);
+ }
+ finalFields.add(createNewSchemaField(field));
+ }
+ }
+ if (!addedFields) {
+ addFields(configs, finalFields, baseFields, fieldPrefix, baseSchema.getNamespace(), toplevel);
+ }
+ Schema finalSchema = Schema.createRecord(baseSchema.getName(), baseSchema.getDoc(),
+ baseSchema.getNamespace(), baseSchema.isError());
+ finalSchema.setFields(finalFields);
+ return finalSchema;
+ }
+
+ private static void addFields(SchemaEvolutionConfigs configs, List finalFields, List baseFields, String fieldPrefix, String namespace, boolean toplevel) {
+ if (toplevel) {
+ if (configs.mapSupport) {
+ List mapFields = new ArrayList<>(baseFields.size());
+ addFieldsHelper(mapFields, baseFields, fieldPrefix + "Map");
+ finalFields.add(new Schema.Field(fieldPrefix + "Map", Schema.createMap(Schema.createRecord("customMapRecord", "", namespace, false, mapFields)), "", null));
+ }
+
+ if (configs.arraySupport) {
+ List arrayFields = new ArrayList<>(baseFields.size());
+ addFieldsHelper(arrayFields, baseFields, fieldPrefix + "Array");
+ finalFields.add(new Schema.Field(fieldPrefix + "Array", Schema.createArray(Schema.createRecord("customArrayRecord", "", namespace, false, arrayFields)), "", null));
+ }
+ }
+ addFieldsHelper(finalFields, baseFields, fieldPrefix);
+ }
+
+ private static void addFieldsHelper(List finalFields, List baseFields, String fieldPrefix) {
+ for (int i = 0; i < baseFields.size(); i++) {
+ if (baseFields.get(i) == Schema.Type.BOOLEAN) {
+ // boolean fields are added fields
+ finalFields.add(new Schema.Field(fieldPrefix + i, AvroSchemaUtils.createNullableSchema(Schema.Type.BOOLEAN), "", null));
+ } else {
+ finalFields.add(new Schema.Field(fieldPrefix + i, Schema.create(baseFields.get(i)), "", null));
+ }
+ }
+ }
+
+ private void generateCustomValues(GenericRecord rec, String customPrefix) {
+ for (Schema.Field field : rec.getSchema().getFields()) {
+ if (field.name().startsWith(customPrefix)) {
+ switch (field.schema().getType()) {
+ case INT:
+ rec.put(field.name(), rand.nextInt());
+ break;
+ case LONG:
+ rec.put(field.name(), rand.nextLong());
+ break;
+ case FLOAT:
+ rec.put(field.name(), rand.nextFloat());
+ break;
+ case DOUBLE:
+ rec.put(field.name(), rand.nextDouble());
+ break;
+ case STRING:
+ rec.put(field.name(), genPseudoRandomUUID(rand).toString());
+ break;
+ case BYTES:
+ rec.put(field.name(), ByteBuffer.wrap(getUTF8Bytes(genPseudoRandomUUID(rand).toString())));
+ break;
+ case UNION:
+ if (!AvroSchemaUtils.getNonNullTypeFromUnion(field.schema()).getType().equals(Schema.Type.BOOLEAN)) {
+ throw new IllegalStateException("Union should only be boolean");
+ }
+ rec.put(field.name(), rand.nextBoolean());
+ break;
+ case BOOLEAN:
+ rec.put(field.name(), rand.nextBoolean());
+ break;
+ case MAP:
+ rec.put(field.name(), genMap(field.schema(), field.name()));
+ break;
+ case ARRAY:
+ rec.put(field.name(), genArray(field.schema(), field.name()));
+ break;
+ default:
+ throw new UnsupportedOperationException("Unsupported type: " + field.schema().getType());
+ }
+ }
+ }
+ }
+
+ private GenericArray genArray(Schema arraySchema, String customPrefix) {
+ GenericArray customArray = new GenericData.Array<>(1, arraySchema);
+ Schema arrayElementSchema = arraySchema.getElementType();
+ GenericRecord customRecord = new GenericData.Record(arrayElementSchema);
+ generateCustomValues(customRecord, customPrefix);
+ customArray.add(customRecord);
+ return customArray;
+ }
+
+ private Map genMap(Schema mapSchema, String customPrefix) {
+ Schema mapElementSchema = mapSchema.getValueType();
+ GenericRecord customRecord = new GenericData.Record(mapElementSchema);
+ generateCustomValues(customRecord, customPrefix);
+ return Collections.singletonMap("customMapKey", customRecord);
+ }
+
+ public static List recordsToStrings(List records) {
+ return records.stream().map(HoodieTestDataGenerator::recordToString).filter(Option::isPresent).map(Option::get)
+ .collect(Collectors.toList());
+ }
+
+ public static Option recordToString(HoodieRecord record) {
+ try {
+ String str = ((GenericRecord) record.getData()).toString();
+ // Remove the last } bracket
+ str = str.substring(0, str.length() - 1);
+ return Option.of(str + ", \"partition\": \"" + record.getPartitionPath() + "\"}");
+ } catch (Exception e) {
+ return Option.empty();
+ }
+ }
+
+ public static List deleteRecordsToStrings(List records) {
+ return records.stream().map(record -> "{\"_row_key\": \"" + record.getRecordKey() + "\",\"partition\": \"" + record.getPartitionPath() + "\"}")
+ .collect(Collectors.toList());
+ }
}
diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestUtils.java
index a11e65dc78045..f9cb49f14220d 100644
--- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestUtils.java
+++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestUtils.java
@@ -35,16 +35,23 @@
import com.esotericsoftware.kryo.serializers.JavaSerializer;
import org.apache.hadoop.conf.Configuration;
+import java.io.BufferedOutputStream;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
+import java.io.File;
import java.io.IOException;
+import java.io.InputStream;
import java.io.Serializable;
+import java.nio.file.Files;
+import java.nio.file.Path;
import java.util.ArrayList;
import java.util.List;
import java.util.Objects;
import java.util.Properties;
import java.util.UUID;
import org.junit.jupiter.api.Assumptions;
+import java.util.zip.ZipEntry;
+import java.util.zip.ZipInputStream;
/**
* A utility class for testing.
@@ -78,7 +85,7 @@ public static HoodieTableMetaClient init(String basePath, HoodieTableType tableT
}
public static HoodieTableMetaClient init(String basePath, HoodieTableType tableType, String bootstrapBasePath, boolean bootstrapIndexEnable, String keyGenerator,
- String partitionFieldConfigValue) throws IOException {
+ String partitionFieldConfigValue) throws IOException {
Properties props = new Properties();
props.setProperty(HoodieTableConfig.BOOTSTRAP_BASE_PATH.key(), bootstrapBasePath);
props.put(HoodieTableConfig.BOOTSTRAP_INDEX_ENABLE.key(), bootstrapIndexEnable);
@@ -256,6 +263,49 @@ public static DistributedFileSystem useExternalHdfs() throws IOException {
return (DistributedFileSystem) DistributedFileSystem.get(conf);
}
+ /**
+ * Extracts a ZIP file from resources to a target directory.
+ *
+ * @param resourcePath the path to the ZIP resource (relative to classpath)
+ * @param targetDirectory the target directory to extract files to
+ * @param resourceClass the class to use for resource loading
+ * @throws IOException if extraction fails
+ */
+ public static void extractZipToDirectory(String resourcePath, Path targetDirectory, Class> resourceClass) throws IOException {
+ InputStream resourceStream = resourceClass.getClassLoader().getResourceAsStream(resourcePath);
+ if (resourceStream == null) {
+ // Fallback to getResourceAsStream if getClassLoader().getResourceAsStream() fails
+ resourceStream = resourceClass.getResourceAsStream(resourcePath);
+ }
+
+ if (resourceStream == null) {
+ throw new IOException("Resource not found at: " + resourcePath);
+ }
+
+ try (ZipInputStream zip = new ZipInputStream(resourceStream)) {
+ ZipEntry entry;
+ while ((entry = zip.getNextEntry()) != null) {
+ File file = targetDirectory.resolve(entry.getName()).toFile();
+ if (entry.isDirectory()) {
+ file.mkdirs();
+ continue;
+ }
+
+ // Create parent directories if they don't exist
+ file.getParentFile().mkdirs();
+
+ // Extract file content
+ byte[] buffer = new byte[10000];
+ try (BufferedOutputStream out = new BufferedOutputStream(Files.newOutputStream(file.toPath()))) {
+ int count;
+ while ((count = zip.read(buffer)) != -1) {
+ out.write(buffer, 0, count);
+ }
+ }
+ }
+ }
+ }
+
public static HoodieTableMetaClient createMetaClient(Configuration conf, String basePath) {
return HoodieTableMetaClient.builder().setConf(conf).setBasePath(basePath).build();
}
diff --git a/hudi-hadoop-common/pom.xml b/hudi-hadoop-common/pom.xml
new file mode 100644
index 0000000000000..a1bf50f5dadbb
--- /dev/null
+++ b/hudi-hadoop-common/pom.xml
@@ -0,0 +1,169 @@
+
+
+
+
+ hudi
+ org.apache.hudi
+ 0.14.2-SNAPSHOT
+
+ 4.0.0
+
+ hudi-hadoop-common
+
+
+ ${project.parent.basedir}
+
+
+
+
+
+ src/main/resources
+
+
+
+
+
+ org.apache.maven.plugins
+ maven-jar-plugin
+ ${maven-jar-plugin.version}
+
+
+
+ test-jar
+
+ test-compile
+
+
+
+ false
+
+
+
+ org.apache.rat
+ apache-rat-plugin
+
+
+ org.jacoco
+ jacoco-maven-plugin
+
+
+ org.codehaus.mojo
+ build-helper-maven-plugin
+ 3.5.0
+
+
+ add-spark34plus-avro-sources
+ generate-sources
+
+ add-source
+
+
+ ${spark33orEarlier}
+
+ src/avro/java
+ src/parquet/java
+
+
+
+
+ add-spark34plus-parquet-test-sources
+ generate-test-sources
+
+ add-test-source
+
+
+ ${spark33orEarlier}
+
+ src/parquet/test/java
+
+
+
+
+
+
+
+
+
+
+ org.apache.hudi
+ hudi-common
+ ${project.version}
+
+
+
+
+ org.apache.hadoop
+ hadoop-client
+
+
+ javax.servlet
+ *
+
+
+ provided
+
+
+ org.apache.hadoop
+ hadoop-hdfs
+ provided
+
+
+
+ org.apache.parquet
+ parquet-avro
+
+
+
+
+ com.esotericsoftware
+ kryo-shaded
+
+
+
+ org.apache.hudi
+ hudi-tests-common
+ ${project.version}
+ test
+
+
+
+ org.apache.hudi
+ hudi-io
+ tests
+ ${project.version}
+ test
+
+
+ org.apache.hudi
+ hudi-common
+ ${project.version}
+ tests
+ test-jar
+ test
+
+
+ com.github.stefanbirkner
+ system-rules
+ 1.17.2
+ test
+
+
+
diff --git a/hudi-hadoop-common/src/test/resources/parquet-java/all.avsc b/hudi-hadoop-common/src/test/resources/parquet-java/all.avsc
new file mode 100644
index 0000000000000..116e98c519de4
--- /dev/null
+++ b/hudi-hadoop-common/src/test/resources/parquet-java/all.avsc
@@ -0,0 +1,110 @@
+/*
+ * 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.
+ */
+{
+ "name" : "myrecord",
+ "namespace": "org.apache.parquet.avro",
+ "type" : "record",
+ "fields" : [ {
+ "name" : "mynull",
+ "type" : "null"
+ }, {
+ "name" : "myboolean",
+ "type" : "boolean"
+ }, {
+ "name" : "myint",
+ "type" : "int"
+ }, {
+ "name" : "mylong",
+ "type" : "long"
+ }, {
+ "name" : "myfloat",
+ "type" : "float"
+ }, {
+ "name" : "mydouble",
+ "type" : "double"
+ }, {
+ "name" : "mybytes",
+ "type" : "bytes"
+ }, {
+ "name" : "mystring",
+ "type" : "string"
+ }, {
+ "name" : "mynestedrecord",
+ "type" : {
+ "type" : "record",
+ "name" : "ignored1",
+ "namespace" : "",
+ "fields" : [ {
+ "name" : "mynestedint",
+ "type" : "int"
+ } ]
+ }
+ }, {
+ "name" : "myenum",
+ "type" : {
+ "type" : "enum",
+ "name" : "ignored2",
+ "namespace" : "",
+ "symbols" : [ "a", "b" ]
+ }
+ }, {
+ "name" : "myarray",
+ "type" : {
+ "type" : "array",
+ "items" : "int"
+ }
+ }, {
+ "name" : "myemptyarray",
+ "type" : {
+ "type" : "array",
+ "items" : "int"
+ }
+ }, {
+ "name" : "myoptionalarray",
+ "type" : [ "null", {
+ "type" : "array",
+ "items" : "int"
+ }]
+ }, {
+ "name" : "myarrayofoptional",
+ "type" : {
+ "type" : "array",
+ "items" : [ "null", "int" ]
+ }
+ }, {
+ "name" : "mymap",
+ "type" : {
+ "type" : "map",
+ "values" : "int"
+ }
+ }, {
+ "name" : "myemptymap",
+ "type" : {
+ "type" : "map",
+ "values" : "int"
+ }
+ }, {
+ "name" : "myfixed",
+ "type" : {
+ "type" : "fixed",
+ "name" : "ignored3",
+ "namespace" : "",
+ "size" : 1
+ }
+ } ]
+}
\ No newline at end of file
diff --git a/hudi-hadoop-common/src/test/resources/parquet-java/allFromParquetNewBehavior.avsc b/hudi-hadoop-common/src/test/resources/parquet-java/allFromParquetNewBehavior.avsc
new file mode 100644
index 0000000000000..606213cb16830
--- /dev/null
+++ b/hudi-hadoop-common/src/test/resources/parquet-java/allFromParquetNewBehavior.avsc
@@ -0,0 +1,108 @@
+/*
+ * 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.
+ */
+{
+ "name" : "myrecord",
+ "namespace": "org.apache.parquet.avro",
+ "type" : "record",
+ "fields" : [ {
+ "name" : "myboolean",
+ "type" : "boolean"
+ }, {
+ "name" : "myint",
+ "type" : "int"
+ }, {
+ "name" : "mylong",
+ "type" : "long"
+ }, {
+ "name" : "myfloat",
+ "type" : "float"
+ }, {
+ "name" : "mydouble",
+ "type" : "double"
+ }, {
+ "name" : "mybytes",
+ "type" : "bytes"
+ }, {
+ "name" : "mystring",
+ "type" : "string"
+ }, {
+ "name" : "mynestedrecord",
+ "type" : {
+ "type" : "record",
+ "name" : "mynestedrecord",
+ "namespace" : "",
+ "fields" : [ {
+ "name" : "mynestedint",
+ "type" : "int"
+ } ]
+ }
+ }, {
+ "name" : "myenum",
+ "type" : "string"
+ }, {
+ "name" : "myarray",
+ "type" : {
+ "type" : "array",
+ "items" : "int"
+ }
+ }, {
+ "name" : "myoptionalarray",
+ "type" : [ "null", {
+ "type" : "array",
+ "items" : "int"
+ }],
+ "default" : null
+ }, {
+ "name" : "myarrayofoptional",
+ "type" : {
+ "type" : "array",
+ "items" : ["null", "int"]
+ }
+ }, {
+ "name" : "myrecordarray",
+ "type" : {
+ "type" : "array",
+ "items" : {
+ "type" : "record",
+ "name" : "array",
+ "namespace" : "",
+ "fields" : [ {
+ "name" : "a",
+ "type" : "int"
+ }, {
+ "name" : "b",
+ "type" : "int"
+ } ]
+ }
+ }
+ }, {
+ "name" : "mymap",
+ "type" : {
+ "type" : "map",
+ "values" : "int"
+ }
+ }, {
+ "name" : "myfixed",
+ "type" : {
+ "type" : "fixed",
+ "name" : "myfixed",
+ "namespace" : "",
+ "size" : 1
+ }
+ } ]
+}
\ No newline at end of file
diff --git a/hudi-hadoop-common/src/test/resources/parquet-java/allFromParquetOldBehavior.avsc b/hudi-hadoop-common/src/test/resources/parquet-java/allFromParquetOldBehavior.avsc
new file mode 100644
index 0000000000000..7a98a74633559
--- /dev/null
+++ b/hudi-hadoop-common/src/test/resources/parquet-java/allFromParquetOldBehavior.avsc
@@ -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.
+ */
+{
+ "name" : "myrecord",
+ "namespace": "org.apache.parquet.avro",
+ "type" : "record",
+ "fields" : [ {
+ "name" : "myboolean",
+ "type" : "boolean"
+ }, {
+ "name" : "myint",
+ "type" : "int"
+ }, {
+ "name" : "mylong",
+ "type" : "long"
+ }, {
+ "name" : "myfloat",
+ "type" : "float"
+ }, {
+ "name" : "mydouble",
+ "type" : "double"
+ }, {
+ "name" : "mybytes",
+ "type" : "bytes"
+ }, {
+ "name" : "mystring",
+ "type" : "string"
+ }, {
+ "name" : "mynestedrecord",
+ "type" : {
+ "type" : "record",
+ "name" : "mynestedrecord",
+ "namespace" : "",
+ "fields" : [ {
+ "name" : "mynestedint",
+ "type" : "int"
+ } ]
+ }
+ }, {
+ "name" : "myenum",
+ "type" : "string"
+ }, {
+ "name" : "myarray",
+ "type" : {
+ "type" : "array",
+ "items" : "int"
+ }
+ }, {
+ "name" : "myoptionalarray",
+ "type" : [ "null", {
+ "type" : "array",
+ "items" : "int"
+ }],
+ "default" : null
+ }, {
+ "name" : "myarrayofoptional",
+ "type" : {
+ "type" : "array",
+ "items" : {
+ "type": "record",
+ "name": "list",
+ "namespace": "",
+ "fields": [ {
+ "name": "element",
+ "type": ["null", "int"],
+ "default": null
+ } ]
+ }
+ }
+ }, {
+ "name" : "myrecordarray",
+ "type" : {
+ "type" : "array",
+ "items" : {
+ "type" : "record",
+ "name" : "array",
+ "namespace" : "",
+ "fields" : [ {
+ "name" : "a",
+ "type" : "int"
+ }, {
+ "name" : "b",
+ "type" : "int"
+ } ]
+ }
+ }
+ }, {
+ "name" : "mymap",
+ "type" : {
+ "type" : "map",
+ "values" : "int"
+ }
+ }, {
+ "name" : "myfixed",
+ "type" : {
+ "type" : "fixed",
+ "name" : "myfixed",
+ "namespace" : "",
+ "size" : 1
+ }
+ } ]
+}
\ No newline at end of file
diff --git a/hudi-hadoop-common/src/test/resources/parquet-java/fixedToInt96.avsc b/hudi-hadoop-common/src/test/resources/parquet-java/fixedToInt96.avsc
new file mode 100644
index 0000000000000..ca1e505ec3380
--- /dev/null
+++ b/hudi-hadoop-common/src/test/resources/parquet-java/fixedToInt96.avsc
@@ -0,0 +1,97 @@
+/*
+ * 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.
+ */
+{
+ "name": "fixedToInt96",
+ "namespace": "org.apache.parquet.avro",
+ "type": "record",
+ "fields": [
+ {
+ "name": "int96",
+ "type": {
+ "type": "fixed",
+ "name": "ignored1",
+ "namespace": "",
+ "size": 12
+ }
+ },
+ {
+ "name": "notanint96",
+ "type": {
+ "type": "fixed",
+ "name": "ignored2",
+ "namespace": "",
+ "size": 12
+ }
+ },
+ {
+ "name": "mynestedrecord",
+ "type": {
+ "type": "record",
+ "name": "ignored3",
+ "namespace": "",
+ "fields": [
+ {
+ "name": "int96inrecord",
+ "type": {
+ "type": "fixed",
+ "name": "ignored4",
+ "namespace": "",
+ "size": 12
+ }
+ },
+ {
+ "name": "myarrayofoptional",
+ "type": {
+ "type": "array",
+ "items": [
+ "null",
+ {
+ "type": "fixed",
+ "name": "ignored5",
+ "namespace": "",
+ "size": 12
+ }
+ ]
+ }
+ },
+ {
+ "name": "mymap",
+ "type": {
+ "type": "map",
+ "values": {
+ "type": "fixed",
+ "name": "ignored6",
+ "namespace": "",
+ "size": 12
+ }
+ }
+ }
+ ]
+ }
+ },
+ {
+ "name": "onebytefixed",
+ "type": {
+ "type": "fixed",
+ "name": "ignored7",
+ "namespace": "",
+ "size": 1
+ }
+ }
+ ]
+}
\ No newline at end of file
diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieParquetInputFormat.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieParquetInputFormat.java
index 9e6565299040b..fc9290a5c3742 100644
--- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieParquetInputFormat.java
+++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieParquetInputFormat.java
@@ -19,6 +19,7 @@
package org.apache.hudi.hadoop;
import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.hadoop.utils.HoodieHiveUtils;
import org.apache.hudi.exception.HoodieException;
@@ -126,7 +127,7 @@ private RecordReader getRecordReaderInternal(InputS
Reporter reporter) throws IOException {
try {
if (supportAvroRead && HoodieColumnProjectionUtils.supportTimestamp(job)) {
- return new ParquetRecordReaderWrapper(new HoodieTimestampAwareParquetInputFormat(), split, job, reporter);
+ return new ParquetRecordReaderWrapper(new HoodieTimestampAwareParquetInputFormat(Option.empty(), Option.empty()), split, job, reporter);
} else {
return super.getRecordReader(split, job, reporter);
}
diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/SchemaEvolutionContext.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/SchemaEvolutionContext.java
index 746066e1c1c74..b8d0241881916 100644
--- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/SchemaEvolutionContext.java
+++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/SchemaEvolutionContext.java
@@ -252,6 +252,9 @@ private TypeInfo constructHiveSchemaFromType(Type type, TypeInfo typeInfo) {
case DOUBLE:
case DATE:
case TIMESTAMP:
+ case TIMESTAMP_MILLIS:
+ case LOCAL_TIMESTAMP_MICROS:
+ case LOCAL_TIMESTAMP_MILLIS:
case STRING:
case UUID:
case FIXED:
@@ -259,6 +262,7 @@ private TypeInfo constructHiveSchemaFromType(Type type, TypeInfo typeInfo) {
case DECIMAL:
return typeInfo;
case TIME:
+ case TIME_MILLIS:
throw new UnsupportedOperationException(String.format("cannot convert %s type to hive", type));
default:
LOG.error(String.format("cannot convert unknown type: %s to Hive", type));
diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/avro/HoodieAvroParquetReader.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/avro/HoodieAvroParquetReader.java
index c31041ddc76b0..9aa5887b9cbcc 100644
--- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/avro/HoodieAvroParquetReader.java
+++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/avro/HoodieAvroParquetReader.java
@@ -19,8 +19,12 @@
package org.apache.hudi.hadoop.avro;
import org.apache.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.common.util.Option;
import org.apache.hudi.hadoop.HoodieColumnProjectionUtils;
import org.apache.hudi.hadoop.utils.HoodieRealtimeRecordReaderUtils;
+import org.apache.hudi.internal.schema.InternalSchema;
+import org.apache.hudi.internal.schema.action.InternalSchemaMerger;
+import org.apache.hudi.internal.schema.convert.AvroInternalSchemaConverter;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericData;
@@ -31,7 +35,6 @@
import org.apache.hadoop.mapreduce.RecordReader;
import org.apache.hadoop.mapreduce.TaskAttemptContext;
import org.apache.parquet.avro.AvroReadSupport;
-import org.apache.parquet.avro.AvroSchemaConverter;
import org.apache.parquet.format.converter.ParquetMetadataConverter;
import org.apache.parquet.hadoop.ParquetFileReader;
import org.apache.parquet.hadoop.ParquetInputSplit;
@@ -43,6 +46,7 @@
import java.util.Arrays;
import java.util.List;
+import static org.apache.parquet.avro.HoodieAvroParquetSchemaConverter.getAvroSchemaConverter;
import static org.apache.parquet.hadoop.ParquetInputFormat.getFilter;
public class HoodieAvroParquetReader extends RecordReader {
@@ -55,7 +59,7 @@ public HoodieAvroParquetReader(InputSplit inputSplit, Configuration conf) throws
ParquetMetadata fileFooter =
ParquetFileReader.readFooter(conf, ((ParquetInputSplit) inputSplit).getPath(), ParquetMetadataConverter.NO_FILTER);
MessageType messageType = fileFooter.getFileMetaData().getSchema();
- baseSchema = new AvroSchemaConverter(conf).convert(messageType);
+ baseSchema = getAvroSchemaConverter(conf).convert(messageType);
// if exists read columns, we need to filter columns.
List readColNames = Arrays.asList(HoodieColumnProjectionUtils.getReadColumnNames(conf));
@@ -64,7 +68,37 @@ public HoodieAvroParquetReader(InputSplit inputSplit, Configuration conf) throws
AvroReadSupport.setAvroReadSchema(conf, filterSchema);
AvroReadSupport.setRequestedProjection(conf, filterSchema);
}
+ parquetRecordReader = new ParquetRecordReader<>(new AvroReadSupport<>(), getFilter(conf));
+ }
+ public HoodieAvroParquetReader(InputSplit inputSplit, Configuration conf, Option internalSchemaOption, Option dataSchema) throws IOException {
+ if (dataSchema.isPresent()) {
+ baseSchema = dataSchema.get();
+ } else {
+ // get base schema
+ ParquetMetadata fileFooter =
+ ParquetFileReader.readFooter(conf, ((ParquetInputSplit) inputSplit).getPath(), ParquetMetadataConverter.NO_FILTER);
+ MessageType messageType = fileFooter.getFileMetaData().getSchema();
+ baseSchema = getAvroSchemaConverter(conf).convert(messageType);
+
+ if (internalSchemaOption.isPresent()) {
+ // do schema reconciliation in case there exists read column which is not in the file schema.
+ InternalSchema mergedInternalSchema = new InternalSchemaMerger(
+ AvroInternalSchemaConverter.convert(baseSchema),
+ internalSchemaOption.get(),
+ true,
+ true).mergeSchema();
+ baseSchema = AvroInternalSchemaConverter.convert(mergedInternalSchema, baseSchema.getFullName());
+ }
+
+ // if exists read columns, we need to filter columns.
+ List readColNames = Arrays.asList(HoodieColumnProjectionUtils.getReadColumnNames(conf));
+ if (!readColNames.isEmpty()) {
+ Schema filterSchema = HoodieAvroUtils.generateProjectionSchema(baseSchema, readColNames);
+ AvroReadSupport.setAvroReadSchema(conf, filterSchema);
+ AvroReadSupport.setRequestedProjection(conf, filterSchema);
+ }
+ }
parquetRecordReader = new ParquetRecordReader<>(new AvroReadSupport<>(), getFilter(conf));
}
diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/avro/HoodieTimestampAwareParquetInputFormat.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/avro/HoodieTimestampAwareParquetInputFormat.java
index 8f9aae530e412..4c55fe26551f7 100644
--- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/avro/HoodieTimestampAwareParquetInputFormat.java
+++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/avro/HoodieTimestampAwareParquetInputFormat.java
@@ -18,6 +18,10 @@
package org.apache.hudi.hadoop.avro;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.internal.schema.InternalSchema;
+
+import org.apache.avro.Schema;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.io.ArrayWritable;
import org.apache.hadoop.mapreduce.InputSplit;
@@ -33,6 +37,14 @@
* we need to handle timestamp types separately based on the parquet-avro approach.
*/
public class HoodieTimestampAwareParquetInputFormat extends ParquetInputFormat {
+ private final Option internalSchemaOption;
+ private final Option dataSchema;
+
+ public HoodieTimestampAwareParquetInputFormat(Option internalSchemaOption, Option dataSchema) {
+ super();
+ this.internalSchemaOption = internalSchemaOption;
+ this.dataSchema = dataSchema;
+ }
@Override
public RecordReader createRecordReader(
diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HiveAvroSerializer.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HiveAvroSerializer.java
index a0d1b086e0357..7c11467fd30c3 100644
--- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HiveAvroSerializer.java
+++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HiveAvroSerializer.java
@@ -32,6 +32,7 @@
import org.apache.hadoop.hive.common.type.HiveChar;
import org.apache.hadoop.hive.common.type.HiveDecimal;
import org.apache.hadoop.hive.common.type.HiveVarchar;
+import org.apache.hadoop.hive.ql.io.parquet.serde.ArrayWritableObjectInspector;
import org.apache.hadoop.hive.serde2.avro.AvroSerdeUtils;
import org.apache.hadoop.hive.serde2.avro.InstanceCache;
import org.apache.hadoop.hive.serde2.objectinspector.ListObjectInspector;
@@ -61,7 +62,7 @@
import java.util.Set;
import java.util.stream.Collectors;
-import static org.apache.hudi.avro.AvroSchemaUtils.resolveNullableSchema;
+import static org.apache.hudi.avro.AvroSchemaUtils.getNonNullTypeFromUnion;
import static org.apache.hudi.avro.AvroSchemaUtils.resolveUnionSchema;
import static org.apache.hudi.avro.HoodieAvroUtils.isMetadataField;
@@ -76,7 +77,7 @@ public class HiveAvroSerializer {
private static final Logger LOG = LoggerFactory.getLogger(HiveAvroSerializer.class);
- public HiveAvroSerializer(ObjectInspector objectInspector, List columnNames, List columnTypes) {
+ public HiveAvroSerializer(ArrayWritableObjectInspector objectInspector, List