-
Notifications
You must be signed in to change notification settings - Fork 2.4k
[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
base: master
Are you sure you want to change the base?
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
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 |
---|---|---|
|
@@ -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 | ||
|
@@ -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 | ||
|
@@ -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] | ||
|
@@ -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) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. +1, why not just fix the original conversion, is the Spark There was a problem hiding this comment. Choose a reason for hiding this commentThe 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 There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Hmm, I think |
||
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())) | ||
|
There was a problem hiding this comment.
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