Skip to content

[HUDI-9172] Fix timestamp millis being read as micros for log files #13288

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Draft
wants to merge 1 commit into
base: master
Choose a base branch
from
Draft
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -48,6 +48,7 @@
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;
Expand Down Expand Up @@ -101,6 +102,7 @@
import java.util.stream.Collectors;

import static org.apache.avro.Schema.Type.ARRAY;
import static org.apache.avro.Schema.Type.FLOAT;
import static org.apache.avro.Schema.Type.MAP;
import static org.apache.avro.Schema.Type.UNION;
import static org.apache.hudi.avro.AvroSchemaUtils.createNewSchemaFromFieldsWithReference;
Expand Down Expand Up @@ -1321,6 +1323,64 @@ private static boolean needsRewriteToString(Schema schema, boolean isEnum) {
}
}

/**
* Used to repair the schema after conversion from StructType to avro. Timestamp logical types are assumed
* to be micros so we use the table schema to repair that field
*/
public static Schema repairSchema(Schema convertedSchema, Schema tableSchema) {
if (convertedSchema.getType() == tableSchema.getType()) {
if (convertedSchema.getType() == Schema.Type.RECORD) {
List<Schema.Field> fields = new ArrayList<>(convertedSchema.getFields().size());
for (Schema.Field convertedField : convertedSchema.getFields()) {
Schema.Field tableField = tableSchema.getField(convertedField.name());
if (tableField == null) {
throw new IllegalArgumentException("Missing field: " + convertedField.name());
}
fields.add(new Schema.Field(
tableField.name(),
repairSchema(convertedField.schema(), tableField.schema()),
tableField.doc(),
tableField.defaultVal()
));
}
return createNewSchemaFromFieldsWithReference(convertedSchema, fields);
} else if (convertedSchema.getType() == Schema.Type.ARRAY) {
return Schema.createArray(repairSchema(convertedSchema.getElementType(), tableSchema.getElementType()));
} else if (convertedSchema.getType() == Schema.Type.MAP) {
return Schema.createMap(repairSchema(convertedSchema.getValueType(), tableSchema.getValueType()));
} else if (convertedSchema.getType() == Schema.Type.UNION) {
List<Schema> sourceNestedSchemas = convertedSchema.getTypes();
List<Schema> targetNestedSchemas = tableSchema.getTypes();
if (sourceNestedSchemas.size() != targetNestedSchemas.size()) {
throw new IllegalArgumentException("Union sizes do not match.");
}
List<Schema> types = new ArrayList<>(sourceNestedSchemas.size());
for (int i = 0; i < sourceNestedSchemas.size(); i++) {
types.add(repairSchema(sourceNestedSchemas.get(i), targetNestedSchemas.get(i)));
}
return Schema.createUnion(types);
}
}

// Only repair timestamp-millis <-> timestamp-micros logical types
if (convertedSchema.getType() == Schema.Type.LONG && tableSchema.getType() == Schema.Type.LONG) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We can also fix the Enum -> String -> Enum issue here. I think the logically they are the same. CC: @yihua , @nsivabalan

LogicalType convertedLogical = convertedSchema.getLogicalType();
LogicalType tableLogical = tableSchema.getLogicalType();

boolean isConvertedTimestampMillis = LogicalTypes.timestampMillis().equals(convertedLogical);
boolean isConvertedTimestampMicros = LogicalTypes.timestampMicros().equals(convertedLogical);
boolean isTableTimestampMillis = LogicalTypes.timestampMillis().equals(tableLogical);
boolean isTableTimestampMicros = LogicalTypes.timestampMicros().equals(tableLogical);

if ((isConvertedTimestampMillis && isTableTimestampMicros)
|| (isConvertedTimestampMicros && isTableTimestampMillis)) {
return tableSchema;
}
}

return convertedSchema;
}

/**
* convert days to Date
* <p>
Expand Down
124 changes: 124 additions & 0 deletions hudi-common/src/test/java/org/apache/hudi/avro/TestSchemaRepair.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,124 @@
/*
* 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 org.apache.avro.LogicalTypes;
import org.apache.avro.Schema;
import org.junit.jupiter.api.Test;

import java.util.Arrays;
import java.util.Collections;

import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertThrows;

public class TestSchemaRepair {

private final Schema longSchema = Schema.create(Schema.Type.LONG);
private final Schema milliSchema = LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG));
private final Schema microSchema = LogicalTypes.timestampMicros().addToSchema(Schema.create(Schema.Type.LONG));

@Test
public void testRepairTimestampMillisToMicros() {
Schema result = HoodieAvroUtils.repairSchema(milliSchema, microSchema);
assertEquals(Schema.Type.LONG, result.getType());
assertEquals(LogicalTypes.timestampMicros(), result.getLogicalType());
}

@Test
public void testNoChangeIfLogicalTypesAreSame() {
Schema result = HoodieAvroUtils.repairSchema(milliSchema, milliSchema);
assertEquals(milliSchema, result);

result = HoodieAvroUtils.repairSchema(microSchema, microSchema);
assertEquals(microSchema, result);
}

@Test
public void testNoRepairForNonTimestampLogicalTypes() {
Schema result = HoodieAvroUtils.repairSchema(longSchema, Schema.create(Schema.Type.INT));
assertEquals(longSchema, result);
}

@Test
void testRecordTypeRepair() {
Schema source = Schema.createRecord("TestRecord", null, null, false);
Schema target = Schema.createRecord("TestRecord", null, null, false);

Schema.Field sourceField = new Schema.Field("id", microSchema, null, null);
Schema.Field targetField = new Schema.Field("id", milliSchema, null, null);

source.setFields(Collections.singletonList(sourceField));
target.setFields(Collections.singletonList(targetField));

Schema result = HoodieAvroUtils.repairSchema(source, target);

assertEquals("TestRecord", result.getName());
assertEquals(Schema.Type.LONG, result.getField("id").schema().getType());
assertEquals(LogicalTypes.timestampMillis(), result.getField("id").schema().getLogicalType());
}

@Test
void testArrayTypeRepair() {
Schema source = Schema.createArray(milliSchema);
Schema target = Schema.createArray(microSchema);

Schema result = HoodieAvroUtils.repairSchema(source, target);
assertEquals(Schema.Type.ARRAY, result.getType());
assertEquals(Schema.Type.LONG, result.getElementType().getType());
assertEquals(LogicalTypes.timestampMicros(), result.getElementType().getLogicalType());
}

@Test
void testMapTypeRepair() {
Schema source = Schema.createMap(microSchema);
Schema target = Schema.createMap(milliSchema);

Schema result = HoodieAvroUtils.repairSchema(source, target);
assertEquals(Schema.Type.MAP, result.getType());
assertEquals(Schema.Type.LONG, result.getValueType().getType());
assertEquals(LogicalTypes.timestampMillis(), result.getValueType().getLogicalType());
}

@Test
void testUnionTypeRepair() {
Schema source = Schema.createUnion(Arrays.asList(Schema.create(Schema.Type.NULL), milliSchema));
Schema target = Schema.createUnion(Arrays.asList(Schema.create(Schema.Type.NULL), microSchema));

Schema result = HoodieAvroUtils.repairSchema(source, target);
assertEquals(2, result.getTypes().size());
assertEquals(Schema.Type.NULL, result.getTypes().get(0).getType());
assertEquals(Schema.Type.LONG, result.getTypes().get(1).getType());
assertEquals(LogicalTypes.timestampMicros(), result.getTypes().get(1).getLogicalType());
}

@Test
void testFieldMissingThrowsException() {
Schema source = Schema.createRecord("Test", null, null, false);
Schema target = Schema.createRecord("Test", null, null, false);

source.setFields(Collections.singletonList(new Schema.Field("fieldA", Schema.create(Schema.Type.STRING), null, null)));
target.setFields(Collections.emptyList()); // Missing field

assertThrows(IllegalArgumentException.class, () -> {
HoodieAvroUtils.repairSchema(source, target);
});
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -297,7 +297,7 @@ class HoodieMergeOnReadSnapshotHadoopFsRelationFactory(override val sqlContext:
new HoodieFileGroupReaderBasedParquetFileFormat(basePath.toString,
HoodieTableSchema(tableStructSchema, tableAvroSchema.toString, internalSchemaOpt),
metaClient.getTableConfig.getTableName, queryTimestamp.get, mandatoryFields, true, isBootstrap,
false, fileIndex.isInstanceOf[HoodieCDCFileIndex], validCommits, shouldUseRecordPosition, Seq.empty)
false, tableAvroSchema, validCommits, shouldUseRecordPosition, Seq.empty)
}
}

Expand Down Expand Up @@ -341,7 +341,7 @@ class HoodieMergeOnReadIncrementalHadoopFsRelationFactory(override val sqlContex
new HoodieFileGroupReaderBasedParquetFileFormat(
basePath.toString, HoodieTableSchema(tableStructSchema, tableAvroSchema.toString, internalSchemaOpt),
metaClient.getTableConfig.getTableName, queryTimestamp.get, mandatoryFields,
true, isBootstrap, true, fileIndex.isInstanceOf[HoodieCDCFileIndex],
true, isBootstrap, true, tableAvroSchema,
validCommits, shouldUseRecordPosition, fileIndex.getRequiredFilters)
}
}
Expand Down Expand Up @@ -373,7 +373,7 @@ class HoodieCopyOnWriteSnapshotHadoopFsRelationFactory(override val sqlContext:
new HoodieFileGroupReaderBasedParquetFileFormat(
basePath.toString, HoodieTableSchema(tableStructSchema, tableAvroSchema.toString, internalSchemaOpt),
metaClient.getTableConfig.getTableName, queryTimestamp.get, mandatoryFields,
false, isBootstrap, false, fileIndex.isInstanceOf[HoodieCDCFileIndex], validCommits,
false, isBootstrap, false, tableAvroSchema, validCommits,
shouldUseRecordPosition, Seq.empty)
}
}
Expand Down Expand Up @@ -402,7 +402,7 @@ class HoodieCopyOnWriteIncrementalHadoopFsRelationFactory(override val sqlContex
new HoodieFileGroupReaderBasedParquetFileFormat(
basePath.toString, HoodieTableSchema(tableStructSchema, tableAvroSchema.toString, internalSchemaOpt),
metaClient.getTableConfig.getTableName, queryTimestamp.get, mandatoryFields,
false, isBootstrap, true, fileIndex.isInstanceOf[HoodieCDCFileIndex],
false, isBootstrap, true, tableAvroSchema,
validCommits, shouldUseRecordPosition, fileIndex.getRequiredFilters)
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@
package org.apache.spark.sql.execution.datasources.parquet

import org.apache.hudi.{AvroConversionUtils, HoodieFileIndex, HoodiePartitionCDCFileGroupMapping, HoodiePartitionFileSliceMapping, HoodieTableSchema, HoodieTableState, SparkAdapterSupport, SparkFileFormatInternalRowReaderContext}
import org.apache.hudi.avro.AvroSchemaUtils
import org.apache.hudi.avro.{AvroSchemaUtils, HoodieAvroUtils}
import org.apache.hudi.cdc.{CDCFileGroupIterator, CDCRelation, HoodieCDCFileGroupSplit}
import org.apache.hudi.client.common.HoodieSparkEngineContext
import org.apache.hudi.client.utils.SparkInternalSchemaConverter
Expand All @@ -32,6 +32,7 @@ import org.apache.hudi.io.IOUtils
import org.apache.hudi.storage.StorageConfiguration
import org.apache.hudi.storage.hadoop.{HadoopStorageConfiguration, HoodieHadoopStorage}

import org.apache.avro.Schema
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.Path
import org.apache.spark.api.java.JavaSparkContext
Expand Down Expand Up @@ -70,7 +71,7 @@ class HoodieFileGroupReaderBasedParquetFileFormat(tablePath: String,
isMOR: Boolean,
isBootstrap: Boolean,
isIncremental: Boolean,
isCDC: Boolean,
avroTableSchema: Schema,
validCommits: String,
shouldUseRecordPosition: Boolean,
requiredFilters: Seq[Filter]
Expand Down Expand Up @@ -152,8 +153,8 @@ class HoodieFileGroupReaderBasedParquetFileFormat(tablePath: String,

// schema that we want fg reader to output to us
val requestedSchema = StructType(requiredSchema.fields ++ partitionSchema.fields.filter(f => mandatoryFields.contains(f.name)))
val requestedAvroSchema = AvroConversionUtils.convertStructTypeToAvroSchema(requestedSchema, sanitizedTableName)
val dataAvroSchema = AvroConversionUtils.convertStructTypeToAvroSchema(dataSchema, sanitizedTableName)
val requestedAvroSchema = HoodieAvroUtils.repairSchema(AvroConversionUtils.convertStructTypeToAvroSchema(requestedSchema, sanitizedTableName), avroTableSchema)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Instead of constructing a schema that needs repair, is it possible to just incorporate this into the initial conversion?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1, why not just fix the original conversion, is the Spark StructType missed something?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

yes, it only uses microseconds. Also it's missing enum type as well

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hmm, I think StructType at least suppors timestamp with explicit precision? like timestamp(3) or timestamp(9)? Did you try that?

val dataAvroSchema = HoodieAvroUtils.repairSchema(AvroConversionUtils.convertStructTypeToAvroSchema(dataSchema, sanitizedTableName), avroTableSchema)
val parquetFileReader = spark.sparkContext.broadcast(sparkAdapter.createParquetFileReader(supportBatchResult,
spark.sessionState.conf, options, augmentedStorageConf.unwrap()))
val broadcastedStorageConf = spark.sparkContext.broadcast(new SerializableConfiguration(augmentedStorageConf.unwrap()))
Expand Down
Loading