diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestRepairsCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestRepairsCommand.java index 10f3f8962d3b9..443ab1e8fac11 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestRepairsCommand.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestRepairsCommand.java @@ -104,7 +104,7 @@ public void init() throws Exception { "", HoodieTableVersion.current().versionCode(), "org.apache.hudi.common.model.HoodieAvroPayload"); - HoodieSparkWriteableTestTable cowTable = HoodieSparkWriteableTestTable.of(HoodieCLI.getTableMetaClient(), schema.toAvroSchema()); + HoodieSparkWriteableTestTable cowTable = HoodieSparkWriteableTestTable.of(HoodieCLI.getTableMetaClient(), schema); cowTable.addCommit("20160401010101") .withInserts(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, "1", hoodieRecords1) @@ -127,7 +127,7 @@ public void init() throws Exception { morTablePath, "mor_table", HoodieTableType.MERGE_ON_READ.name(), "", HoodieTableVersion.current().versionCode(), "org.apache.hudi.common.model.HoodieAvroPayload"); - HoodieSparkWriteableTestTable morTable = HoodieSparkWriteableTestTable.of(HoodieCLI.getTableMetaClient(), schema.toAvroSchema()); + HoodieSparkWriteableTestTable morTable = HoodieSparkWriteableTestTable.of(HoodieCLI.getTableMetaClient(), schema); morTable.addDeltaCommit("20160401010101"); morTable.withInserts(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, "1", hoodieRecords1) @@ -151,7 +151,7 @@ public void init() throws Exception { "", HoodieTableVersion.current().versionCode(), "org.apache.hudi.common.model.HoodieAvroPayload"); - HoodieSparkWriteableTestTable cowNonPartitionedTable = HoodieSparkWriteableTestTable.of(HoodieCLI.getTableMetaClient(), schema.toAvroSchema()); + HoodieSparkWriteableTestTable cowNonPartitionedTable = HoodieSparkWriteableTestTable.of(HoodieCLI.getTableMetaClient(), schema); cowNonPartitionedTable.addCommit("20160401010101") .withInserts(HoodieTestDataGenerator.NO_PARTITION_PATH, "1", hoodieRecords1) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieBinaryCopyHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieBinaryCopyHandle.java index d851c5feee6c2..94a86f1f94762 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieBinaryCopyHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieBinaryCopyHandle.java @@ -69,7 +69,7 @@ private MessageType getWriteSchema(HoodieWriteConfig config, List i // All files should have the same schema in this case try { ParquetUtils parquetUtils = new ParquetUtils(); - MessageType fileSchema = parquetUtils.readSchema(table.getStorage(), inputFiles.get(0)); + MessageType fileSchema = parquetUtils.readMessageType(table.getStorage(), inputFiles.get(0)); log.info("Binary copy schema evolution disabled. Using schema from input file: " + inputFiles.get(0)); return fileSchema; } catch (Exception e) { @@ -79,7 +79,7 @@ private MessageType getWriteSchema(HoodieWriteConfig config, List i } } else { // Default behavior: use the table's write schema for evolution - return getAvroSchemaConverter(conf).convert(writeSchemaWithMetaFields.toAvroSchema()); + return getAvroSchemaConverter(conf).convert(writeSchemaWithMetaFields); } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/SecondaryIndexRecordGenerationUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/SecondaryIndexRecordGenerationUtils.java index df52b99a90b66..6b8e1107e6241 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/SecondaryIndexRecordGenerationUtils.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/SecondaryIndexRecordGenerationUtils.java @@ -54,8 +54,6 @@ import org.apache.hudi.storage.StoragePath; import org.apache.hudi.storage.StoragePathInfo; -import org.apache.avro.Schema; - import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; @@ -248,9 +246,9 @@ public static HoodieData readSecondaryKeysFromFileSlices(Hoodi } final int parallelism = Math.min(partitionFileSlicePairs.size(), secondaryIndexMaxParallelism); final StoragePath basePath = metaClient.getBasePath(); - Schema tableSchema; + HoodieSchema tableSchema; try { - tableSchema = new TableSchemaResolver(metaClient).getTableAvroSchema(); + tableSchema = new TableSchemaResolver(metaClient).getTableSchema(); } catch (Exception e) { throw new HoodieException("Failed to get latest schema for " + metaClient.getBasePath(), e); } @@ -261,16 +259,16 @@ public static HoodieData readSecondaryKeysFromFileSlices(Hoodi final String partition = partitionAndBaseFile.getKey(); final FileSlice fileSlice = partitionAndBaseFile.getValue(); Option dataFilePath = Option.ofNullable(fileSlice.getBaseFile().map(baseFile -> filePath(basePath, partition, baseFile.getFileName())).orElseGet(null)); - Schema readerSchema; + HoodieSchema readerSchema; if (dataFilePath.isPresent()) { readerSchema = HoodieIOFactory.getIOFactory(metaClient.getStorage()) .getFileFormatUtils(baseFileFormat) - .readAvroSchema(metaClient.getStorage(), dataFilePath.get()); + .readSchema(metaClient.getStorage(), dataFilePath.get()); } else { readerSchema = tableSchema; } ClosableIterator> secondaryIndexGenerator = createSecondaryIndexRecordGenerator( - readerContextFactory.getContext(), metaClient, fileSlice, HoodieSchema.fromAvroSchema(readerSchema), indexDefinition, + readerContextFactory.getContext(), metaClient, fileSlice, readerSchema, indexDefinition, metaClient.getActiveTimeline().filterCompletedInstants().lastInstant().map(HoodieInstant::requestedTime).orElse(""), props, false); return new CloseableMappingIterator<>(secondaryIndexGenerator, pair -> createSecondaryIndexRecord(pair.getKey(), pair.getValue(), indexDefinition.getIndexName(), false)); }); diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/avro/TestHoodieAvroParquetWriter.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/avro/TestHoodieAvroParquetWriter.java index 59797f58bb3aa..c359ff2e43a97 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/avro/TestHoodieAvroParquetWriter.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/avro/TestHoodieAvroParquetWriter.java @@ -23,6 +23,7 @@ import org.apache.hudi.common.bloom.BloomFilterTypeCode; import org.apache.hudi.common.config.HoodieParquetConfig; import org.apache.hudi.common.engine.LocalTaskContextSupplier; +import org.apache.hudi.common.schema.HoodieSchema; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.testutils.HoodieTestUtils; import org.apache.hudi.common.util.Option; @@ -31,7 +32,6 @@ import org.apache.hudi.storage.HoodieStorage; import org.apache.hudi.storage.StoragePath; -import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.apache.parquet.avro.AvroSchemaConverter; import org.apache.parquet.hadoop.ParquetWriter; @@ -61,11 +61,11 @@ public void testProperWriting() throws IOException { HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator(0xDEED); List records = dataGen.generateGenericRecords(10); - Schema schema = records.get(0).getSchema(); + HoodieSchema schema = HoodieSchema.fromAvroSchema(records.get(0).getSchema()); BloomFilter filter = BloomFilterFactory.createBloomFilter(1000, 0.0001, 10000, BloomFilterTypeCode.DYNAMIC_V0.name()); - HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(new AvroSchemaConverter().convert(schema), + HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(new AvroSchemaConverter().convert(schema.toAvroSchema()), schema, Option.of(filter), new Properties()); HoodieParquetConfig parquetConfig = diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/timeline/TestHoodieTimelineArchiver.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/timeline/TestHoodieTimelineArchiver.java index 51a6c45221570..8d335a828ca5f 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/timeline/TestHoodieTimelineArchiver.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/timeline/TestHoodieTimelineArchiver.java @@ -70,7 +70,7 @@ void archiveIfRequired_instantsAreArchived() throws Exception { .build(); HoodieEngineContext context = new HoodieLocalEngineContext(metaClient.getStorageConf()); HoodieStorage hoodieStorage = HoodieStorageUtils.getStorage(basePath, metaClient.getStorageConf()); - HoodieWriteableTestTable testTable = new HoodieWriteableTestTable(basePath, hoodieStorage, metaClient, SCHEMA.toAvroSchema(), null, null, Option.of(context)); + HoodieWriteableTestTable testTable = new HoodieWriteableTestTable(basePath, hoodieStorage, metaClient, SCHEMA, null, null, Option.of(context)); testTable.addCommit(InProcessTimeGenerator.createNewInstantTime()); testTable.addCommit(InProcessTimeGenerator.createNewInstantTime()); testTable.addCommit(InProcessTimeGenerator.createNewInstantTime()); diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/index/simple/TestGlobalSimpleIndex.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/index/simple/TestGlobalSimpleIndex.java index 682d61ca8ecd9..af92792dbee7a 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/index/simple/TestGlobalSimpleIndex.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/index/simple/TestGlobalSimpleIndex.java @@ -100,7 +100,7 @@ void testTagLocation(boolean manuallySetPartitions) throws Exception { assertFalse(taggedRecordRDD.collectAsList().stream().anyMatch(HoodieRecord::isCurrentLocationKnown)); HoodieStorage hoodieStorage = HoodieStorageUtils.getStorage(basePath, HadoopFSUtils.getStorageConf(conf)); - HoodieWriteableTestTable testTable = new HoodieWriteableTestTable(basePath, hoodieStorage, metaClient, SCHEMA.toAvroSchema(), null, null, Option.of(context)); + HoodieWriteableTestTable testTable = new HoodieWriteableTestTable(basePath, hoodieStorage, metaClient, SCHEMA, null, null, Option.of(context)); String fileId1 = UUID.randomUUID().toString(); String fileId2 = UUID.randomUUID().toString(); diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/index/simple/TestSimpleIndex.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/index/simple/TestSimpleIndex.java index 72af1fb94f04d..7290f1919eb07 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/index/simple/TestSimpleIndex.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/index/simple/TestSimpleIndex.java @@ -101,7 +101,7 @@ void testTagLocation(boolean manuallySetPartitions) throws Exception { assertFalse(taggedRecordRDD.collectAsList().stream().anyMatch(HoodieRecord::isCurrentLocationKnown)); HoodieStorage hoodieStorage = HoodieStorageUtils.getStorage(basePath, HadoopFSUtils.getStorageConf(conf)); - HoodieWriteableTestTable testTable = new HoodieWriteableTestTable(basePath, hoodieStorage, metaClient, SCHEMA.toAvroSchema(), null, null, Option.of(context)); + HoodieWriteableTestTable testTable = new HoodieWriteableTestTable(basePath, hoodieStorage, metaClient, SCHEMA, null, null, Option.of(context)); String fileId1 = UUID.randomUUID().toString(); String fileId2 = UUID.randomUUID().toString(); diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/TestHoodieBinaryCopyHandleSchemaEvolution.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/TestHoodieBinaryCopyHandleSchemaEvolution.java index 606ca4881e733..d766ba512d8c1 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/TestHoodieBinaryCopyHandleSchemaEvolution.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/TestHoodieBinaryCopyHandleSchemaEvolution.java @@ -113,7 +113,7 @@ public void testSchemaEvolutionDisabled_UsesFileSchema() throws Exception { // Mock ParquetUtils to return file schema try (MockedConstruction parquetUtilsConstruction = mockConstruction(ParquetUtils.class, (mock, context) -> { - when(mock.readSchema(eq(storage), eq(inputFiles.get(0)))).thenReturn(fileSchema); + when(mock.readMessageType(eq(storage), eq(inputFiles.get(0)))).thenReturn(fileSchema); })) { // When: Creating HoodieBinaryCopyHandle (we can't instantiate directly due to complex dependencies, @@ -201,7 +201,7 @@ public MessageType testGetWriteSchema(HoodieWriteConfig config, List filter; protected final boolean populateMetaFields; protected HoodieWriteableTestTable(String basePath, HoodieStorage storage, HoodieTableMetaClient metaClient, - Schema schema, BloomFilter filter) { + HoodieSchema schema, BloomFilter filter) { this(basePath, storage, metaClient, schema, filter, null); } protected HoodieWriteableTestTable(String basePath, HoodieStorage storage, - HoodieTableMetaClient metaClient, Schema schema, + HoodieTableMetaClient metaClient, HoodieSchema schema, BloomFilter filter, HoodieTableMetadataWriter metadataWriter) { this(basePath, storage, metaClient, schema, filter, metadataWriter, Option.empty()); } public HoodieWriteableTestTable(String basePath, HoodieStorage storage, - HoodieTableMetaClient metaClient, Schema schema, + HoodieTableMetaClient metaClient, HoodieSchema schema, BloomFilter filter, HoodieTableMetadataWriter metadataWriter, Option context) { super(basePath, storage, metaClient, metadataWriter, context); @@ -116,7 +116,7 @@ public StoragePath withInserts(String partition, String fileId, List config = new HoodieParquetConfig<>(writeSupport, CompressionCodecName.GZIP, ParquetWriter.DEFAULT_BLOCK_SIZE, ParquetWriter.DEFAULT_PAGE_SIZE, 120 * 1024 * 1024, storage.getConf(), Double.parseDouble(HoodieStorageConfig.PARQUET_COMPRESSION_RATIO_FRACTION.defaultValue()), true); @@ -125,7 +125,8 @@ public StoragePath withInserts(String partition, String fileId, List result = @@ -88,7 +88,7 @@ public ByteArrayOutputStream getContentBytes(HoodieStorage storage) throws IOExc recordIterator, HoodieRecord.HoodieRecordType.FLINK, writerSchema, - getSchema().toAvroSchema(), + getSchema(), getKeyFieldName(), paramsMap); ValidationUtils.checkArgument(result.getRight() instanceof ParquetMetadata, diff --git a/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/index/bloom/TestFlinkHoodieBloomIndex.java b/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/index/bloom/TestFlinkHoodieBloomIndex.java index 86f85c746bfee..66c87b3ff5078 100644 --- a/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/index/bloom/TestFlinkHoodieBloomIndex.java +++ b/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/index/bloom/TestFlinkHoodieBloomIndex.java @@ -108,7 +108,7 @@ public void testLoadInvolvedFiles(boolean rangePruning, boolean treeFiltering, b HoodieWriteConfig config = makeConfig(rangePruning, treeFiltering, bucketizedChecking); HoodieBloomIndex index = new HoodieBloomIndex(config, ListBasedHoodieBloomIndexHelper.getInstance()); HoodieTable hoodieTable = HoodieFlinkTable.create(config, context, metaClient); - HoodieFlinkWriteableTestTable testTable = HoodieFlinkWriteableTestTable.of(hoodieTable, SCHEMA.toAvroSchema()); + HoodieFlinkWriteableTestTable testTable = HoodieFlinkWriteableTestTable.of(hoodieTable, SCHEMA); // Create some partitions, and put some files // "2016/01/21": 0 file @@ -205,7 +205,7 @@ public void testCheckUUIDsAgainstOneFile() throws Exception { // record2, record3). BloomFilter filter = BloomFilterFactory.createBloomFilter(10000, 0.0000001, -1, BloomFilterTypeCode.SIMPLE.name()); filter.add(record3.getRecordKey()); - HoodieFlinkWriteableTestTable testTable = HoodieFlinkWriteableTestTable.of(metaClient, SCHEMA.toAvroSchema(), filter); + HoodieFlinkWriteableTestTable testTable = HoodieFlinkWriteableTestTable.of(metaClient, SCHEMA, filter); String fileId = testTable.addCommit("000").getFileIdWithInserts(partition, record1, record2); String filename = testTable.getBaseFileNameById(fileId); @@ -266,7 +266,7 @@ public void testTagLocation(boolean rangePruning, boolean treeFiltering, boolean // Also create the metadata and config HoodieWriteConfig config = makeConfig(rangePruning, treeFiltering, bucketizedChecking); HoodieFlinkTable hoodieTable = HoodieFlinkTable.create(config, context, metaClient); - HoodieFlinkWriteableTestTable testTable = HoodieFlinkWriteableTestTable.of(hoodieTable, SCHEMA.toAvroSchema()); + HoodieFlinkWriteableTestTable testTable = HoodieFlinkWriteableTestTable.of(hoodieTable, SCHEMA); // Let's tag HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, ListBasedHoodieBloomIndexHelper.getInstance()); @@ -317,7 +317,7 @@ public void testCheckExists(boolean rangePruning, boolean treeFiltering, boolean // Also create the metadata and config HoodieWriteConfig config = makeConfig(rangePruning, treeFiltering, bucketizedChecking); HoodieTable hoodieTable = HoodieFlinkTable.create(config, context, metaClient); - HoodieFlinkWriteableTestTable testTable = HoodieFlinkWriteableTestTable.of(hoodieTable, SCHEMA.toAvroSchema()); + HoodieFlinkWriteableTestTable testTable = HoodieFlinkWriteableTestTable.of(hoodieTable, SCHEMA); // Let's tag HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, ListBasedHoodieBloomIndexHelper.getInstance()); @@ -384,7 +384,7 @@ public void testBloomFilterFalseError(boolean rangePruning, boolean treeFilterin BloomFilter filter = BloomFilterFactory.createBloomFilter(10000, 0.0000001, -1, BloomFilterTypeCode.SIMPLE.name()); filter.add(record2.getRecordKey()); - HoodieFlinkWriteableTestTable testTable = HoodieFlinkWriteableTestTable.of(metaClient, SCHEMA.toAvroSchema(), filter); + HoodieFlinkWriteableTestTable testTable = HoodieFlinkWriteableTestTable.of(metaClient, SCHEMA, filter); String fileId = testTable.addCommit("000").getFileIdWithInserts("2016/01/31", record1); assertTrue(filter.mightContain(record1.getRecordKey())); assertTrue(filter.mightContain(record2.getRecordKey())); diff --git a/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/testutils/HoodieFlinkWriteableTestTable.java b/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/testutils/HoodieFlinkWriteableTestTable.java index 1405057019a80..09e24fbbb0fe4 100644 --- a/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/testutils/HoodieFlinkWriteableTestTable.java +++ b/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/testutils/HoodieFlinkWriteableTestTable.java @@ -29,6 +29,7 @@ import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordLocation; import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.schema.HoodieSchema; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.log.HoodieLogFormat; import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock; @@ -39,7 +40,6 @@ import org.apache.hudi.table.HoodieTable; import lombok.extern.slf4j.Slf4j; -import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.IndexedRecord; @@ -58,29 +58,29 @@ public class HoodieFlinkWriteableTestTable extends HoodieWriteableTestTable { private HoodieFlinkWriteableTestTable(String basePath, HoodieStorage storage, - HoodieTableMetaClient metaClient, Schema schema, + HoodieTableMetaClient metaClient, HoodieSchema schema, BloomFilter filter) { super(basePath, storage, metaClient, schema, filter); } - public static HoodieFlinkWriteableTestTable of(HoodieTableMetaClient metaClient, Schema schema, + public static HoodieFlinkWriteableTestTable of(HoodieTableMetaClient metaClient, HoodieSchema schema, BloomFilter filter) { return new HoodieFlinkWriteableTestTable(metaClient.getBasePath().toString(), metaClient.getRawStorage(), metaClient, schema, filter); } - public static HoodieFlinkWriteableTestTable of(HoodieTableMetaClient metaClient, Schema schema) { + public static HoodieFlinkWriteableTestTable of(HoodieTableMetaClient metaClient, HoodieSchema schema) { BloomFilter filter = BloomFilterFactory.createBloomFilter(10000, 0.0000001, -1, BloomFilterTypeCode.SIMPLE.name()); return of(metaClient, schema, filter); } - public static HoodieFlinkWriteableTestTable of(HoodieTable hoodieTable, Schema schema) { + public static HoodieFlinkWriteableTestTable of(HoodieTable hoodieTable, HoodieSchema schema) { HoodieTableMetaClient metaClient = hoodieTable.getMetaClient(); return of(metaClient, schema); } - public static HoodieFlinkWriteableTestTable of(HoodieTable hoodieTable, Schema schema, BloomFilter filter) { + public static HoodieFlinkWriteableTestTable of(HoodieTable hoodieTable, HoodieSchema schema, BloomFilter filter) { HoodieTableMetaClient metaClient = hoodieTable.getMetaClient(); return of(metaClient, schema, filter); } @@ -146,7 +146,7 @@ private Pair appendRecordsToLogFile(List gr logWriter.appendBlock(new HoodieAvroDataBlock(groupedRecords.stream().map(r -> { try { GenericRecord val = - (GenericRecord) ((HoodieRecordPayload) r.getData()).getInsertValue(schema).get(); + (GenericRecord) ((HoodieRecordPayload) r.getData()).getInsertValue(schema.toAvroSchema()).get(); HoodieAvroUtils.addHoodieKeyToRecord(val, r.getRecordKey(), r.getPartitionPath(), ""); return (IndexedRecord) val; } catch (IOException e) { 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 553d4c79f6cf6..c28dd4581662f 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 @@ -141,7 +141,7 @@ public ClosableIterator getUnsafeRowIterator(HoodieSchema requestedSc public ClosableIterator getUnsafeRowIterator(HoodieSchema requestedSchema, List readFilters) throws IOException { HoodieSchema nonNullSchema = HoodieSchemaUtils.getNonNullTypeFromUnion(requestedSchema); StructType structSchema = HoodieInternalRowUtils.getCachedSchema(nonNullSchema); - Option messageSchema = Option.of(getAvroSchemaConverter(storage.getConf().unwrapAs(Configuration.class)).convert(nonNullSchema.toAvroSchema())); + Option messageSchema = Option.of(getAvroSchemaConverter(storage.getConf().unwrapAs(Configuration.class)).convert(nonNullSchema)); boolean enableTimestampFieldRepair = storage.getConf().getBoolean(ENABLE_LOGICAL_TIMESTAMP_REPAIR, true); StructType dataStructType = convertToStruct(enableTimestampFieldRepair ? SchemaRepair.repairLogicalTypes(getFileSchema(), messageSchema) : getFileSchema()); SparkBasicSchemaEvolution evolution = new SparkBasicSchemaEvolution(dataStructType, structSchema, SQLConf.get().sessionLocalTimeZone()); @@ -189,7 +189,7 @@ public ClosableIterator getUnsafeRowIterator(HoodieSchema requestedSc private MessageType getFileSchema() { if (fileSchemaOption.isEmpty()) { - MessageType messageType = ((ParquetUtils) parquetUtils).readSchema(storage, path); + MessageType messageType = ((ParquetUtils) parquetUtils).readMessageType(storage, path); fileSchemaOption = Option.of(messageType); } return fileSchemaOption.get(); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/OrcBootstrapMetadataHandler.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/OrcBootstrapMetadataHandler.java index 9e6475a090cb2..c61e075914b42 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/OrcBootstrapMetadataHandler.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/OrcBootstrapMetadataHandler.java @@ -36,7 +36,6 @@ import org.apache.hudi.util.ExecutorFactory; import lombok.extern.slf4j.Slf4j; -import org.apache.avro.Schema; import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericRecord; import org.apache.hadoop.conf.Configuration; @@ -62,8 +61,7 @@ HoodieSchema getSchema(StoragePath sourceFilePath) throws IOException { Reader orcReader = OrcFile.createReader( new Path(sourceFilePath.toUri()), OrcFile.readerOptions((Configuration) table.getStorageConf().unwrap())); TypeDescription orcSchema = orcReader.getSchema(); - Schema schema = AvroOrcUtils.createAvroSchema(orcSchema); - return HoodieSchema.fromAvroSchema(schema); + return AvroOrcUtils.createSchema(orcSchema); } @Override @@ -76,11 +74,11 @@ void executeBootstrap(HoodieBootstrapHandle bootstrapHandle, } Reader orcReader = OrcFile.createReader( new Path(sourceFilePath.toUri()), OrcFile.readerOptions((Configuration) table.getStorageConf().unwrap())); - TypeDescription orcSchema = AvroOrcUtils.createOrcSchema(schema.getAvroSchema()); + TypeDescription orcSchema = AvroOrcUtils.createOrcSchema(schema); HoodieExecutor executor = null; RecordReader reader = orcReader.rows(new Reader.Options((Configuration) table.getStorageConf().unwrap()).schema(orcSchema)); try { - executor = ExecutorFactory.create(config, new OrcReaderIterator(reader, schema.getAvroSchema(), orcSchema), + executor = ExecutorFactory.create(config, new OrcReaderIterator(reader, schema, orcSchema), new BootstrapRecordConsumer(bootstrapHandle), inp -> { String recKey = keyGenerator.getKey(inp).getRecordKey(); GenericRecord gr = new GenericData.Record(METADATA_BOOTSTRAP_RECORD_SCHEMA.toAvroSchema()); 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 d0c75bd8ae8ee..9c6b6f89162c4 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 @@ -18,7 +18,6 @@ package org.apache.hudi.table.action.bootstrap; -import org.apache.avro.Schema; import org.apache.hudi.avro.model.HoodieFileStatus; import org.apache.hudi.common.model.HoodieAvroIndexedRecord; import org.apache.hudi.common.model.HoodieKey; @@ -71,8 +70,7 @@ HoodieSchema getSchema(StoragePath sourceFilePath) throws IOException { (Configuration) table.getStorageConf().unwrap(), new Path(sourceFilePath.toUri()), ParquetMetadataConverter.NO_FILTER); MessageType parquetSchema = readFooter.getFileMetaData().getSchema(); - Schema schema = getAvroSchemaConverter((Configuration) table.getStorageConf().unwrap()).convert(parquetSchema); - return HoodieSchema.fromAvroSchema(schema); + return getAvroSchemaConverter((Configuration) table.getStorageConf().unwrap()).convert(parquetSchema); } @Override diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala index 5a0f796d377b9..e4d97a2df6a29 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala @@ -95,7 +95,7 @@ class SparkFileFormatInternalRowReaderContext(baseFileReader: SparkColumnarFileR // Convert Avro dataSchema to Parquet MessageType for timestamp precision conversion val tableSchemaOpt = if (dataSchema != null) { val hadoopConf = storage.getConf.unwrapAs(classOf[Configuration]) - val parquetSchema = getAvroSchemaConverter(hadoopConf).convert(dataSchema.toAvroSchema) + val parquetSchema = getAvroSchemaConverter(hadoopConf).convert(dataSchema) org.apache.hudi.common.util.Option.of(parquetSchema) } else { org.apache.hudi.common.util.Option.empty[org.apache.parquet.schema.MessageType]() diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestUpdateSchemaEvolution.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestUpdateSchemaEvolution.java index e97d45fa88cc0..7f9a794165a78 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestUpdateSchemaEvolution.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestUpdateSchemaEvolution.java @@ -127,7 +127,7 @@ private void assertSchemaEvolutionOnUpdateResult(WriteStatus insertResult, Hoodi .getFileFormatUtils(updateTable.getBaseFileFormat()) .readAvroRecords(updateTable.getStorage(), new StoragePath(updateTable.getConfig().getBasePath() + "/" + insertResult.getStat().getPath()), - mergeHandle.getWriterSchemaWithMetaFields().toAvroSchema()); + mergeHandle.getWriterSchemaWithMetaFields()); for (GenericRecord rec : oldRecords) { // TODO create hoodie record with rec can getRecordKey mergeHandle.write(new HoodieAvroIndexedRecord(rec)); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieBloomIndex.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieBloomIndex.java index 859b85173b6aa..3a17863e0ab47 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieBloomIndex.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieBloomIndex.java @@ -160,7 +160,7 @@ public void testLoadInvolvedFiles( HoodieBloomIndex index = new HoodieBloomIndex(config, SparkHoodieBloomIndexHelper.getInstance()); HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); metadataWriter = SparkHoodieBackedTableMetadataWriter.create(storageConf, config, context); - HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(metaClient, SCHEMA.toAvroSchema(), metadataWriter, Option.of(context)); + HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(metaClient, SCHEMA, metadataWriter, Option.of(context)); // Create some partitions, and put some files // "2016/01/21": 0 file @@ -289,7 +289,7 @@ public void testCheckUUIDsAgainstOneFile() throws Exception { // record2, record3). BloomFilter filter = BloomFilterFactory.createBloomFilter(10000, 0.0000001, -1, BloomFilterTypeCode.SIMPLE.name()); filter.add(record3.getRecordKey()); - HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(metaClient, SCHEMA.toAvroSchema(), filter, metadataWriter, Option.of(context)); + HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(metaClient, SCHEMA, filter, metadataWriter, Option.of(context)); final Map>> partitionToFilesNameLengthMap = new HashMap<>(); final String commitTime = "0000001"; @@ -370,7 +370,7 @@ public void testTagLocationOnPartitionedTable( makeConfig(rangePruning, treeFiltering, bucketizedChecking, useMetadataTable, enableFileGroupIdKeySorting); HoodieSparkTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); metadataWriter = SparkHoodieBackedTableMetadataWriter.create(storageConf, config, context); - HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(metaClient, SCHEMA.toAvroSchema(), metadataWriter, Option.of(context)); + HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(metaClient, SCHEMA, metadataWriter, Option.of(context)); // Let's tag HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, SparkHoodieBloomIndexHelper.getInstance()); @@ -460,7 +460,7 @@ public void testTagLocationOnNonpartitionedTable( makeConfig(rangePruning, treeFiltering, bucketizedChecking, useMetadataTable, enableFileGroupIdKeySorting); HoodieSparkTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); metadataWriter = SparkHoodieBackedTableMetadataWriter.create(storageConf, config, context); - HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(metaClient, SCHEMA.toAvroSchema(), metadataWriter, Option.of(context)); + HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(metaClient, SCHEMA, metadataWriter, Option.of(context)); // Let's tag HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, SparkHoodieBloomIndexHelper.getInstance()); @@ -540,7 +540,7 @@ public void testCheckExists( makeConfig(rangePruning, treeFiltering, bucketizedChecking, useMetadataTable, enableFileGroupIdKeySorting); HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); metadataWriter = SparkHoodieBackedTableMetadataWriter.create(storageConf, config, context); - HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(metaClient, SCHEMA.toAvroSchema(), metadataWriter, Option.of(context)); + HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(metaClient, SCHEMA, metadataWriter, Option.of(context)); // Let's tag HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, SparkHoodieBloomIndexHelper.getInstance()); @@ -634,7 +634,7 @@ public void testBloomFilterFalseError( BloomFilter filter = BloomFilterFactory.createBloomFilter(10000, 0.0000001, -1, BloomFilterTypeCode.SIMPLE.name()); filter.add(record2.getRecordKey()); - HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(metaClient, SCHEMA.toAvroSchema(), filter, Option.of(context)); + HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(metaClient, SCHEMA, filter, Option.of(context)); String fileId = testTable.addCommit("000").getFileIdWithInserts("2016/01/31", record1); assertTrue(filter.mightContain(record1.getRecordKey())); assertTrue(filter.mightContain(record2.getRecordKey())); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieGlobalBloomIndex.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieGlobalBloomIndex.java index e9be566557bbc..1cbdc68979629 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieGlobalBloomIndex.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieGlobalBloomIndex.java @@ -92,7 +92,7 @@ public void testLoadInvolvedFiles() throws Exception { HoodieGlobalBloomIndex index = new HoodieGlobalBloomIndex(config, SparkHoodieBloomIndexHelper.getInstance()); HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); - HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(metaClient, SCHEMA.toAvroSchema(), metadataWriter); + HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(metaClient, SCHEMA, metadataWriter); // Create some partitions, and put some files, along with the meta file // "2016/01/21": 0 file @@ -225,7 +225,7 @@ public void testTagLocation() throws Exception { .build(); HoodieGlobalBloomIndex index = new HoodieGlobalBloomIndex(config, SparkHoodieBloomIndexHelper.getInstance()); HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); - HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(metaClient, SCHEMA.toAvroSchema(), metadataWriter); + HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(metaClient, SCHEMA, metadataWriter); // Create some partitions, and put some files, along with the meta file // "2016/01/21": 0 file diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bucket/TestHoodieSimpleBucketIndex.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bucket/TestHoodieSimpleBucketIndex.java index efba04b6dfc0a..c8c7b57fb32bd 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bucket/TestHoodieSimpleBucketIndex.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bucket/TestHoodieSimpleBucketIndex.java @@ -107,7 +107,7 @@ public void testTagLocation(boolean isInsert) throws Exception { HoodieData> taggedRecordRDD = bucketIndex.tagLocation(HoodieJavaRDD.of(recordRDD), context, table); assertFalse(taggedRecordRDD.collectAsList().stream().anyMatch(r -> r.isCurrentLocationKnown())); - HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(table, SCHEMA.toAvroSchema()); + HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(table, SCHEMA); if (isInsert) { testTable.addCommit("001").withInserts("2016/01/31", getRecordFileId(record1), record1); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieKeyLocationFetchHandle.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieKeyLocationFetchHandle.java index 6be35e862a05f..87c72e7464253 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieKeyLocationFetchHandle.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieKeyLocationFetchHandle.java @@ -58,7 +58,7 @@ import scala.Tuple2; import static java.util.stream.Collectors.toList; -import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.AVRO_SCHEMA_WITH_METADATA_FIELDS; +import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.HOODIE_SCHEMA_WITH_METADATA_FIELDS; import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA; import static org.apache.hudi.common.testutils.HoodieTestTable.makeNewCommitTime; import static org.apache.hudi.common.testutils.Transformations.recordsToPartitionRecordsMap; @@ -96,7 +96,7 @@ public void testFetchHandle(boolean populateMetaFields) throws Exception { List records = dataGen.generateInserts(makeNewCommitTime(), 100); Map> partitionRecordsMap = recordsToPartitionRecordsMap(records); HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); - HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(hoodieTable, AVRO_SCHEMA_WITH_METADATA_FIELDS); + HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(hoodieTable, HOODIE_SCHEMA_WITH_METADATA_FIELDS); Map, List>> expectedList = writeToParquetAndGetExpectedRecordLocations(partitionRecordsMap, testTable); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestMarkerBasedRollbackStrategy.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestMarkerBasedRollbackStrategy.java index 271cea4574f1a..b19b8038f5f45 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestMarkerBasedRollbackStrategy.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestMarkerBasedRollbackStrategy.java @@ -33,6 +33,7 @@ import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.IOType; +import org.apache.hudi.common.schema.HoodieSchemaUtils; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableVersion; import org.apache.hudi.common.table.timeline.HoodieInstant; @@ -75,7 +76,6 @@ import java.util.stream.IntStream; import java.util.stream.Stream; -import static org.apache.hudi.avro.HoodieAvroUtils.addMetadataFields; import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; import static org.apache.hudi.common.testutils.HoodieTestUtils.createSimpleRecord; import static org.apache.hudi.config.HoodieWriteConfig.ROLLBACK_PARALLELISM_VALUE; @@ -317,7 +317,7 @@ void testRollbackMultipleAppendLogFilesInOneFileGroupInMOR(HoodieTableVersion ta initMetaClient(tableType, props); String partition = "partA"; HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of( - metaClient, addMetadataFields(HoodieTestUtils.SIMPLE_RECORD_SCHEMA.toAvroSchema())); + metaClient, HoodieSchemaUtils.addMetadataFields(HoodieTestUtils.SIMPLE_RECORD_SCHEMA)); String fileId = UUID.randomUUID().toString(); HoodieRecord tripRecord = createSimpleRecord("key1", "2016-01-31T03:16:41.415Z", 123); String instantTime1 = "001"; diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieSparkWriteableTestTable.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieSparkWriteableTestTable.java index a1f55b44102db..69f1e84c561c9 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieSparkWriteableTestTable.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieSparkWriteableTestTable.java @@ -26,6 +26,7 @@ import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.schema.HoodieSchema; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.util.Option; import org.apache.hudi.metadata.HoodieTableMetadataWriter; @@ -34,7 +35,6 @@ import org.apache.hudi.table.HoodieTable; import lombok.extern.slf4j.Slf4j; -import org.apache.avro.Schema; import java.util.Arrays; import java.util.List; @@ -45,67 +45,67 @@ public class HoodieSparkWriteableTestTable extends HoodieWriteableTestTable { private HoodieSparkWriteableTestTable(String basePath, HoodieStorage storage, - HoodieTableMetaClient metaClient, Schema schema, + HoodieTableMetaClient metaClient, HoodieSchema schema, BloomFilter filter, HoodieTableMetadataWriter metadataWriter) { this(basePath, storage, metaClient, schema, filter, metadataWriter, Option.empty()); } private HoodieSparkWriteableTestTable(String basePath, HoodieStorage storage, - HoodieTableMetaClient metaClient, Schema schema, + HoodieTableMetaClient metaClient, HoodieSchema schema, BloomFilter filter, HoodieTableMetadataWriter metadataWriter, Option context) { super(basePath, storage, metaClient, schema, filter, metadataWriter, context); } - public static HoodieSparkWriteableTestTable of(HoodieTableMetaClient metaClient, Schema schema, + public static HoodieSparkWriteableTestTable of(HoodieTableMetaClient metaClient, HoodieSchema schema, BloomFilter filter) { return of(metaClient, schema, filter, Option.empty()); } - public static HoodieSparkWriteableTestTable of(HoodieTableMetaClient metaClient, Schema schema, BloomFilter filter, Option context) { + public static HoodieSparkWriteableTestTable of(HoodieTableMetaClient metaClient, HoodieSchema schema, BloomFilter filter, Option context) { return new HoodieSparkWriteableTestTable(metaClient.getBasePath().toString(), metaClient.getRawStorage(), metaClient, schema, filter, null, context); } - public static HoodieSparkWriteableTestTable of(HoodieTableMetaClient metaClient, Schema schema, BloomFilter filter, + public static HoodieSparkWriteableTestTable of(HoodieTableMetaClient metaClient, HoodieSchema schema, BloomFilter filter, HoodieTableMetadataWriter metadataWriter) { return of(metaClient, schema, filter, metadataWriter, Option.empty()); } - public static HoodieSparkWriteableTestTable of(HoodieTableMetaClient metaClient, Schema schema, BloomFilter filter, + public static HoodieSparkWriteableTestTable of(HoodieTableMetaClient metaClient, HoodieSchema schema, BloomFilter filter, HoodieTableMetadataWriter metadataWriter, Option context) { return new HoodieSparkWriteableTestTable(metaClient.getBasePath().toString(), metaClient.getRawStorage(), metaClient, schema, filter, metadataWriter, context); } - public static HoodieSparkWriteableTestTable of(HoodieTableMetaClient metaClient, Schema schema) { + public static HoodieSparkWriteableTestTable of(HoodieTableMetaClient metaClient, HoodieSchema schema) { BloomFilter filter = BloomFilterFactory .createBloomFilter(10000, 0.0000001, -1, BloomFilterTypeCode.SIMPLE.name()); return of(metaClient, schema, filter); } - public static HoodieSparkWriteableTestTable of(HoodieTableMetaClient metaClient, Schema schema, + public static HoodieSparkWriteableTestTable of(HoodieTableMetaClient metaClient, HoodieSchema schema, HoodieTableMetadataWriter metadataWriter) { return of(metaClient, schema, metadataWriter, Option.empty()); } - public static HoodieSparkWriteableTestTable of(HoodieTableMetaClient metaClient, Schema schema, + public static HoodieSparkWriteableTestTable of(HoodieTableMetaClient metaClient, HoodieSchema schema, HoodieTableMetadataWriter metadataWriter, Option context) { BloomFilter filter = BloomFilterFactory .createBloomFilter(10000, 0.0000001, -1, BloomFilterTypeCode.DYNAMIC_V0.name()); return of(metaClient, schema, filter, metadataWriter, context); } - public static HoodieSparkWriteableTestTable of(HoodieTable hoodieTable, Schema schema) { + public static HoodieSparkWriteableTestTable of(HoodieTable hoodieTable, HoodieSchema schema) { HoodieTableMetaClient metaClient = hoodieTable.getMetaClient(); return of(metaClient, schema); } - public static HoodieSparkWriteableTestTable of(HoodieTable hoodieTable, Schema schema, BloomFilter filter) { + public static HoodieSparkWriteableTestTable of(HoodieTable hoodieTable, HoodieSchema schema, BloomFilter filter) { HoodieTableMetaClient metaClient = hoodieTable.getMetaClient(); return of(metaClient, schema, filter); } 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 c11f87daf0fec..81e0b8bc05c63 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 @@ -22,6 +22,7 @@ import org.apache.hudi.common.model.HoodieOperation; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.schema.HoodieSchema; +import org.apache.hudi.common.schema.HoodieSchemaUtils; import org.apache.hudi.common.util.DateTimeUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.SpillableMapUtils; @@ -246,10 +247,6 @@ public static boolean isTypeNumeric(Schema.Type type) { return type == Schema.Type.INT || type == Schema.Type.LONG || type == Schema.Type.FLOAT || type == Schema.Type.DOUBLE; } - public static boolean isMetadataField(String fieldName) { - return HoodieRecord.HOODIE_META_COLUMNS_WITH_OPERATION.contains(fieldName); - } - public static Schema createHoodieWriteSchema(Schema originalSchema) { return HoodieAvroUtils.addMetadataFields(originalSchema); } @@ -308,7 +305,7 @@ public static Schema addMetadataFields(Schema schema, boolean withOperationField } for (Schema.Field field : schema.getFields()) { - if (!isMetadataField(field.name())) { + if (!HoodieSchemaUtils.isMetadataField(field.name())) { Schema.Field newField = createNewSchemaField(field); for (Map.Entry prop : field.getObjectProps().entrySet()) { newField.addProp(prop.getKey(), prop.getValue()); @@ -1096,7 +1093,7 @@ private static Object rewriteRecordWithNewSchemaInternal(Object oldRecord, for (int i = 0; i < newSchema.getFields().size(); i++) { Schema.Field newField = newSchema.getFields().get(i); String newFieldName = newField.name(); - if (skipMetadataFields && isMetadataField(newFieldName)) { + if (skipMetadataFields && HoodieSchemaUtils.isMetadataField(newFieldName)) { continue; } fieldNames.push(newFieldName); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/schema/HoodieSchemaUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/schema/HoodieSchemaUtils.java index ec49fdee0fbef..82fac31a07bd4 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/schema/HoodieSchemaUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/schema/HoodieSchemaUtils.java @@ -33,6 +33,7 @@ import java.math.BigInteger; import java.math.MathContext; import java.math.RoundingMode; +import java.util.ArrayList; import java.util.Collections; import java.util.List; import java.util.Map; @@ -651,6 +652,25 @@ public static Object convertValueForSpecificDataTypes(HoodieSchema fieldSchema, ); } + /** + * Fetch schema for record key and partition path. + * This is equivalent to HoodieAvroUtils.getRecordKeyPartitionPathSchema() but returns HoodieSchema. + * + * @return HoodieSchema containing record key and partition path fields + */ + public static HoodieSchema getRecordKeyPartitionPathSchema() { + List toBeAddedFields = new ArrayList<>(2); + + HoodieSchemaField recordKeyField = + createNewSchemaField(HoodieRecord.RECORD_KEY_METADATA_FIELD, METADATA_FIELD_SCHEMA, "", JsonProperties.NULL_VALUE); + HoodieSchemaField partitionPathField = + createNewSchemaField(HoodieRecord.PARTITION_PATH_METADATA_FIELD, METADATA_FIELD_SCHEMA, "", JsonProperties.NULL_VALUE); + + toBeAddedFields.add(recordKeyField); + toBeAddedFields.add(partitionPathField); + return HoodieSchema.createRecord("HoodieRecordKey", "", "", false, toBeAddedFields); + } + /** * Fetches projected schema given list of fields to project. The field can be nested in format `a.b.c` where a is * the top level field, b is at second level and so on. @@ -770,4 +790,8 @@ public static HoodieSchema resolveUnionSchema(HoodieSchema schema, String fieldS public static String addMetadataColumnTypes(String hiveColumnTypes) { return "string,string,string,string,string," + hiveColumnTypes; } + + public static boolean isMetadataField(String fieldName) { + return HoodieRecord.HOODIE_META_COLUMNS_WITH_OPERATION.contains(fieldName); + } } 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 e879c8f8d222b..768db4b533e51 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 @@ -384,7 +384,7 @@ public Schema readSchemaFromLastCompaction(Option lastCompactionC + lastCompactionCommit + ", could not get schema for table " + metaClient.getBasePath())); StoragePath path = new StoragePath(filePath); return HoodieIOFactory.getIOFactory(metaClient.getStorage()) - .getFileFormatUtils(path).readAvroSchema(metaClient.getStorage(), path); + .getFileFormatUtils(path).readSchema(metaClient.getStorage(), path).toAvroSchema(); } private Schema readSchemaFromLogFile(StoragePath path) throws IOException { @@ -567,8 +567,9 @@ private Schema fetchSchemaFromFiles(Stream filePaths) { // this is a log file return readSchemaFromLogFile(filePath); } else { - return HoodieIOFactory.getIOFactory(metaClient.getStorage()) - .getFileFormatUtils(filePath).readAvroSchema(metaClient.getStorage(), filePath); + HoodieSchema hoodieSchema = HoodieIOFactory.getIOFactory(metaClient.getStorage()) + .getFileFormatUtils(filePath).readSchema(metaClient.getStorage(), filePath); + return hoodieSchema != null ? hoodieSchema.toAvroSchema() : null; } } catch (IOException e) { throw new HoodieIOException("Failed to read schema from file: " + filePath, e); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/FileFormatUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/FileFormatUtils.java index a1fb702ca5588..ef3fa63175d46 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/FileFormatUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/FileFormatUtils.java @@ -18,7 +18,6 @@ package org.apache.hudi.common.util; -import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.avro.HoodieBloomFilterWriteSupport; import org.apache.hudi.common.bloom.BloomFilter; import org.apache.hudi.common.bloom.BloomFilterFactory; @@ -27,6 +26,7 @@ import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.schema.HoodieSchema; +import org.apache.hudi.common.schema.HoodieSchemaUtils; import org.apache.hudi.common.util.collection.ClosableIterator; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieException; @@ -38,7 +38,6 @@ import org.apache.hudi.storage.HoodieStorage; import org.apache.hudi.storage.StoragePath; -import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import javax.annotation.Nonnull; @@ -218,7 +217,7 @@ public String[] readMinMaxRecordKeys(HoodieStorage storage, StoragePath filePath * @param filePath the data file path. * @return a list of GenericRecord. */ - public abstract List readAvroRecords(HoodieStorage storage, StoragePath filePath, Schema schema); + public abstract List readAvroRecords(HoodieStorage storage, StoragePath filePath, HoodieSchema schema); /** * Read the footer data of the given data file. @@ -283,15 +282,15 @@ public abstract ClosableIterator getHoodieKeyIterator(HoodieStorage s */ public abstract ClosableIterator getHoodieKeyIterator(HoodieStorage storage, StoragePath filePath); - protected Schema getKeyIteratorSchema(HoodieStorage storage, StoragePath filePath, Option keyGeneratorOpt, Option partitionPath) { + protected HoodieSchema getKeyIteratorSchema(HoodieStorage storage, StoragePath filePath, Option keyGeneratorOpt, Option partitionPath) { return keyGeneratorOpt .map(keyGenerator -> { List fields = new ArrayList<>(); fields.addAll(keyGenerator.getRecordKeyFieldNames()); fields.addAll(keyGenerator.getPartitionPathFields()); - return HoodieAvroUtils.projectSchema(readAvroSchema(storage, filePath), fields); + return HoodieSchemaUtils.projectSchema(readSchema(storage, filePath), fields); }) - .orElse(partitionPath.isPresent() ? HoodieAvroUtils.getRecordKeySchema() : HoodieAvroUtils.getRecordKeyPartitionPathSchema()); + .orElse(partitionPath.isPresent() ? HoodieSchemaUtils.getRecordKeySchema() : HoodieSchemaUtils.getRecordKeyPartitionPathSchema()); } /** @@ -315,7 +314,7 @@ public abstract ClosableIterator> fetchRecordKeysWithPosit * @param filePath the data file path. * @return the Avro schema of the data file. */ - public abstract Schema readAvroSchema(HoodieStorage storage, StoragePath filePath); + public abstract HoodieSchema readSchema(HoodieStorage storage, StoragePath filePath); /** * Reads column statistics stored in the metadata. @@ -381,8 +380,8 @@ public abstract ByteArrayOutputStream serializeRecordsToLogBlock(HoodieStorage s public abstract Pair serializeRecordsToLogBlock(HoodieStorage storage, Iterator records, HoodieRecord.HoodieRecordType recordType, - Schema writerSchema, - Schema readerSchema, + HoodieSchema writerSchema, + HoodieSchema readerSchema, String keyFieldName, Map paramsMap) throws IOException; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/HFileUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/HFileUtils.java index 9ea05627c2cc8..75cdd9f6bc99f 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/HFileUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/HFileUtils.java @@ -25,6 +25,7 @@ import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType; import org.apache.hudi.common.schema.HoodieSchema; import org.apache.hudi.common.schema.HoodieSchemaField; import org.apache.hudi.common.util.collection.ClosableIterator; @@ -44,7 +45,6 @@ import org.apache.hudi.storage.HoodieStorage; import org.apache.hudi.storage.StoragePath; -import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -87,7 +87,7 @@ public List readAvroRecords(HoodieStorage storage, StoragePath fi } @Override - public List readAvroRecords(HoodieStorage storage, StoragePath filePath, Schema schema) { + public List readAvroRecords(HoodieStorage storage, StoragePath filePath, HoodieSchema schema) { throw new UnsupportedOperationException("HFileUtils does not support readAvroRecords"); } @@ -152,7 +152,7 @@ public ClosableIterator> fetchRecordKeysWithPositions(Hood } @Override - public Schema readAvroSchema(HoodieStorage storage, StoragePath filePath) { + public HoodieSchema readSchema(HoodieStorage storage, StoragePath filePath) { LOG.info("Reading schema from {}", filePath); try (HoodieFileReader fileReader = @@ -161,8 +161,7 @@ public Schema readAvroSchema(HoodieStorage storage, StoragePath filePath) { .getFileReader( ConfigUtils.DEFAULT_HUDI_CONFIG_FOR_READER, filePath)) { - //TODO boundary to revisit in later pr to use HoodieSchema directly - return fileReader.getSchema().toAvroSchema(); + return fileReader.getSchema(); } catch (IOException e) { throw new HoodieIOException("Failed to read schema from HFile", e); } @@ -218,9 +217,9 @@ public ByteArrayOutputStream serializeRecordsToLogBlock(HoodieStorage storage, if (i > 0 && recordKey.equals(previousRecordKey)) { LOG.error("Found duplicate record with recordKey: {}", recordKey); logRecordMetadata("Previous record", - serializeRecord(records.get(i - 1), writerSchema, keyField), writerSchema.toAvroSchema()); + serializeRecord(records.get(i - 1), writerSchema, keyField), writerSchema); logRecordMetadata("Current record", - serializeRecord(record, writerSchema, keyField), writerSchema.toAvroSchema()); + serializeRecord(record, writerSchema, keyField), writerSchema); throw new HoodieException(String.format( "Writing multiple records with same key %s not supported for Hfile format with Metadata table", recordKey)); } @@ -244,8 +243,8 @@ public ByteArrayOutputStream serializeRecordsToLogBlock(HoodieStorage storage, /** * Print the meta fields of the record of interest */ - private void logRecordMetadata(String msg, byte[] bs, Schema schema) throws IOException { - GenericRecord record = HoodieAvroUtils.bytesToAvro(bs, schema); + private void logRecordMetadata(String msg, byte[] bs, HoodieSchema schema) throws IOException { + GenericRecord record = HoodieAvroUtils.bytesToAvro(bs, schema.toAvroSchema()); if (schema.getField(HoodieRecord.RECORD_KEY_METADATA_FIELD) != null) { LOG.error("{}: Hudi meta field values -> Record key: {}, Partition Path: {}, FileName: {}, CommitTime: {}, CommitSeqNo: {}", msg, record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD), record.get(HoodieRecord.PARTITION_PATH_METADATA_FIELD), @@ -258,9 +257,9 @@ private void logRecordMetadata(String msg, byte[] bs, Schema schema) throws IOEx public Pair serializeRecordsToLogBlock( HoodieStorage storage, Iterator records, - HoodieRecord.HoodieRecordType recordType, - Schema writerSchema, - Schema readerSchema, + HoodieRecordType recordType, + HoodieSchema writerSchema, + HoodieSchema readerSchema, String keyFieldName, Map paramsMap) throws IOException { throw new UnsupportedOperationException("HFileUtils does not support serializeRecordsToLogBlock returning HoodieColumnRangeMetadata."); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/LanceUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/LanceUtils.java index c0bf94f07489d..53e7bdbf95008 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/LanceUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/LanceUtils.java @@ -36,7 +36,6 @@ import org.apache.hudi.storage.HoodieStorage; import org.apache.hudi.storage.StoragePath; -import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import java.io.ByteArrayOutputStream; @@ -104,7 +103,7 @@ public HoodieKey next() { } @Override - public Schema readAvroSchema(HoodieStorage storage, StoragePath filePath) { + public HoodieSchema readSchema(HoodieStorage storage, StoragePath filePath) { try (HoodieFileReader fileReader = HoodieIOFactory.getIOFactory(storage) .getReaderFactory(HoodieRecord.HoodieRecordType.SPARK) @@ -112,7 +111,7 @@ public Schema readAvroSchema(HoodieStorage storage, StoragePath filePath) { ConfigUtils.DEFAULT_HUDI_CONFIG_FOR_READER, filePath, HoodieFileFormat.LANCE)) { - return fileReader.getSchema().getAvroSchema(); + return fileReader.getSchema(); } catch (IOException e) { throw new HoodieIOException("Failed to read schema from Lance file", e); } @@ -129,7 +128,7 @@ public List readAvroRecords(HoodieStorage storage, StoragePath fi } @Override - public List readAvroRecords(HoodieStorage storage, StoragePath filePath, Schema schema) { + public List readAvroRecords(HoodieStorage storage, StoragePath filePath, HoodieSchema schema) { throw new UnsupportedOperationException("readAvroRecords with schema is not yet supported for Lance format"); } @@ -195,8 +194,8 @@ public ByteArrayOutputStream serializeRecordsToLogBlock(HoodieStorage storage, public Pair serializeRecordsToLogBlock(HoodieStorage storage, Iterator records, HoodieRecord.HoodieRecordType recordType, - Schema writerSchema, - Schema readerSchema, + HoodieSchema writerSchema, + HoodieSchema readerSchema, String keyFieldName, Map paramsMap) throws IOException { throw new UnsupportedOperationException("serializeRecordsToLogBlock with iterator is not yet supported for Lance format"); diff --git a/hudi-common/src/test/java/org/apache/hudi/avro/TestHoodieAvroUtils.java b/hudi-common/src/test/java/org/apache/hudi/avro/TestHoodieAvroUtils.java index 2d460af7e9fcc..f3e906d9f75b9 100644 --- a/hudi-common/src/test/java/org/apache/hudi/avro/TestHoodieAvroUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/avro/TestHoodieAvroUtils.java @@ -63,6 +63,7 @@ import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.RewriteAvroPayload; import org.apache.hudi.common.schema.HoodieSchema; +import org.apache.hudi.common.schema.HoodieSchemaUtils; import org.apache.hudi.common.testutils.SchemaTestUtil; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieException; @@ -283,7 +284,7 @@ public void testPropsPresent() { Schema schema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(EXAMPLE_SCHEMA)); boolean piiPresent = false; for (Schema.Field field : schema.getFields()) { - if (HoodieAvroUtils.isMetadataField(field.name())) { + if (HoodieSchemaUtils.isMetadataField(field.name())) { continue; } diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HiveSchemaUtils.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HiveSchemaUtils.java index 561b73effdc7c..5adb0c538b81e 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HiveSchemaUtils.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HiveSchemaUtils.java @@ -18,8 +18,8 @@ package org.apache.hudi.table.catalog; -import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.schema.HoodieSchemaUtils; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.configuration.FlinkOptions; @@ -64,7 +64,7 @@ public static List getFieldNames(List fieldSchemas) { public static org.apache.flink.table.api.Schema convertTableSchema(Table hiveTable) { List allCols = hiveTable.getSd().getCols().stream() // filter out the metadata columns - .filter(s -> !HoodieAvroUtils.isMetadataField(s.getName())) + .filter(s -> !HoodieSchemaUtils.isMetadataField(s.getName())) .collect(Collectors.toList()); // need to refactor the partition key field positions: they are not always in the last allCols.addAll(hiveTable.getPartitionKeys()); diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/HoodieRowDataParquetReader.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/HoodieRowDataParquetReader.java index 4465124c9a891..dbc9bcce9ad2d 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/HoodieRowDataParquetReader.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/HoodieRowDataParquetReader.java @@ -116,7 +116,7 @@ public DataType getRowType() { // 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.readSchema(storage, path); + MessageType messageType = parquetUtils.readMessageType(storage, path); RowType rowType = ParquetSchemaConverter.convertToRowType(messageType); fileRowType = DataTypes.of(rowType); } diff --git a/hudi-hadoop-common/src/main/java/org/apache/hudi/avro/HoodieAvroWriteSupport.java b/hudi-hadoop-common/src/main/java/org/apache/hudi/avro/HoodieAvroWriteSupport.java index 878f68a693ace..97428a6819e43 100644 --- a/hudi-hadoop-common/src/main/java/org/apache/hudi/avro/HoodieAvroWriteSupport.java +++ b/hudi-hadoop-common/src/main/java/org/apache/hudi/avro/HoodieAvroWriteSupport.java @@ -20,11 +20,11 @@ package org.apache.hudi.avro; import org.apache.hudi.common.bloom.BloomFilter; +import org.apache.hudi.common.schema.HoodieSchema; import org.apache.hudi.common.util.CollectionUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; -import org.apache.avro.Schema; import org.apache.parquet.avro.AvroWriteSupport; import org.apache.parquet.hadoop.api.WriteSupport; import org.apache.parquet.schema.MessageType; @@ -43,9 +43,9 @@ public class HoodieAvroWriteSupport extends AvroWriteSupport { private final Map footerMetadata = new HashMap<>(); protected final Properties properties; - public HoodieAvroWriteSupport(MessageType schema, Schema avroSchema, Option bloomFilterOpt, + public HoodieAvroWriteSupport(MessageType schema, HoodieSchema hoodieSchema, Option bloomFilterOpt, Properties properties) { - super(schema, avroSchema, ConvertingGenericData.INSTANCE); + super(schema, hoodieSchema.toAvroSchema(), ConvertingGenericData.INSTANCE); this.bloomFilterWriteSupportOpt = bloomFilterOpt.map(HoodieBloomFilterAvroWriteSupport::new); this.properties = properties; } diff --git a/hudi-hadoop-common/src/main/java/org/apache/hudi/common/table/ParquetTableSchemaResolver.java b/hudi-hadoop-common/src/main/java/org/apache/hudi/common/table/ParquetTableSchemaResolver.java index 08ec8f24654c9..b3afaf93966ea 100644 --- a/hudi-hadoop-common/src/main/java/org/apache/hudi/common/table/ParquetTableSchemaResolver.java +++ b/hudi-hadoop-common/src/main/java/org/apache/hudi/common/table/ParquetTableSchemaResolver.java @@ -19,7 +19,8 @@ package org.apache.hudi.common.table; -import org.apache.avro.Schema; +import org.apache.hudi.common.schema.HoodieSchema; + import org.apache.hadoop.conf.Configuration; import org.apache.parquet.schema.MessageType; @@ -31,7 +32,7 @@ public ParquetTableSchemaResolver(HoodieTableMetaClient metaClient) { super(metaClient); } - public static MessageType convertAvroSchemaToParquet(Schema schema, Configuration hadoopConf) { + public static MessageType convertAvroSchemaToParquet(HoodieSchema schema, Configuration hadoopConf) { return getAvroSchemaConverter(hadoopConf).convert(schema); } } diff --git a/hudi-hadoop-common/src/main/java/org/apache/hudi/common/util/AvroOrcUtils.java b/hudi-hadoop-common/src/main/java/org/apache/hudi/common/util/AvroOrcUtils.java index 77445f1115802..320d8b7472e47 100644 --- a/hudi-hadoop-common/src/main/java/org/apache/hudi/common/util/AvroOrcUtils.java +++ b/hudi-hadoop-common/src/main/java/org/apache/hudi/common/util/AvroOrcUtils.java @@ -19,15 +19,12 @@ package org.apache.hudi.common.util; import org.apache.hudi.common.schema.HoodieSchema; +import org.apache.hudi.common.schema.HoodieSchema.TimePrecision; import org.apache.hudi.common.schema.HoodieSchemaField; import org.apache.hudi.common.schema.HoodieSchemaType; import org.apache.hudi.exception.HoodieIOException; import org.apache.avro.Conversions; -import org.apache.avro.LogicalType; -import org.apache.avro.LogicalTypes; -import org.apache.avro.Schema; -import org.apache.avro.Schema.Field; import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericData.StringType; import org.apache.avro.generic.GenericRecord; @@ -82,12 +79,12 @@ public class AvroOrcUtils { * * @param type ORC schema of the value Object. * @param colVector The column vector to store the value Object. - * @param avroSchema Avro schema of the value Object. + * @param schema Schema of the value Object. * Only used to check logical types for timestamp unit conversion. * @param value Object to be added to the column vector * @param vectorPos The position in the vector where value will be stored at. */ - public static void addToVector(TypeDescription type, ColumnVector colVector, Schema avroSchema, Object value, int vectorPos) { + public static void addToVector(TypeDescription type, ColumnVector colVector, HoodieSchema schema, Object value, int vectorPos) { final int currentVecLength = colVector.isNull.length; if (vectorPos >= currentVecLength) { @@ -99,12 +96,11 @@ public static void addToVector(TypeDescription type, ColumnVector colVector, Sch return; } - if (avroSchema.getType().equals(Schema.Type.UNION)) { - avroSchema = getActualSchemaType(avroSchema); + HoodieSchemaType schemaType = schema.getType(); + if (schemaType == HoodieSchemaType.UNION) { + schema = getActualSchemaType(schema); } - LogicalType logicalType = avroSchema != null ? avroSchema.getLogicalType() : null; - switch (type.getCategory()) { case BOOLEAN: LongColumnVector boolVec = (LongColumnVector) colVector; @@ -170,7 +166,7 @@ public static void addToVector(TypeDescription type, ColumnVector colVector, Sch case DATE: LongColumnVector dateColVec = (LongColumnVector) colVector; int daysSinceEpoch; - if (logicalType instanceof LogicalTypes.Date) { + if (schemaType == HoodieSchemaType.DATE) { daysSinceEpoch = (int) value; } else if (value instanceof java.sql.Date) { daysSinceEpoch = DateWritable.dateToDays((java.sql.Date) value); @@ -178,7 +174,7 @@ public static void addToVector(TypeDescription type, ColumnVector colVector, Sch daysSinceEpoch = DateWritable.millisToDays(((Date) value).getTime()); } else { throw new IllegalStateException(String.format( - "Unrecognized type for Avro DATE field value, which has type %s, value %s", + "Unrecognized type for DATE field value, which has type %s, value %s", value.getClass().getName(), value )); @@ -186,15 +182,16 @@ public static void addToVector(TypeDescription type, ColumnVector colVector, Sch dateColVec.vector[vectorPos] = daysSinceEpoch; break; case TIMESTAMP: + HoodieSchema.Timestamp timestampSchema = (HoodieSchema.Timestamp) schema; TimestampColumnVector tsColVec = (TimestampColumnVector) colVector; long time; int nanos = 0; // The unit for Timestamp in ORC is millis, convert timestamp to millis if needed - if (logicalType instanceof LogicalTypes.TimestampMillis) { + if (TimePrecision.MILLIS == timestampSchema.getPrecision()) { time = (long) value; - } else if (logicalType instanceof LogicalTypes.TimestampMicros) { + } else if (TimePrecision.MICROS == timestampSchema.getPrecision()) { final long logicalTsValue = (long) value; time = logicalTsValue / MICROS_PER_MILLI; nanos = NANOS_PER_MICRO * ((int) (logicalTsValue % MICROS_PER_MILLI)); @@ -210,7 +207,7 @@ public static void addToVector(TypeDescription type, ColumnVector colVector, Sch time = dateValue.getTime(); } else { throw new IllegalStateException(String.format( - "Unrecognized type for Avro TIMESTAMP field value, which has type %s, value %s", + "Unrecognized type for TIMESTAMP field value, which has type %s, value %s", value.getClass().getName(), value )); @@ -262,11 +259,11 @@ public static void addToVector(TypeDescription type, ColumnVector colVector, Sch } } else if (value instanceof GenericData.Fixed) { final BigDecimal decimal = new Conversions.DecimalConversion() - .fromFixed((GenericData.Fixed) value, avroSchema, logicalType); + .fromFixed((GenericData.Fixed) value, schema.toAvroSchema(), schema.toAvroSchema().getLogicalType()); decimalValue = HiveDecimal.create(decimal); } else { throw new IllegalStateException(String.format( - "Unexpected type for decimal (%s), cannot convert from Avro value", + "Unexpected type for decimal (%s), cannot convert from value", value.getClass().getCanonicalName() )); } @@ -285,7 +282,7 @@ public static void addToVector(TypeDescription type, ColumnVector colVector, Sch TypeDescription listType = type.getChildren().get(0); for (Object listItem : list) { - addToVector(listType, listColVec.child, avroSchema.getElementType(), listItem, listColVec.childCount++); + addToVector(listType, listColVec.child, schema.getElementType(), listItem, listColVec.childCount++); } break; case MAP: @@ -296,7 +293,7 @@ public static void addToVector(TypeDescription type, ColumnVector colVector, Sch mapColumnVector.lengths[vectorPos] = mapValue.size(); // keys are always strings - Schema keySchema = Schema.create(Schema.Type.STRING); + HoodieSchema keySchema = HoodieSchema.create(HoodieSchemaType.STRING); for (Map.Entry entry : mapValue.entrySet()) { addToVector( type.getChildren().get(0), @@ -309,7 +306,7 @@ public static void addToVector(TypeDescription type, ColumnVector colVector, Sch addToVector( type.getChildren().get(1), mapColumnVector.values, - avroSchema.getValueType(), + schema.getValueType(), entry.getValue(), mapColumnVector.childCount ); @@ -327,7 +324,7 @@ public static void addToVector(TypeDescription type, ColumnVector colVector, Sch String fieldName = type.getFieldNames().get(i); Object fieldValue = record.get(fieldName); TypeDescription fieldType = type.getChildren().get(i); - addToVector(fieldType, structColVec.fields[i], avroSchema.getFields().get(i).schema(), fieldValue, vectorPos); + addToVector(fieldType, structColVec.fields[i], schema.getFields().get(i).schema(), fieldValue, vectorPos); } break; @@ -335,7 +332,7 @@ public static void addToVector(TypeDescription type, ColumnVector colVector, Sch UnionColumnVector unionColVec = (UnionColumnVector) colVector; List childTypes = type.getChildren(); - boolean added = addUnionValue(unionColVec, childTypes, avroSchema, value, vectorPos); + boolean added = addUnionValue(unionColVec, childTypes, schema, value, vectorPos); if (!added) { throw new IllegalStateException(String.format( @@ -356,7 +353,7 @@ public static void addToVector(TypeDescription type, ColumnVector colVector, Sch * * @param unionVector The vector to store value. * @param unionChildTypes All possible types for the value Object. - * @param avroSchema Avro union schema for the value Object. + * @param schema Union schema for the value Object. * @param value Object to be added to the unionVector * @param vectorPos The position in the vector where value will be stored at. * @return succeeded or failed @@ -364,7 +361,7 @@ public static void addToVector(TypeDescription type, ColumnVector colVector, Sch public static boolean addUnionValue( UnionColumnVector unionVector, List unionChildTypes, - Schema avroSchema, + HoodieSchema schema, Object value, int vectorPos ) { @@ -434,7 +431,7 @@ public static boolean addUnionValue( if (value == null) { matches = children == null || children.size() == 0; } else { - matches = addUnionValue(unionVector, children, avroSchema, value, vectorPos); + matches = addUnionValue(unionVector, children, schema, value, vectorPos); } break; default: @@ -458,7 +455,7 @@ public static boolean addUnionValue( unionVector.isNull[vectorPos] = true; unionVector.noNulls = false; } else { - addToVector(matchType, unionVector.fields[matchIndex], avroSchema.getTypes().get(matchIndex), value, vectorPos); + addToVector(matchType, unionVector.fields[matchIndex], schema.getTypes().get(matchIndex), value, vectorPos); } return true; } else { @@ -471,12 +468,12 @@ public static boolean addUnionValue( * * @param type ORC schema of the object to read. * @param colVector The column vector to read. - * @param avroSchema Avro schema of the object to read. + * @param schema Schema of the object to read. * Only used to check logical types for timestamp unit conversion. * @param vectorPos The position in the vector where the value to read is stored at. * @return The object being read. */ - public static Object readFromVector(TypeDescription type, ColumnVector colVector, Schema avroSchema, int vectorPos) { + public static Object readFromVector(TypeDescription type, ColumnVector colVector, HoodieSchema schema, int vectorPos) { if (colVector.isRepeating) { vectorPos = 0; @@ -486,10 +483,9 @@ public static Object readFromVector(TypeDescription type, ColumnVector colVector return null; } - if (avroSchema.getType().equals(Schema.Type.UNION)) { - avroSchema = getActualSchemaType(avroSchema); + if (schema.getType() == HoodieSchemaType.UNION) { + schema = getActualSchemaType(schema); } - LogicalType logicalType = avroSchema != null ? avroSchema.getLogicalType() : null; switch (type.getCategory()) { case BOOLEAN: @@ -516,7 +512,7 @@ public static Object readFromVector(TypeDescription type, ColumnVector colVector throw new HoodieIOException("CHAR/VARCHAR has length " + result.length() + " greater than Max Length allowed"); } case STRING: - String stringType = avroSchema.getProp(GenericData.STRING_PROP); + Object stringType = schema.getProp(GenericData.STRING_PROP); Object parsedValue; if (stringType == null || !stringType.equals(StringType.String)) { int stringLength = ((BytesColumnVector) colVector).length[vectorPos]; @@ -527,10 +523,10 @@ public static Object readFromVector(TypeDescription type, ColumnVector colVector } else { parsedValue = ((BytesColumnVector) colVector).toString(vectorPos); } - if (avroSchema.getType() == Schema.Type.ENUM) { + if (schema.getType() == HoodieSchemaType.ENUM) { String enumValue = parsedValue.toString(); if (!enumValue.isEmpty()) { - return new GenericData.EnumSymbol(avroSchema, enumValue); + return new GenericData.EnumSymbol(schema.toAvroSchema(), enumValue); } } return parsedValue; @@ -538,12 +534,13 @@ public static Object readFromVector(TypeDescription type, ColumnVector colVector // convert to daysSinceEpoch for LogicalType.Date return (int) ((LongColumnVector) colVector).vector[vectorPos]; case TIMESTAMP: + HoodieSchema.Timestamp timestampSchema = (HoodieSchema.Timestamp) schema; // The unit of time in ORC is millis. Convert (time,nanos) to the desired unit per logicalType long time = ((TimestampColumnVector) colVector).time[vectorPos]; int nanos = ((TimestampColumnVector) colVector).nanos[vectorPos]; - if (logicalType instanceof LogicalTypes.TimestampMillis) { + if (TimePrecision.MILLIS == timestampSchema.getPrecision()) { return time; - } else if (logicalType instanceof LogicalTypes.TimestampMicros) { + } else if (TimePrecision.MICROS == timestampSchema.getPrecision()) { return time * MICROS_PER_MILLI + nanos / NANOS_PER_MICRO; } else { return ((TimestampColumnVector) colVector).getTimestampAsLong(vectorPos); @@ -556,18 +553,16 @@ public static Object readFromVector(TypeDescription type, ColumnVector colVector // return a ByteBuffer to be consistent with AvroRecordConverter return ByteBuffer.wrap(binaryBytes); case DECIMAL: + HoodieSchema.Decimal decimalSchema = (HoodieSchema.Decimal) schema; // HiveDecimal always ignores trailing zeros, thus modifies the scale implicitly, // therefore, the scale must be enforced here. BigDecimal bigDecimal = ((DecimalColumnVector) colVector).vector[vectorPos] .getHiveDecimal().bigDecimalValue() - .setScale(((LogicalTypes.Decimal) logicalType).getScale()); - Schema.Type baseType = avroSchema.getType(); - if (baseType.equals(Schema.Type.FIXED)) { - return new Conversions.DecimalConversion().toFixed(bigDecimal, avroSchema, logicalType); - } else if (baseType.equals(Schema.Type.BYTES)) { - return ByteBuffer.wrap(bigDecimal.unscaledValue().toByteArray()); + .setScale(decimalSchema.getScale()); + if (decimalSchema.isFixed()) { + return new Conversions.DecimalConversion().toFixed(bigDecimal, schema.toAvroSchema(), schema.toAvroSchema().getLogicalType()); } else { - throw new HoodieIOException(baseType.getName() + "is not a valid type for LogicalTypes.DECIMAL."); + return ByteBuffer.wrap(bigDecimal.unscaledValue().toByteArray()); } case LIST: ArrayList list = new ArrayList<>(); @@ -577,90 +572,46 @@ public static Object readFromVector(TypeDescription type, ColumnVector colVector list.ensureCapacity(listLength); TypeDescription childType = type.getChildren().get(0); for (int i = 0; i < listLength; i++) { - list.add(readFromVector(childType, listVector.child, avroSchema.getElementType(), listOffset + i)); + list.add(readFromVector(childType, listVector.child, schema.getElementType(), listOffset + i)); } return list; case MAP: - Map map = new HashMap(); + Map map = new HashMap<>(); MapColumnVector mapVector = (MapColumnVector) colVector; int mapLength = (int) mapVector.lengths[vectorPos]; int mapOffset = (int) mapVector.offsets[vectorPos]; - // keys are always strings for maps in Avro - Schema keySchema = Schema.create(Schema.Type.STRING); + // keys are always strings for maps + HoodieSchema keySchema = HoodieSchema.create(HoodieSchemaType.STRING); for (int i = 0; i < mapLength; i++) { map.put( readFromVector(type.getChildren().get(0), mapVector.keys, keySchema, i + mapOffset).toString(), readFromVector(type.getChildren().get(1), mapVector.values, - avroSchema.getValueType(), i + mapOffset)); + schema.getValueType(), i + mapOffset)); } return map; case STRUCT: StructColumnVector structVector = (StructColumnVector) colVector; List children = type.getChildren(); - GenericData.Record record = new GenericData.Record(avroSchema); + GenericData.Record record = new GenericData.Record(schema.toAvroSchema()); for (int i = 0; i < children.size(); i++) { record.put(i, readFromVector(children.get(i), structVector.fields[i], - avroSchema.getFields().get(i).schema(), vectorPos)); + schema.getFields().get(i).schema(), vectorPos)); } return record; case UNION: UnionColumnVector unionVector = (UnionColumnVector) colVector; int tag = unionVector.tags[vectorPos]; ColumnVector fieldVector = unionVector.fields[tag]; - return readFromVector(type.getChildren().get(tag), fieldVector, avroSchema.getTypes().get(tag), vectorPos); + return readFromVector(type.getChildren().get(tag), fieldVector, schema.getTypes().get(tag), vectorPos); default: throw new HoodieIOException("Unrecognized TypeDescription " + type); } } - public static TypeDescription createOrcSchema(Schema avroSchema) { + public static TypeDescription createOrcSchema(HoodieSchema schema) { - LogicalType logicalType = avroSchema.getLogicalType(); + final HoodieSchemaType type = schema.getType(); - if (logicalType != null) { - if (logicalType instanceof LogicalTypes.Decimal) { - return TypeDescription.createDecimal() - .withPrecision(((LogicalTypes.Decimal) logicalType).getPrecision()) - .withScale(((LogicalTypes.Decimal) logicalType).getScale()); - } else if (logicalType instanceof LogicalTypes.Date) { - // The date logical type represents a date within the calendar, with no reference to a particular time zone - // or time of day. - // - // A date logical type annotates an Avro int, where the int stores the number of days from the unix epoch, 1 - // January 1970 (ISO calendar). - return TypeDescription.createDate(); - } else if (logicalType instanceof LogicalTypes.TimeMillis) { - // The time-millis logical type represents a time of day, with no reference to a particular calendar, time - // zone or date, with a precision of one millisecond. - // - // A time-millis logical type annotates an Avro int, where the int stores the number of milliseconds after - // midnight, 00:00:00.000. - return TypeDescription.createInt(); - } else if (logicalType instanceof LogicalTypes.TimeMicros) { - // The time-micros logical type represents a time of day, with no reference to a particular calendar, time - // zone or date, with a precision of one microsecond. - // - // A time-micros logical type annotates an Avro long, where the long stores the number of microseconds after - // midnight, 00:00:00.000000. - return TypeDescription.createLong(); - } else if (logicalType instanceof LogicalTypes.TimestampMillis) { - // The timestamp-millis logical type represents an instant on the global timeline, independent of a - // particular time zone or calendar, with a precision of one millisecond. - // - // A timestamp-millis logical type annotates an Avro long, where the long stores the number of milliseconds - // from the unix epoch, 1 January 1970 00:00:00.000 UTC. - return TypeDescription.createTimestamp(); - } else if (logicalType instanceof LogicalTypes.TimestampMicros) { - // The timestamp-micros logical type represents an instant on the global timeline, independent of a - // particular time zone or calendar, with a precision of one microsecond. - // - // A timestamp-micros logical type annotates an Avro long, where the long stores the number of microseconds - // from the unix epoch, 1 January 1970 00:00:00.000000 UTC. - return TypeDescription.createTimestamp(); - } - } - - final Schema.Type type = avroSchema.getType(); switch (type) { case NULL: // empty union represents null type @@ -671,13 +622,74 @@ public static TypeDescription createOrcSchema(Schema avroSchema) { case INT: return TypeDescription.createInt(); case BYTES: + case FIXED: return TypeDescription.createBinary(); + case DECIMAL: + return TypeDescription.createDecimal() + .withPrecision(((HoodieSchema.Decimal) schema).getPrecision()) + .withScale(((HoodieSchema.Decimal) schema).getScale()); + case DATE: + // The date logical type represents a date within the calendar, with no reference to a particular time zone + // or time of day. + // + // A date logical type annotates an Avro int, where the int stores the number of days from the unix epoch, 1 + // January 1970 (ISO calendar). + return TypeDescription.createDate(); + case TIME: + HoodieSchema.Time timeSchema = (HoodieSchema.Time) schema; + if (timeSchema.getPrecision() == TimePrecision.MILLIS) { + // The time-millis logical type represents a time of day, with no reference to a particular calendar, time + // zone or date, with a precision of one millisecond. + // + // A time-millis logical type annotates an Avro int, where the int stores the number of milliseconds after + // midnight, 00:00:00.000. + return TypeDescription.createInt(); + } else if (timeSchema.getPrecision() == TimePrecision.MICROS) { + // The time-micros logical type represents a time of day, with no reference to a particular calendar, time + // zone or date, with a precision of one microsecond. + // + // A time-micros logical type annotates an Avro long, where the long stores the number of microseconds after + // midnight, 00:00:00.000000. + return TypeDescription.createLong(); + } else { + throw new IllegalStateException( + String.format("Unrecognized TimePrecision for: %s for Time type: %s", timeSchema.getPrecision(), timeSchema)); + } + case TIMESTAMP: + HoodieSchema.Timestamp timestampSchema = (HoodieSchema.Timestamp) schema; + // NOTE: Preserving old behavior from before HoodieSchema refactoring: + // - UTC-adjusted timestamps (TimestampMillis/Micros) are converted to ORC Timestamp + // - Local timestamps (LocalTimestampMillis/Micros) are converted to ORC Long + // This is because the old code did not handle LocalTimestamp logical types explicitly, + // causing them to fall through to the base type (LONG) conversion. + if (!timestampSchema.isUtcAdjusted()) { + // Local timestamp - treat as ORC Long (old behavior) + return TypeDescription.createLong(); + } + if (timestampSchema.getPrecision() == TimePrecision.MILLIS) { + // The timestamp-millis logical type represents an instant on the global timeline, independent of a + // particular time zone or calendar, with a precision of one millisecond. + // + // A timestamp-millis logical type annotates an Avro long, where the long stores the number of milliseconds + // from the unix epoch, 1 January 1970 00:00:00.000 UTC. + return TypeDescription.createTimestamp(); + } else if (timestampSchema.getPrecision() == TimePrecision.MICROS) { + // The timestamp-micros logical type represents an instant on the global timeline, independent of a + // particular time zone or calendar, with a precision of one microsecond. + // + // A timestamp-micros logical type annotates an Avro long, where the long stores the number of microseconds + // from the unix epoch, 1 January 1970 00:00:00.000000 UTC. + return TypeDescription.createTimestamp(); + } else { + throw new IllegalStateException( + String.format("Unrecognized TimePrecision for: %s for Timestamp type: %s", timestampSchema.getPrecision(), timestampSchema)); + } case ARRAY: - return TypeDescription.createList(createOrcSchema(avroSchema.getElementType())); + return TypeDescription.createList(createOrcSchema(schema.getElementType())); case RECORD: final TypeDescription recordStruct = TypeDescription.createStruct(); - for (Schema.Field field : avroSchema.getFields()) { - final Schema fieldSchema = field.schema(); + for (HoodieSchemaField field : schema.getFields()) { + final HoodieSchema fieldSchema = field.schema(); final TypeDescription fieldType = createOrcSchema(fieldSchema); if (fieldType != null) { recordStruct.addField(field.name(), fieldType); @@ -688,11 +700,11 @@ public static TypeDescription createOrcSchema(Schema avroSchema) { return TypeDescription.createMap( // in Avro maps, keys are always strings TypeDescription.createString(), - createOrcSchema(avroSchema.getValueType()) + createOrcSchema(schema.getValueType()) ); case UNION: - final List nonNullMembers = avroSchema.getTypes().stream().filter( - schema -> !Schema.Type.NULL.equals(schema.getType()) + final List nonNullMembers = schema.getTypes().stream().filter( + s -> HoodieSchemaType.NULL != s.getType() ).collect(Collectors.toList()); if (nonNullMembers.isEmpty()) { @@ -706,7 +718,7 @@ public static TypeDescription createOrcSchema(Schema avroSchema) { } else { // more than one non-null type; represent as an actual ORC union of them final TypeDescription union = TypeDescription.createUnion(); - for (final Schema childSchema : nonNullMembers) { + for (final HoodieSchema childSchema : nonNullMembers) { union.addUnionChild(createOrcSchema(childSchema)); } return union; @@ -722,68 +734,60 @@ public static TypeDescription createOrcSchema(Schema avroSchema) { case ENUM: // represent as String for now return TypeDescription.createString(); - case FIXED: - return TypeDescription.createBinary(); default: - throw new IllegalStateException(String.format("Unrecognized Avro type: %s", type.getName())); + throw new IllegalStateException(String.format("Unrecognized type: %s", type.name())); } } - public static Schema createAvroSchema(TypeDescription orcSchema) { + public static HoodieSchema createSchema(TypeDescription orcSchema) { switch (orcSchema.getCategory()) { case BOOLEAN: - return Schema.create(Schema.Type.BOOLEAN); + return HoodieSchema.create(HoodieSchemaType.BOOLEAN); case BYTE: // tinyint (8 bit), use int to hold it - return Schema.create(Schema.Type.INT); + return HoodieSchema.create(HoodieSchemaType.INT); case SHORT: // smallint (16 bit), use int to hold it - return Schema.create(Schema.Type.INT); + return HoodieSchema.create(HoodieSchemaType.INT); case INT: // the Avro logical type could be AvroTypeUtil.LOGICAL_TYPE_TIME_MILLIS, but there is no way to distinguish - return Schema.create(Schema.Type.INT); + return HoodieSchema.create(HoodieSchemaType.INT); case LONG: // the Avro logical type could be AvroTypeUtil.LOGICAL_TYPE_TIME_MICROS, but there is no way to distinguish - return Schema.create(Schema.Type.LONG); + return HoodieSchema.create(HoodieSchemaType.LONG); case FLOAT: - return Schema.create(Schema.Type.FLOAT); + return HoodieSchema.create(HoodieSchemaType.FLOAT); case DOUBLE: - return Schema.create(Schema.Type.DOUBLE); + return HoodieSchema.create(HoodieSchemaType.DOUBLE); case VARCHAR: case CHAR: case STRING: - return Schema.create(Schema.Type.STRING); + return HoodieSchema.create(HoodieSchemaType.STRING); case DATE: - Schema date = Schema.create(Schema.Type.INT); - LogicalTypes.date().addToSchema(date); - return date; + return HoodieSchema.create(HoodieSchemaType.DATE); case TIMESTAMP: // Cannot distinguish between TIMESTAMP_MILLIS and TIMESTAMP_MICROS // Assume TIMESTAMP_MILLIS because Timestamp in ORC is in millis - Schema timestamp = Schema.create(Schema.Type.LONG); - LogicalTypes.timestampMillis().addToSchema(timestamp); - return timestamp; + return HoodieSchema.createTimestampMillis(); case BINARY: - return Schema.create(Schema.Type.BYTES); + return HoodieSchema.create(HoodieSchemaType.BYTES); case DECIMAL: - Schema decimal = Schema.create(Schema.Type.BYTES); - LogicalTypes.decimal(orcSchema.getPrecision(), orcSchema.getScale()).addToSchema(decimal); - return decimal; + return HoodieSchema.createDecimal(orcSchema.getPrecision(), orcSchema.getScale()); case LIST: - return Schema.createArray(createAvroSchema(orcSchema.getChildren().get(0))); + return HoodieSchema.createArray(createSchema(orcSchema.getChildren().get(0))); case MAP: - return Schema.createMap(createAvroSchema(orcSchema.getChildren().get(1))); + return HoodieSchema.createMap(createSchema(orcSchema.getChildren().get(1))); case STRUCT: - List childFields = new ArrayList<>(); + List childFields = new ArrayList<>(); for (int i = 0; i < orcSchema.getChildren().size(); i++) { TypeDescription childType = orcSchema.getChildren().get(i); String childName = orcSchema.getFieldNames().get(i); - childFields.add(new Field(childName, createAvroSchema(childType), "", null)); + childFields.add(HoodieSchemaField.of(childName, createSchema(childType), "", null)); } - return Schema.createRecord(childFields); + return HoodieSchema.createRecord("record", null, null, childFields); case UNION: - return Schema.createUnion(orcSchema.getChildren().stream() - .map(AvroOrcUtils::createAvroSchema) + return HoodieSchema.createUnion(orcSchema.getChildren().stream() + .map(AvroOrcUtils::createSchema) .collect(Collectors.toList())); default: throw new IllegalStateException(String.format("Unrecognized ORC type: %s", orcSchema.getCategory().getName())); @@ -797,18 +801,18 @@ public static Schema createAvroSchema(TypeDescription orcSchema) { * the nullability of an Avro type. To achieve consistency between the Avro and ORC schema, * non-NULL types are extracted from the union type. * @param unionSchema A schema of union type. - * @return An Avro schema that is either NULL or a UNION without NULL fields. + * @return A schema that is either NULL or a UNION without NULL fields. */ - private static Schema getActualSchemaType(Schema unionSchema) { - final List nonNullMembers = unionSchema.getTypes().stream().filter( - schema -> !Schema.Type.NULL.equals(schema.getType()) - ).collect(Collectors.toList()); + private static HoodieSchema getActualSchemaType(HoodieSchema unionSchema) { + final List nonNullMembers = unionSchema.getTypes().stream() + .filter(schema -> HoodieSchemaType.NULL != schema.getType()) + .collect(Collectors.toList()); if (nonNullMembers.isEmpty()) { - return Schema.create(Schema.Type.NULL); + return HoodieSchema.create(HoodieSchemaType.NULL); } else if (nonNullMembers.size() == 1) { return nonNullMembers.get(0); } else { - return Schema.createUnion(nonNullMembers); + return HoodieSchema.createUnion(nonNullMembers); } } diff --git a/hudi-hadoop-common/src/main/java/org/apache/hudi/common/util/HoodieAvroParquetReaderIterator.java b/hudi-hadoop-common/src/main/java/org/apache/hudi/common/util/HoodieAvroParquetReaderIterator.java index 8a72d48609cd5..bc6be9db8641f 100644 --- a/hudi-hadoop-common/src/main/java/org/apache/hudi/common/util/HoodieAvroParquetReaderIterator.java +++ b/hudi-hadoop-common/src/main/java/org/apache/hudi/common/util/HoodieAvroParquetReaderIterator.java @@ -20,18 +20,18 @@ package org.apache.hudi.common.util; import org.apache.hudi.avro.HoodieAvroUtils; +import org.apache.hudi.common.schema.HoodieSchema; -import org.apache.avro.Schema; import org.apache.avro.generic.IndexedRecord; import org.apache.parquet.hadoop.ParquetReader; import java.util.Map; public class HoodieAvroParquetReaderIterator extends ParquetReaderIterator { - private final Schema promotedSchema; + private final HoodieSchema promotedSchema; private final Map renamedColumns; - public HoodieAvroParquetReaderIterator(ParquetReader parquetReader, Schema promotedSchema, Map renamedColumns) { + public HoodieAvroParquetReaderIterator(ParquetReader parquetReader, HoodieSchema promotedSchema, Map renamedColumns) { super(parquetReader); this.promotedSchema = promotedSchema; this.renamedColumns = renamedColumns; @@ -39,6 +39,6 @@ public HoodieAvroParquetReaderIterator(ParquetReader parquetReade @Override public IndexedRecord next() { - return HoodieAvroUtils.rewriteRecordWithNewSchema(super.next(), promotedSchema, renamedColumns); + return HoodieAvroUtils.rewriteRecordWithNewSchema(super.next(), promotedSchema.toAvroSchema(), renamedColumns); } } diff --git a/hudi-hadoop-common/src/main/java/org/apache/hudi/common/util/OrcUtils.java b/hudi-hadoop-common/src/main/java/org/apache/hudi/common/util/OrcUtils.java index b41ca863b8d83..532e7f954d16c 100644 --- a/hudi-hadoop-common/src/main/java/org/apache/hudi/common/util/OrcUtils.java +++ b/hudi-hadoop-common/src/main/java/org/apache/hudi/common/util/OrcUtils.java @@ -18,11 +18,12 @@ package org.apache.hudi.common.util; -import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType; import org.apache.hudi.common.schema.HoodieSchema; +import org.apache.hudi.common.schema.HoodieSchemaUtils; import org.apache.hudi.common.util.collection.ClosableIterator; import org.apache.hudi.common.util.collection.CloseableMappingIterator; import org.apache.hudi.common.util.collection.Pair; @@ -37,7 +38,6 @@ import org.apache.hudi.storage.HoodieStorage; import org.apache.hudi.storage.StoragePath; -import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; @@ -119,7 +119,7 @@ public ClosableIterator getHoodieKeyIterator(HoodieStorage storage, S conf.addResource(HadoopFSUtils.getFs(filePath.toString(), conf).getConf()); Reader reader = OrcFile.createReader(convertToHadoopPath(filePath), OrcFile.readerOptions(conf)); - Schema readSchema = getKeyIteratorSchema(storage, filePath, keyGeneratorOpt, partitionPath); + HoodieSchema readSchema = getKeyIteratorSchema(storage, filePath, keyGeneratorOpt, partitionPath); TypeDescription orcSchema = AvroOrcUtils.createOrcSchema(readSchema); RecordReader recordReader = reader.rows(new Options(conf).schema(orcSchema)); @@ -135,28 +135,28 @@ public ClosableIterator getHoodieKeyIterator(HoodieStorage storage, S */ @Override public List readAvroRecords(HoodieStorage storage, StoragePath filePath) { - Schema avroSchema; + HoodieSchema schema; try (Reader reader = OrcFile.createReader( convertToHadoopPath(filePath), OrcFile.readerOptions(storage.getConf().unwrapAs(Configuration.class)))) { - avroSchema = AvroOrcUtils.createAvroSchema(reader.getSchema()); + schema = AvroOrcUtils.createSchema(reader.getSchema()); } catch (IOException io) { throw new HoodieIOException("Unable to read Avro records from an ORC file:" + filePath, io); } - return readAvroRecords(storage, filePath, avroSchema); + return readAvroRecords(storage, filePath, schema); } /** * NOTE: This literally reads the entire file contents, thus should be used with caution. */ @Override - public List readAvroRecords(HoodieStorage storage, StoragePath filePath, Schema avroSchema) { + public List readAvroRecords(HoodieStorage storage, StoragePath filePath, HoodieSchema schema) { List records = new ArrayList<>(); try (Reader reader = OrcFile.createReader( convertToHadoopPath(filePath), OrcFile.readerOptions(storage.getConf().unwrapAs(Configuration.class)))) { TypeDescription orcSchema = reader.getSchema(); try (RecordReader recordReader = reader.rows( new Options(storage.getConf().unwrapAs(Configuration.class)).schema(orcSchema))) { - OrcReaderIterator iterator = new OrcReaderIterator<>(recordReader, avroSchema, orcSchema); + OrcReaderIterator iterator = new OrcReaderIterator<>(recordReader, schema, orcSchema); while (iterator.hasNext()) { GenericRecord record = iterator.next(); records.add(record); @@ -242,16 +242,16 @@ public Map readFooter(HoodieStorage storage, boolean required, } @Override - public Schema readAvroSchema(HoodieStorage storage, StoragePath filePath) { + public HoodieSchema readSchema(HoodieStorage storage, StoragePath filePath) { try (Reader reader = OrcFile.createReader( convertToHadoopPath(filePath), OrcFile.readerOptions(storage.getConf().unwrapAs(Configuration.class)))) { if (reader.hasMetadataValue("orc.avro.schema")) { ByteBuffer metadataValue = reader.getMetadataValue("orc.avro.schema"); byte[] bytes = toBytes(metadataValue); - return new Schema.Parser().parse(new String(bytes)); + return HoodieSchema.parse(new String(bytes)); } else { TypeDescription orcSchema = reader.getSchema(); - return AvroOrcUtils.createAvroSchema(orcSchema); + return AvroOrcUtils.createSchema(orcSchema); } } catch (IOException io) { throw new HoodieIOException("Unable to get Avro schema for ORC file:" + filePath, io); @@ -283,7 +283,7 @@ public long getRowCount(HoodieStorage storage, StoragePath filePath) { public void writeMetaFile(HoodieStorage storage, StoragePath filePath, Properties props) throws IOException { // Since we are only interested in saving metadata to the footer, the schema, blocksizes and other // parameters are not important. - Schema schema = HoodieAvroUtils.getRecordKeySchema(); + HoodieSchema schema = HoodieSchemaUtils.getRecordKeySchema(); OrcFile.WriterOptions writerOptions = OrcFile.writerOptions(storage.getConf().unwrapAs(Configuration.class)) .fileSystem((FileSystem) storage.getFileSystem()) .setSchema(AvroOrcUtils.createOrcSchema(schema)); @@ -307,9 +307,9 @@ public ByteArrayOutputStream serializeRecordsToLogBlock(HoodieStorage storage, @Override public Pair serializeRecordsToLogBlock(HoodieStorage storage, Iterator records, - HoodieRecord.HoodieRecordType recordType, - Schema writerSchema, - Schema readerSchema, + HoodieRecordType recordType, + HoodieSchema writerSchema, + HoodieSchema readerSchema, String keyFieldName, Map paramsMap) throws IOException { throw new UnsupportedOperationException("Hudi log blocks do not support ORC format yet"); diff --git a/hudi-hadoop-common/src/main/java/org/apache/hudi/common/util/ParquetUtils.java b/hudi-hadoop-common/src/main/java/org/apache/hudi/common/util/ParquetUtils.java index 0d8cdc3f99402..9ad7d48eb5d8c 100644 --- a/hudi-hadoop-common/src/main/java/org/apache/hudi/common/util/ParquetUtils.java +++ b/hudi-hadoop-common/src/main/java/org/apache/hudi/common/util/ParquetUtils.java @@ -19,13 +19,14 @@ package org.apache.hudi.common.util; -import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.avro.HoodieAvroWriteSupport; import org.apache.hudi.common.config.HoodieConfig; import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType; import org.apache.hudi.common.schema.HoodieSchema; +import org.apache.hudi.common.schema.HoodieSchemaUtils; import org.apache.hudi.common.util.collection.ClosableIterator; import org.apache.hudi.common.util.collection.CloseableMappingIterator; import org.apache.hudi.common.util.collection.Pair; @@ -42,7 +43,6 @@ import org.apache.hudi.storage.StoragePath; import lombok.extern.slf4j.Slf4j; -import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; @@ -107,7 +107,7 @@ public class ParquetUtils extends FileFormatUtils { */ @Override public Set> filterRowKeys(HoodieStorage storage, StoragePath filePath, Set filter) { - return filterParquetRowKeys(storage, new Path(filePath.toUri()), filter, HoodieAvroUtils.getRecordKeySchema()); + return filterParquetRowKeys(storage, new Path(filePath.toUri()), filter, HoodieSchemaUtils.getRecordKeySchema()); } public static ParquetMetadata readMetadata(HoodieStorage storage, StoragePath parquetFilePath) { @@ -143,15 +143,15 @@ private static ParquetMetadata readMetadata(HoodieStorage storage, StoragePath p */ private static Set> filterParquetRowKeys(HoodieStorage storage, Path filePath, Set filter, - Schema readSchema) { + HoodieSchema readSchema) { Option filterFunction = Option.empty(); if (filter != null && !filter.isEmpty()) { filterFunction = Option.of(new RecordKeysFilterFunction(filter)); } Configuration conf = storage.getConf().unwrapCopyAs(Configuration.class); conf.addResource(storage.newInstance(convertToStoragePath(filePath), storage.getConf()).getConf().unwrapAs(Configuration.class)); - AvroReadSupport.setAvroReadSchema(conf, readSchema); - AvroReadSupport.setRequestedProjection(conf, readSchema); + AvroReadSupport.setAvroReadSchema(conf, readSchema.toAvroSchema()); + AvroReadSupport.setRequestedProjection(conf, readSchema.toAvroSchema()); Set> rowKeys = new HashSet<>(); long rowPosition = 0; try (ParquetReader reader = AvroParquetReader.builder(filePath).withConf(conf).build()) { @@ -213,9 +213,9 @@ public ClosableIterator getHoodieKeyIterator(HoodieStorage storage, S try { Configuration conf = storage.getConf().unwrapCopyAs(Configuration.class); conf.addResource(storage.newInstance(filePath, storage.getConf()).getConf().unwrapAs(Configuration.class)); - Schema readSchema = getKeyIteratorSchema(storage, filePath, keyGeneratorOpt, partitionPath); - AvroReadSupport.setAvroReadSchema(conf, readSchema); - AvroReadSupport.setRequestedProjection(conf, readSchema); + HoodieSchema readSchema = getKeyIteratorSchema(storage, filePath, keyGeneratorOpt, partitionPath); + AvroReadSupport.setAvroReadSchema(conf, readSchema.toAvroSchema()); + AvroReadSupport.setRequestedProjection(conf, readSchema.toAvroSchema()); ParquetReader reader = AvroParquetReader.builder(new Path(filePath.toUri())).withConf(conf).build(); return HoodieKeyIterator.getInstance(new ParquetReaderIterator<>(reader), keyGeneratorOpt, partitionPath); @@ -242,7 +242,7 @@ public ClosableIterator> fetchRecordKeysWithPositions(Hood /** * Get the schema of the given parquet file. */ - public MessageType readSchema(HoodieStorage storage, StoragePath parquetFilePath) { + public MessageType readMessageType(HoodieStorage storage, StoragePath parquetFilePath) { return readFileMetadataOnly(storage, parquetFilePath).getFileMetaData().getSchema(); } @@ -253,7 +253,7 @@ public MessageType readSchema(HoodieStorage storage, StoragePath parquetFilePath public static Integer readSchemaHash(HoodieStorage storage, StoragePath parquetFilePath) { try { ParquetUtils parquetUtils = new ParquetUtils(); - MessageType schema = parquetUtils.readSchema(storage, parquetFilePath); + MessageType schema = parquetUtils.readMessageType(storage, parquetFilePath); return schema.hashCode(); } catch (Exception e) { log.warn("Failed to read schema hash from file: {}", parquetFilePath, e); @@ -279,8 +279,8 @@ public Map readFooter(HoodieStorage storage, boolean required, } @Override - public Schema readAvroSchema(HoodieStorage storage, StoragePath filePath) { - MessageType parquetSchema = readSchema(storage, filePath); + public HoodieSchema readSchema(HoodieStorage storage, StoragePath filePath) { + MessageType parquetSchema = readMessageType(storage, filePath); return getAvroSchemaConverter(storage.getConf().unwrapAs(Configuration.class)).convert(parquetSchema); } @@ -369,8 +369,8 @@ public List readAvroRecords(HoodieStorage storage, StoragePath fi } @Override - public List readAvroRecords(HoodieStorage storage, StoragePath filePath, Schema schema) { - AvroReadSupport.setAvroReadSchema(storage.getConf().unwrapAs(Configuration.class), schema); + public List readAvroRecords(HoodieStorage storage, StoragePath filePath, HoodieSchema schema) { + AvroReadSupport.setAvroReadSchema(storage.getConf().unwrapAs(Configuration.class), schema.toAvroSchema()); return readAvroRecords(storage, filePath); } @@ -397,7 +397,7 @@ public void writeMetaFile(HoodieStorage storage, Properties props) throws IOException { // Since we are only interested in saving metadata to the footer, the schema, blocksizes and other // parameters are not important. - Schema schema = HoodieAvroUtils.getRecordKeySchema(); + HoodieSchema schema = HoodieSchemaUtils.getRecordKeySchema(); MessageType type = Types.buildMessage().optional(PrimitiveType.PrimitiveTypeName.INT64).named("dummyint").named("dummy"); HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(type, schema, Option.empty(), new Properties()); try (ParquetWriter writer = new ParquetWriter(new Path(filePath.toUri()), writeSupport, CompressionCodecName.UNCOMPRESSED, 1024, 1024)) { @@ -441,9 +441,9 @@ public ByteArrayOutputStream serializeRecordsToLogBlock(HoodieStorage storage, @Override public Pair serializeRecordsToLogBlock(HoodieStorage storage, Iterator recordItr, - HoodieRecord.HoodieRecordType recordType, - Schema writerSchema, - Schema readerSchema, + HoodieRecordType recordType, + HoodieSchema writerSchema, + HoodieSchema readerSchema, String keyFieldName, Map paramsMap) throws IOException { ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); @@ -453,13 +453,12 @@ public Pair serializeRecordsToLogBlock(HoodieStor config.setValue(PARQUET_PAGE_SIZE.key(), String.valueOf(ParquetWriter.DEFAULT_PAGE_SIZE)); config.setValue(PARQUET_MAX_FILE_SIZE.key(), String.valueOf(1024 * 1024 * 1024)); - HoodieSchema schema = HoodieSchema.fromAvroSchema(writerSchema); HoodieFileWriter parquetWriter = HoodieFileWriterFactory.getFileWriter( - HoodieFileFormat.PARQUET, outputStream, storage, config, schema, recordType); + HoodieFileFormat.PARQUET, outputStream, storage, config, writerSchema, recordType); while (recordItr.hasNext()) { HoodieRecord record = recordItr.next(); - String recordKey = record.getRecordKey(readerSchema, keyFieldName); - parquetWriter.write(recordKey, record, schema); + String recordKey = record.getRecordKey(readerSchema.toAvroSchema(), keyFieldName); + parquetWriter.write(recordKey, record, writerSchema); } outputStream.flush(); parquetWriter.close(); diff --git a/hudi-hadoop-common/src/main/java/org/apache/hudi/io/hadoop/OrcReaderIterator.java b/hudi-hadoop-common/src/main/java/org/apache/hudi/io/hadoop/OrcReaderIterator.java index 0c2758e2d63f4..fce039e2fa558 100644 --- a/hudi-hadoop-common/src/main/java/org/apache/hudi/io/hadoop/OrcReaderIterator.java +++ b/hudi-hadoop-common/src/main/java/org/apache/hudi/io/hadoop/OrcReaderIterator.java @@ -19,12 +19,13 @@ package org.apache.hudi.io.hadoop; +import org.apache.hudi.common.schema.HoodieSchema; import org.apache.hudi.common.util.AvroOrcUtils; -import org.apache.hudi.io.util.FileIOUtils; import org.apache.hudi.common.util.collection.ClosableIterator; import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.internal.schema.HoodieSchemaException; +import org.apache.hudi.io.util.FileIOUtils; -import org.apache.avro.Schema; import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericData.Record; import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch; @@ -40,22 +41,24 @@ public class OrcReaderIterator implements ClosableIterator { private final RecordReader recordReader; - private final Schema avroSchema; + private final HoodieSchema schema; private final List fieldNames; private final List orcFieldTypes; - private final Schema[] avroFieldSchemas; + private final HoodieSchema[] fieldSchemas; private final VectorizedRowBatch batch; private int rowInBatch; private T next; - public OrcReaderIterator(RecordReader recordReader, Schema schema, TypeDescription orcSchema) { + public OrcReaderIterator(RecordReader recordReader, HoodieSchema schema, TypeDescription orcSchema) { this.recordReader = recordReader; - this.avroSchema = schema; + this.schema = schema; this.fieldNames = orcSchema.getFieldNames(); this.orcFieldTypes = orcSchema.getChildren(); - this.avroFieldSchemas = fieldNames.stream() - .map(fieldName -> avroSchema.getField(fieldName).schema()) - .toArray(Schema[]::new); + this.fieldSchemas = fieldNames.stream() + .map(fieldName -> this.schema.getField(fieldName) + .orElseThrow(() -> new HoodieSchemaException("Field not found: " + fieldName)) + .schema()) + .toArray(HoodieSchema[]::new); this.batch = orcSchema.createRowBatch(); this.rowInBatch = 0; } @@ -109,10 +112,10 @@ private GenericData.Record readRecordFromBatch() throws IOException { return null; } - GenericData.Record record = new Record(avroSchema); + GenericData.Record record = new Record(schema.toAvroSchema()); int numFields = orcFieldTypes.size(); for (int i = 0; i < numFields; i++) { - Object data = AvroOrcUtils.readFromVector(orcFieldTypes.get(i), batch.cols[i], avroFieldSchemas[i], rowInBatch); + Object data = AvroOrcUtils.readFromVector(orcFieldTypes.get(i), batch.cols[i], fieldSchemas[i], rowInBatch); record.put(fieldNames.get(i), data); } rowInBatch++; diff --git a/hudi-hadoop-common/src/main/java/org/apache/hudi/io/storage/hadoop/HoodieAvroFileWriterFactory.java b/hudi-hadoop-common/src/main/java/org/apache/hudi/io/storage/hadoop/HoodieAvroFileWriterFactory.java index 2d16d6343d27c..2861cfac30236 100644 --- a/hudi-hadoop-common/src/main/java/org/apache/hudi/io/storage/hadoop/HoodieAvroFileWriterFactory.java +++ b/hudi-hadoop-common/src/main/java/org/apache/hudi/io/storage/hadoop/HoodieAvroFileWriterFactory.java @@ -38,7 +38,6 @@ import org.apache.hudi.storage.HoodieStorage; import org.apache.hudi.storage.StoragePath; -import org.apache.avro.Schema; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.orc.CompressionKind; @@ -62,7 +61,7 @@ protected HoodieFileWriter newParquetFileWriter( String instantTime, StoragePath path, HoodieConfig config, HoodieSchema schema, TaskContextSupplier taskContextSupplier) throws IOException { boolean populateMetaFields = config.getBooleanOrDefault(HoodieTableConfig.POPULATE_META_FIELDS); - HoodieAvroWriteSupport writeSupport = getHoodieAvroWriteSupport(schema.getAvroSchema(), config, enableBloomFilter(populateMetaFields, config)); + HoodieAvroWriteSupport writeSupport = getHoodieAvroWriteSupport(schema, config, enableBloomFilter(populateMetaFields, config)); String compressionCodecName = config.getStringOrDefault(HoodieStorageConfig.PARQUET_COMPRESSION_CODEC_NAME); // Support PARQUET_COMPRESSION_CODEC_NAME is "" @@ -81,7 +80,7 @@ protected HoodieFileWriter newParquetFileWriter( protected HoodieFileWriter newParquetFileWriter( OutputStream outputStream, HoodieConfig config, HoodieSchema schema) throws IOException { - HoodieAvroWriteSupport writeSupport = getHoodieAvroWriteSupport(schema.getAvroSchema(), config, false); + HoodieAvroWriteSupport writeSupport = getHoodieAvroWriteSupport(schema, config, false); HoodieParquetConfig parquetConfig = new HoodieParquetConfig<>(writeSupport, CompressionCodecName.fromConf(config.getString(HoodieStorageConfig.PARQUET_COMPRESSION_CODEC_NAME)), config.getInt(HoodieStorageConfig.PARQUET_BLOCK_SIZE), @@ -103,7 +102,7 @@ protected HoodieFileWriter newHFileFileWriter( config.getInt(HoodieStorageConfig.HFILE_BLOCK_SIZE), config.getLong(HoodieStorageConfig.HFILE_MAX_FILE_SIZE), HoodieAvroHFileReaderImplBase.KEY_FIELD_NAME, filter); - return new HoodieAvroHFileWriter(instantTime, path, hfileConfig, schema.getAvroSchema(), taskContextSupplier, config.getBoolean(HoodieTableConfig.POPULATE_META_FIELDS)); + return new HoodieAvroHFileWriter(instantTime, path, hfileConfig, schema, taskContextSupplier, config.getBoolean(HoodieTableConfig.POPULATE_META_FIELDS)); } protected HoodieFileWriter newOrcFileWriter( @@ -115,15 +114,15 @@ protected HoodieFileWriter newOrcFileWriter( config.getInt(HoodieStorageConfig.ORC_STRIPE_SIZE), config.getInt(HoodieStorageConfig.ORC_BLOCK_SIZE), config.getLong(HoodieStorageConfig.ORC_FILE_MAX_SIZE), filter); - return new HoodieAvroOrcWriter(instantTime, path, orcConfig, schema.getAvroSchema(), taskContextSupplier); + return new HoodieAvroOrcWriter(instantTime, path, orcConfig, schema, taskContextSupplier); } - private HoodieAvroWriteSupport getHoodieAvroWriteSupport(Schema schema, + private HoodieAvroWriteSupport getHoodieAvroWriteSupport(HoodieSchema schema, HoodieConfig config, boolean enableBloomFilter) { Option filter = enableBloomFilter ? Option.of(createBloomFilter(config)) : Option.empty(); return (HoodieAvroWriteSupport) ReflectionUtils.loadClass( config.getStringOrDefault(HoodieStorageConfig.HOODIE_AVRO_WRITE_SUPPORT_CLASS), - new Class[] {MessageType.class, Schema.class, Option.class, Properties.class}, + new Class[] {MessageType.class, HoodieSchema.class, Option.class, Properties.class}, getAvroSchemaConverter(storage.getConf().unwrapAs(Configuration.class)).convert(schema), schema, filter, config.getProps()); } } diff --git a/hudi-hadoop-common/src/main/java/org/apache/hudi/io/storage/hadoop/HoodieAvroHFileWriter.java b/hudi-hadoop-common/src/main/java/org/apache/hudi/io/storage/hadoop/HoodieAvroHFileWriter.java index ca46018e0d172..0c8596fa7ccee 100644 --- a/hudi-hadoop-common/src/main/java/org/apache/hudi/io/storage/hadoop/HoodieAvroHFileWriter.java +++ b/hudi-hadoop-common/src/main/java/org/apache/hudi/io/storage/hadoop/HoodieAvroHFileWriter.java @@ -23,6 +23,8 @@ import org.apache.hudi.common.bloom.BloomFilter; import org.apache.hudi.common.engine.TaskContextSupplier; import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.schema.HoodieSchema; +import org.apache.hudi.common.schema.HoodieSchemaField; import org.apache.hudi.common.util.Option; import org.apache.hudi.exception.HoodieDuplicateKeyException; import org.apache.hudi.hadoop.fs.HadoopFSUtils; @@ -38,7 +40,6 @@ import org.apache.hudi.storage.StoragePath; import org.apache.hudi.storage.hadoop.HadoopStorageConfiguration; -import org.apache.avro.Schema; import org.apache.avro.generic.IndexedRecord; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; @@ -70,13 +71,13 @@ public class HoodieAvroHFileWriter private final String instantTime; private final TaskContextSupplier taskContextSupplier; private final boolean populateMetaFields; - private final Option keyFieldSchema; + private final Option keyFieldSchema; private HFileWriter writer; private String minRecordKey; private String maxRecordKey; private String prevRecordKey; - public HoodieAvroHFileWriter(String instantTime, StoragePath file, HoodieHFileConfig hfileConfig, Schema schema, + public HoodieAvroHFileWriter(String instantTime, StoragePath file, HoodieHFileConfig hfileConfig, HoodieSchema schema, TaskContextSupplier taskContextSupplier, boolean populateMetaFields) throws IOException { Configuration conf = HadoopFSUtils.registerFileSystem(file, (Configuration) hfileConfig.getStorageConf().unwrap()); this.file = HoodieWrapperFileSystem.convertToHoodiePath(file, conf); @@ -84,7 +85,7 @@ public HoodieAvroHFileWriter(String instantTime, StoragePath file, HoodieHFileCo this.isWrapperFileSystem = fs instanceof HoodieWrapperFileSystem; this.wrapperFs = this.isWrapperFileSystem ? Option.of((HoodieWrapperFileSystem) fs) : Option.empty(); this.hfileConfig = hfileConfig; - this.keyFieldSchema = Option.ofNullable(schema.getField(hfileConfig.getKeyFieldName())); + this.keyFieldSchema = schema.getField(hfileConfig.getKeyFieldName()); // TODO - compute this compression ratio dynamically by looking at the bytes written to the // stream and the actual file size reported by HDFS diff --git a/hudi-hadoop-common/src/main/java/org/apache/hudi/io/storage/hadoop/HoodieAvroOrcReader.java b/hudi-hadoop-common/src/main/java/org/apache/hudi/io/storage/hadoop/HoodieAvroOrcReader.java index d1b5e0e98b261..e362eac74efe2 100644 --- a/hudi-hadoop-common/src/main/java/org/apache/hudi/io/storage/hadoop/HoodieAvroOrcReader.java +++ b/hudi-hadoop-common/src/main/java/org/apache/hudi/io/storage/hadoop/HoodieAvroOrcReader.java @@ -24,6 +24,7 @@ import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.schema.HoodieSchema; import org.apache.hudi.common.schema.HoodieSchemaField; +import org.apache.hudi.common.schema.HoodieSchemaUtils; import org.apache.hudi.common.util.AvroOrcUtils; import org.apache.hudi.common.util.FileFormatUtils; import org.apache.hudi.common.util.collection.ClosableIterator; @@ -37,7 +38,6 @@ import org.apache.hudi.storage.HoodieStorage; import org.apache.hudi.storage.StoragePath; -import org.apache.avro.Schema; import org.apache.avro.generic.IndexedRecord; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; @@ -94,11 +94,11 @@ public ClosableIterator getIndexedRecordIterator(HoodieSchema rea Configuration hadoopConf = storage.getConf().unwrapCopyAs(Configuration.class); try (Reader reader = OrcFile.createReader(new Path(path.toUri()), OrcFile.readerOptions(hadoopConf))) { // Limit the ORC schema to requested fields only - Schema fileSchema = AvroOrcUtils.createAvroSchema(reader.getSchema()); + HoodieSchema fileSchema = AvroOrcUtils.createSchema(reader.getSchema()); Set existingFields = fileSchema.getFields().stream() - .map(Schema.Field::name) + .map(HoodieSchemaField::name) .collect(Collectors.toSet()); - Schema prunedFileSchema = HoodieAvroUtils.projectSchema(fileSchema, requestedSchema.getFields().stream().map(HoodieSchemaField::name) + HoodieSchema prunedFileSchema = HoodieSchemaUtils.projectSchema(fileSchema, requestedSchema.getFields().stream().map(HoodieSchemaField::name) .filter(existingFields::contains).collect(Collectors.toList())); TypeDescription orcSchema = AvroOrcUtils.createOrcSchema(prunedFileSchema); RecordReader recordReader = reader.rows(new Options(hadoopConf).schema(orcSchema)); @@ -136,8 +136,7 @@ public void close() { @Override public HoodieSchema getSchema() { - //TODO boundary for now to revisit in later pr to directly use HoodieSchema - return HoodieSchema.fromAvroSchema(orcUtils.readAvroSchema(storage, path)); + return orcUtils.readSchema(storage, path); } @Override diff --git a/hudi-hadoop-common/src/main/java/org/apache/hudi/io/storage/hadoop/HoodieAvroOrcWriter.java b/hudi-hadoop-common/src/main/java/org/apache/hudi/io/storage/hadoop/HoodieAvroOrcWriter.java index 61710c67636ad..5109d3cdf6163 100644 --- a/hudi-hadoop-common/src/main/java/org/apache/hudi/io/storage/hadoop/HoodieAvroOrcWriter.java +++ b/hudi-hadoop-common/src/main/java/org/apache/hudi/io/storage/hadoop/HoodieAvroOrcWriter.java @@ -24,6 +24,8 @@ import org.apache.hudi.common.bloom.HoodieDynamicBoundedBloomFilter; import org.apache.hudi.common.engine.TaskContextSupplier; import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.schema.HoodieSchema; +import org.apache.hudi.common.schema.HoodieSchemaField; import org.apache.hudi.common.util.AvroOrcUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.hadoop.fs.HadoopFSUtils; @@ -32,7 +34,6 @@ import org.apache.hudi.io.storage.HoodieOrcConfig; import org.apache.hudi.storage.StoragePath; -import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.IndexedRecord; import org.apache.hadoop.conf.Configuration; @@ -56,7 +57,7 @@ public class HoodieAvroOrcWriter implements HoodieAvroFileWriter, Closeable { private static final AtomicLong RECORD_INDEX = new AtomicLong(1); private final long maxFileSize; - private final Schema avroSchema; + private final HoodieSchema schema; private final List fieldTypes; private final List fieldNames; private final VectorizedRowBatch batch; @@ -72,7 +73,7 @@ public class HoodieAvroOrcWriter implements HoodieAvroFileWriter, Closeable { private String minRecordKey; private String maxRecordKey; - public HoodieAvroOrcWriter(String instantTime, StoragePath file, HoodieOrcConfig config, Schema schema, + public HoodieAvroOrcWriter(String instantTime, StoragePath file, HoodieOrcConfig config, HoodieSchema schema, TaskContextSupplier taskContextSupplier) throws IOException { Configuration conf = HadoopFSUtils.registerFileSystem(file, config.getStorageConf().unwrapAs(Configuration.class)); @@ -83,8 +84,8 @@ public HoodieAvroOrcWriter(String instantTime, StoragePath file, HoodieOrcConfig this.instantTime = instantTime; this.taskContextSupplier = taskContextSupplier; - this.avroSchema = schema; - final TypeDescription orcSchema = AvroOrcUtils.createOrcSchema(avroSchema); + this.schema = schema; + final TypeDescription orcSchema = AvroOrcUtils.createOrcSchema(this.schema); this.fieldTypes = orcSchema.getChildren(); this.fieldNames = orcSchema.getFieldNames(); this.maxFileSize = config.getMaxFileSize(); @@ -120,8 +121,8 @@ public void writeAvro(String recordKey, IndexedRecord object) throws IOException final TypeDescription type = fieldTypes.get(col); Object fieldValue = ((GenericRecord) object).get(thisField); - Schema.Field avroField = avroSchema.getField(thisField); - AvroOrcUtils.addToVector(type, colVector, avroField.schema(), fieldValue, batch.size); + HoodieSchemaField field = schema.getField(thisField).get(); + AvroOrcUtils.addToVector(type, colVector, field.schema(), fieldValue, batch.size); } batch.size++; @@ -168,7 +169,7 @@ public void close() throws IOException { writer.addUserMetadata(HoodieBloomFilterWriteSupport.HOODIE_BLOOM_FILTER_TYPE_CODE, ByteBuffer.wrap(getUTF8Bytes(bloomFilter.getBloomFilterTypeCode().name()))); } } - writer.addUserMetadata(HoodieOrcConfig.AVRO_SCHEMA_METADATA_KEY, ByteBuffer.wrap(getUTF8Bytes(avroSchema.toString()))); + writer.addUserMetadata(HoodieOrcConfig.AVRO_SCHEMA_METADATA_KEY, ByteBuffer.wrap(getUTF8Bytes(schema.toString()))); writer.close(); } diff --git a/hudi-hadoop-common/src/main/java/org/apache/hudi/io/storage/hadoop/HoodieAvroParquetReader.java b/hudi-hadoop-common/src/main/java/org/apache/hudi/io/storage/hadoop/HoodieAvroParquetReader.java index 9f7bc80cd894d..704ff5448d47c 100644 --- a/hudi-hadoop-common/src/main/java/org/apache/hudi/io/storage/hadoop/HoodieAvroParquetReader.java +++ b/hudi-hadoop-common/src/main/java/org/apache/hudi/io/storage/hadoop/HoodieAvroParquetReader.java @@ -70,7 +70,7 @@ public class HoodieAvroParquetReader extends HoodieAvroFileReader { private final HoodieStorage storage; private final FileFormatUtils parquetUtils; private final List readerIterators = new ArrayList<>(); - private Option fileSchema = Option.empty(); + private Option fileSchema = Option.empty(); public HoodieAvroParquetReader(HoodieStorage storage, StoragePath path) { // We have to clone the Hadoop Config as it might be subsequently modified @@ -107,28 +107,28 @@ public Set> filterRowKeys(Set candidateRowKeys) { @Override protected ClosableIterator getIndexedRecordIterator(HoodieSchema schema) throws IOException { - //TODO boundary for now to revisit in later pr to use HoodieSchema - return getIndexedRecordIteratorInternal(schema.getAvroSchema(), Collections.emptyMap()); + return getIndexedRecordIteratorInternal(schema, Collections.emptyMap()); } @Override public ClosableIterator getIndexedRecordIterator(HoodieSchema readerSchema, HoodieSchema requestedSchema) throws IOException { - //TODO boundary for now to revisit in later pr to use HoodieSchema - return getIndexedRecordIteratorInternal(requestedSchema.getAvroSchema(), Collections.emptyMap()); + return getIndexedRecordIteratorInternal(requestedSchema, Collections.emptyMap()); } @Override public ClosableIterator getIndexedRecordIterator(HoodieSchema readerSchema, HoodieSchema requestedSchema, Map renamedColumns) throws IOException { - //TODO boundary for now to revisit in later pr to use HoodieSchema - return getIndexedRecordIteratorInternal(requestedSchema.getAvroSchema(), renamedColumns); + return getIndexedRecordIteratorInternal(requestedSchema, renamedColumns); } @Override public HoodieSchema getSchema() { - if (fileSchema.isEmpty()) { - fileSchema = Option.ofNullable(parquetUtils.readAvroSchema(storage, path)); - } - return HoodieSchema.fromAvroSchema(fileSchema.get()); + // Lazy initialization with caching: read schema from parquet file footer on first call, + // then cache it in fileSchema to avoid repeated I/O on subsequent calls + return fileSchema.orElseGet(() -> { + HoodieSchema schema = parquetUtils.readSchema(storage, path); + fileSchema = Option.ofNullable(schema); + return schema; + }); } @Override @@ -181,21 +181,21 @@ private static StorageConfiguration tryOverrideDefaultConfigs(StorageConfigur return conf; } - private ClosableIterator getIndexedRecordIteratorInternal(Schema schema, Map renamedColumns) throws IOException { + private ClosableIterator getIndexedRecordIteratorInternal(HoodieSchema schema, Map 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) Configuration hadoopConf = storage.getConf().unwrapCopyAs(Configuration.class); //TODO boundary for now to revisit in later pr to use HoodieSchema - Schema repairedFileSchema = AvroSchemaRepair.repairLogicalTypes(getSchema().getAvroSchema(), schema); + Schema repairedFileSchema = AvroSchemaRepair.repairLogicalTypes(getSchema().toAvroSchema(), schema.toAvroSchema()); Option promotedSchema = Option.empty(); - if (!renamedColumns.isEmpty() || HoodieAvroUtils.recordNeedsRewriteForExtendedAvroTypePromotion(repairedFileSchema, schema)) { + if (!renamedColumns.isEmpty() || HoodieAvroUtils.recordNeedsRewriteForExtendedAvroTypePromotion(repairedFileSchema, schema.toAvroSchema())) { AvroReadSupport.setAvroReadSchema(hadoopConf, repairedFileSchema); AvroReadSupport.setRequestedProjection(hadoopConf, repairedFileSchema); - promotedSchema = Option.of(schema); + promotedSchema = Option.of(schema.toAvroSchema()); } else { - AvroReadSupport.setAvroReadSchema(hadoopConf, schema); - AvroReadSupport.setRequestedProjection(hadoopConf, schema); + AvroReadSupport.setAvroReadSchema(hadoopConf, schema.toAvroSchema()); + AvroReadSupport.setRequestedProjection(hadoopConf, schema.toAvroSchema()); } ParquetReader reader = new HoodieAvroParquetReaderBuilder(path) @@ -205,7 +205,7 @@ private ClosableIterator getIndexedRecordIteratorInternal(Schema .set(ParquetInputFormat.STRICT_TYPE_CHECKING, hadoopConf.get(ParquetInputFormat.STRICT_TYPE_CHECKING)) .build(); ParquetReaderIterator parquetReaderIterator = promotedSchema.isPresent() - ? new HoodieAvroParquetReaderIterator(reader, promotedSchema.get(), renamedColumns) + ? new HoodieAvroParquetReaderIterator(reader, HoodieSchema.fromAvroSchema(promotedSchema.get()), renamedColumns) : new ParquetReaderIterator<>(reader); readerIterators.add(parquetReaderIterator); return parquetReaderIterator; diff --git a/hudi-hadoop-common/src/main/java/org/apache/parquet/avro/AvroSchemaConverterWithTimestampNTZ.java b/hudi-hadoop-common/src/main/java/org/apache/parquet/avro/AvroSchemaConverterWithTimestampNTZ.java index c1bda6ef0ffb7..4f8d88d0f6d91 100644 --- a/hudi-hadoop-common/src/main/java/org/apache/parquet/avro/AvroSchemaConverterWithTimestampNTZ.java +++ b/hudi-hadoop-common/src/main/java/org/apache/parquet/avro/AvroSchemaConverterWithTimestampNTZ.java @@ -19,8 +19,10 @@ package org.apache.parquet.avro; -import org.apache.avro.LogicalType; -import org.apache.avro.LogicalTypes; +import org.apache.hudi.common.schema.HoodieSchema; +import org.apache.hudi.common.schema.HoodieSchemaField; +import org.apache.hudi.common.schema.HoodieSchemaType; + import org.apache.avro.Schema; import org.apache.hadoop.conf.Configuration; import org.apache.parquet.schema.ConversionPatterns; @@ -116,134 +118,150 @@ public AvroSchemaConverterWithTimestampNTZ(Configuration conf) { 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."); + public MessageType convert(HoodieSchema schema) { + if (schema.getType() != HoodieSchemaType.RECORD) { + throw new IllegalArgumentException("Hoodie schema must be a record."); } - return new MessageType(avroSchema.getFullName(), convertFields(avroSchema.getFields(), "")); + return new MessageType(schema.getFullName(), convertFields(schema.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 + private List convertFields(List fields, String schemaPath) { + List types = new ArrayList(fields.size()); + for (HoodieSchemaField field : fields) { + if (field.schema().getType() == HoodieSchemaType.NULL) { + continue; // 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) { + private Type convertField(String fieldName, HoodieSchema 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) { + private Type convertField(String fieldName, HoodieSchema 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 { + HoodieSchemaType type = schema.getType(); + switch (type) { + case BOOLEAN: + builder = Types.primitive(BOOLEAN, repetition); + break; + case INT: + builder = Types.primitive(INT32, repetition); + break; + case LONG: + builder = Types.primitive(INT64, repetition); + break; + case FLOAT: + builder = Types.primitive(FLOAT, repetition); + break; + case DOUBLE: + builder = Types.primitive(DOUBLE, repetition); + break; + case BYTES: + builder = Types.primitive(BINARY, repetition); + break; + case STRING: 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"); + break; + case UUID: + if (writeParquetUUID) { + builder = Types.primitive(FIXED_LEN_BYTE_ARRAY, repetition) + .length(LogicalTypeAnnotation.UUIDLogicalTypeAnnotation.BYTES) + .as(uuidType()); + } else { + builder = Types.primitive(BINARY, repetition).as(stringType()); } - 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); + break; + case DECIMAL: + HoodieSchema.Decimal decimalSchema = (HoodieSchema.Decimal) schema; + if (decimalSchema.isFixed()) { + builder = Types.primitive(FIXED_LEN_BYTE_ARRAY, repetition) + .length(decimalSchema.getFixedSize()) + .as(decimalType(decimalSchema.getScale(), decimalSchema.getPrecision())); + } else { + builder = Types.primitive(BINARY, repetition) + .as(decimalType(decimalSchema.getScale(), decimalSchema.getPrecision())); } - } + break; + case DATE: + builder = Types.primitive(INT32, repetition).as(dateType()); + break; + case TIME: + HoodieSchema.Time timeSchema = (HoodieSchema.Time) schema; + switch (timeSchema.getPrecision()) { + case MILLIS: + builder = Types.primitive(INT32, repetition) + .as(timeType(true, MILLIS)); + break; + case MICROS: + builder = Types.primitive(INT64, repetition) + .as(timeType(true, MICROS)); + break; + default: + throw new IllegalArgumentException("Unsupported precision: " + timeSchema.getPrecision()); + } + break; + case TIMESTAMP: + HoodieSchema.Timestamp timestampSchema = (HoodieSchema.Timestamp) schema; + switch (timestampSchema.getPrecision()) { + case MILLIS: + builder = Types.primitive(INT64, repetition) + .as(timestampType(timestampSchema.isUtcAdjusted(), MILLIS)); + break; + case MICROS: + builder = Types.primitive(INT64, repetition) + .as(timestampType(timestampSchema.isUtcAdjusted(), MICROS)); + break; + default: + throw new IllegalArgumentException("Unsupported precision: " + timestampSchema.getPrecision()); + } + break; + case RECORD: + return new GroupType(repetition, fieldName, convertFields(schema.getFields(), schemaPath)); + case ENUM: + builder = Types.primitive(BINARY, repetition).as(enumType()); + break; + case 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)); + } + case MAP: + Type valType = convertField("value", schema.getValueType(), schemaPath); + // avro map key type is always string + return ConversionPatterns.stringKeyMapType(repetition, fieldName, valType); + case 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()); + } + break; + case UNION: + return convertUnion(fieldName, schema, repetition, schemaPath); + default: + throw new UnsupportedOperationException("Cannot convert Avro type " + type); } - return builder.named(fieldName); } - private Type convertUnion(String fieldName, Schema schema, Type.Repetition repetition, String schemaPath) { - List nonNullSchemas = new ArrayList(schema.getTypes().size()); + private Type convertUnion(String fieldName, HoodieSchema 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)) { + for (HoodieSchema childSchema : schema.getTypes()) { + if (childSchema.getType() == HoodieSchemaType.NULL) { foundNullSchema = true; if (Type.Repetition.REQUIRED == repetition) { repetition = Type.Repetition.OPTIONAL; @@ -267,121 +285,123 @@ private Type convertUnion(String fieldName, Schema schema, Type.Repetition repet } } - private Type convertUnionToGroupType(String fieldName, Type.Repetition repetition, List nonNullSchemas, + 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) { + for (HoodieSchema 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) { + private Type convertField(HoodieSchemaField field, String schemaPath) { return convertField(field.name(), field.schema(), schemaPath); } @Override - public Schema convert(MessageType parquetSchema) { + public HoodieSchema convert(MessageType parquetSchema) { return convertFields(parquetSchema.getName(), parquetSchema.getFields(), new HashMap<>()); } - Schema convert(GroupType parquetSchema) { + HoodieSchema convert(GroupType parquetSchema) { return convertFields(parquetSchema.getName(), parquetSchema.getFields(), new HashMap<>()); } - private Schema convertFields(String name, List parquetFields, Map names) { - List fields = new ArrayList(); + private HoodieSchema convertFields(String name, List parquetFields, Map names) { + List fields = new ArrayList<>(parquetFields.size()); Integer nameCount = names.merge(name, 1, (oldValue, value) -> oldValue + 1); for (Type parquetType : parquetFields) { - Schema fieldSchema = convertField(parquetType, names); + HoodieSchema 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( + fields.add(HoodieSchemaField.of( parquetType.getName(), optional(fieldSchema), null, NULL_VALUE)); } else { // REQUIRED - fields.add(new Schema.Field( + fields.add(HoodieSchemaField.of( parquetType.getName(), fieldSchema, null, (Object) null)); } } - Schema schema = Schema.createRecord(name, null, nameCount > 1 ? name + nameCount : null, false); - schema.setFields(fields); + HoodieSchema schema = HoodieSchema.createRecord(name, null, nameCount > 1 ? name + nameCount : null, false, fields); return schema; } - private Schema convertField(final Type parquetType, Map names) { + private HoodieSchema 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() { + + // Handle logical type annotations directly with HoodieSchema creation methods + if (annotation != null) { + HoodieSchema logicalSchema = convertLogicalTypeAnnotationToHoodieSchema(annotation, parquetType); + if (logicalSchema != null) { + return logicalSchema; + } + } + + // Fallback to basic type conversion if no logical type annotation + HoodieSchema schema = parquetPrimitiveTypeName.convert( + new PrimitiveType.PrimitiveTypeNameConverter() { @Override - public Schema convertBOOLEAN(PrimitiveTypeName primitiveTypeName) { - return Schema.create(Schema.Type.BOOLEAN); + public HoodieSchema convertBOOLEAN(PrimitiveTypeName primitiveTypeName) { + return HoodieSchema.create(HoodieSchemaType.BOOLEAN); } @Override - public Schema convertINT32(PrimitiveTypeName primitiveTypeName) { - return Schema.create(Schema.Type.INT); + public HoodieSchema convertINT32(PrimitiveTypeName primitiveTypeName) { + return HoodieSchema.create(HoodieSchemaType.INT); } @Override - public Schema convertINT64(PrimitiveTypeName primitiveTypeName) { - return Schema.create(Schema.Type.LONG); + public HoodieSchema convertINT64(PrimitiveTypeName primitiveTypeName) { + return HoodieSchema.create(HoodieSchemaType.LONG); } @Override - public Schema convertINT96(PrimitiveTypeName primitiveTypeName) { + public HoodieSchema convertINT96(PrimitiveTypeName primitiveTypeName) { if (readInt96AsFixed) { - return Schema.createFixed("INT96", "INT96 represented as byte[12]", null, 12); + return HoodieSchema.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); + public HoodieSchema convertFLOAT(PrimitiveTypeName primitiveTypeName) { + return HoodieSchema.create(HoodieSchemaType.FLOAT); } @Override - public Schema convertDOUBLE(PrimitiveTypeName primitiveTypeName) { - return Schema.create(Schema.Type.DOUBLE); + public HoodieSchema convertDOUBLE(PrimitiveTypeName primitiveTypeName) { + return HoodieSchema.create(HoodieSchemaType.DOUBLE); } @Override - public Schema convertFIXED_LEN_BYTE_ARRAY(PrimitiveTypeName primitiveTypeName) { + public HoodieSchema convertFIXED_LEN_BYTE_ARRAY(PrimitiveTypeName primitiveTypeName) { if (annotation instanceof LogicalTypeAnnotation.UUIDLogicalTypeAnnotation) { - return Schema.create(Schema.Type.STRING); + return HoodieSchema.createUUID(); } else { int size = parquetType.asPrimitiveType().getTypeLength(); - return Schema.createFixed(parquetType.getName(), null, null, size); + return HoodieSchema.createFixed(parquetType.getName(), null, null, size); } } @Override - public Schema convertBINARY(PrimitiveTypeName primitiveTypeName) { + public HoodieSchema convertBINARY(PrimitiveTypeName primitiveTypeName) { if (annotation instanceof LogicalTypeAnnotation.StringLogicalTypeAnnotation || annotation instanceof LogicalTypeAnnotation.EnumLogicalTypeAnnotation) { - return Schema.create(Schema.Type.STRING); + return HoodieSchema.create(HoodieSchemaType.STRING); } else { - return Schema.create(Schema.Type.BYTES); + return HoodieSchema.create(HoodieSchemaType.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() { + return logicalTypeAnnotation.accept(new LogicalTypeAnnotation.LogicalTypeAnnotationVisitor() { @Override - public java.util.Optional visit(LogicalTypeAnnotation.ListLogicalTypeAnnotation listLogicalType) { + public java.util.Optional visit(LogicalTypeAnnotation.ListLogicalTypeAnnotation listLogicalType) { if (parquetGroupType.getFieldCount()!= 1) { throw new UnsupportedOperationException("Invalid list type " + parquetGroupType); } @@ -391,29 +411,29 @@ public java.util.Optional visit(LogicalTypeAnnotation.ListLogicalTypeAnn } if (isElementType(repeatedType, parquetGroupType.getName())) { // repeated element types are always required - return java.util.Optional.of(Schema.createArray(convertField(repeatedType, names))); + return java.util.Optional.of(HoodieSchema.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)))); + return java.util.Optional.of(HoodieSchema.createArray(optional(convertField(elementType, names)))); } else { - return java.util.Optional.of(Schema.createArray(convertField(elementType, names))); + return java.util.Optional.of(HoodieSchema.createArray(convertField(elementType, names))); } } } @Override // for backward-compatibility - public java.util.Optional visit(LogicalTypeAnnotation.MapKeyValueTypeAnnotation mapKeyValueLogicalType) { + public java.util.Optional visit(LogicalTypeAnnotation.MapKeyValueTypeAnnotation mapKeyValueLogicalType) { return visitMapOrMapKeyValue(); } @Override - public java.util.Optional visit(LogicalTypeAnnotation.MapLogicalTypeAnnotation mapLogicalType) { + public java.util.Optional visit(LogicalTypeAnnotation.MapLogicalTypeAnnotation mapLogicalType) { return visitMapOrMapKeyValue(); } - private java.util.Optional visitMapOrMapKeyValue() { + private java.util.Optional visitMapOrMapKeyValue() { if (parquetGroupType.getFieldCount() != 1 || parquetGroupType.getType(0).isPrimitive()) { throw new UnsupportedOperationException("Invalid map type " + parquetGroupType); } @@ -431,15 +451,15 @@ private java.util.Optional visitMapOrMapKeyValue() { } Type valueType = mapKeyValType.getType(1); if (valueType.isRepetition(Type.Repetition.OPTIONAL)) { - return java.util.Optional.of(Schema.createMap(optional(convertField(valueType, names)))); + return java.util.Optional.of(HoodieSchema.createMap(optional(convertField(valueType, names)))); } else { - return java.util.Optional.of(Schema.createMap(convertField(valueType, names))); + return java.util.Optional.of(HoodieSchema.createMap(convertField(valueType, names))); } } @Override - public java.util.Optional visit(LogicalTypeAnnotation.EnumLogicalTypeAnnotation enumLogicalType) { - return java.util.Optional.of(Schema.create(Schema.Type.STRING)); + public java.util.Optional visit(LogicalTypeAnnotation.EnumLogicalTypeAnnotation enumLogicalType) { + return java.util.Optional.of(HoodieSchema.create(HoodieSchemaType.STRING)); } }).orElseThrow(() -> new UnsupportedOperationException("Cannot convert Parquet type " + parquetType)); } else { @@ -449,100 +469,139 @@ public java.util.Optional visit(LogicalTypeAnnotation.EnumLogicalTypeAnn } } - private LogicalTypeAnnotation convertLogicalType(LogicalType logicalType) { - if (logicalType == null) { + /** + * Converts Parquet LogicalTypeAnnotation directly to HoodieSchema using HoodieSchema's factory methods. + * This replaces the need to convert through Avro's LogicalType intermediate representation. + * Also validates that logical types are only applied to compatible primitive types. + * + * @param annotation Parquet logical type annotation + * @param parquetType The parquet type containing the annotation + * @return HoodieSchema with the logical type applied, or null if no logical type conversion is needed + * @throws IllegalArgumentException if the logical type is not compatible with the primitive type + */ + private HoodieSchema convertLogicalTypeAnnotationToHoodieSchema(LogicalTypeAnnotation annotation, Type parquetType) { + if (annotation == 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); + final PrimitiveTypeName primitiveType = parquetType.asPrimitiveType().getPrimitiveTypeName(); + + return annotation.accept(new LogicalTypeAnnotation.LogicalTypeAnnotationVisitor() { + @Override + public java.util.Optional visit(LogicalTypeAnnotation.DecimalLogicalTypeAnnotation decimalLogicalType) { + // DECIMAL can be BINARY or FIXED_LEN_BYTE_ARRAY + if (primitiveType == FIXED_LEN_BYTE_ARRAY) { + int fixedSize = parquetType.asPrimitiveType().getTypeLength(); + String name = parquetType.getName(); + return java.util.Optional.of(HoodieSchema.createDecimal(name, null, null, + decimalLogicalType.getPrecision(), decimalLogicalType.getScale(), fixedSize)); + } else if (primitiveType == BINARY) { + return java.util.Optional.of(HoodieSchema.createDecimal(decimalLogicalType.getPrecision(), decimalLogicalType.getScale())); + } else { + return java.util.Optional.empty(); + } + } + + @Override + public java.util.Optional visit(LogicalTypeAnnotation.DateLogicalTypeAnnotation dateLogicalType) { + // DATE must be INT32 + if (primitiveType != INT32) { + throw new IllegalArgumentException("DATE can only annotate INT32, found " + primitiveType); + } + return java.util.Optional.of(HoodieSchema.createDate()); } - } - return null; - } + @Override + public java.util.Optional visit(LogicalTypeAnnotation.TimeLogicalTypeAnnotation timeLogicalType) { + LogicalTypeAnnotation.TimeUnit unit = timeLogicalType.getUnit(); + switch (unit) { + case MILLIS: + // TIME_MILLIS must be INT32 + if (primitiveType != INT32) { + throw new IllegalArgumentException("TIME(MILLIS) can only annotate INT32, found " + primitiveType); + } + return java.util.Optional.of(HoodieSchema.createTimeMillis()); + case MICROS: + // TIME_MICROS must be INT64 + if (primitiveType != INT64) { + throw new IllegalArgumentException("TIME(MICROS) can only annotate INT64, found " + primitiveType); + } + return java.util.Optional.of(HoodieSchema.createTimeMicros()); + case NANOS: + // Avro doesn't support nanosecond precision for time + return java.util.Optional.empty(); + default: + return java.util.Optional.empty(); + } + } - 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.TimestampLogicalTypeAnnotation timestampLogicalType) { + LogicalTypeAnnotation.TimeUnit unit = timestampLogicalType.getUnit(); + boolean isAdjustedToUTC = timestampLogicalType.isAdjustedToUTC(); - @Override - public java.util.Optional visit( - LogicalTypeAnnotation.DateLogicalTypeAnnotation dateLogicalType) { - return java.util.Optional.of(LogicalTypes.date()); - } + // TIMESTAMP must be INT64 + if (primitiveType != INT64) { + throw new IllegalArgumentException("TIMESTAMP can only annotate INT64, found " + primitiveType); + } - @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(); + if (isAdjustedToUTC || !avroVersionSupportsLocalTimestampTypes()) { + switch (unit) { + case MILLIS: + return java.util.Optional.of(HoodieSchema.createTimestampMillis()); + case MICROS: + return java.util.Optional.of(HoodieSchema.createTimestampMicros()); + case NANOS: + // Avro doesn't support nanosecond precision for timestamp + return java.util.Optional.empty(); + default: + 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()); - } + } else { + switch (unit) { + case MILLIS: + return java.util.Optional.of(HoodieSchema.createLocalTimestampMillis()); + case MICROS: + return java.util.Optional.of(HoodieSchema.createLocalTimestampMicros()); + case NANOS: + // Avro doesn't support nanosecond precision for timestamp 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()); - } + default: return java.util.Optional.empty(); - } } + } + } - @Override - public java.util.Optional visit(UUIDLogicalTypeAnnotation uuidLogicalType) { - return java.util.Optional.of(LogicalTypes.uuid()); - } - }) - .orElse(null); + @Override + public java.util.Optional visit(UUIDLogicalTypeAnnotation uuidLogicalType) { + // UUID must be FIXED_LEN_BYTE_ARRAY with length 16 + if (primitiveType != FIXED_LEN_BYTE_ARRAY) { + return java.util.Optional.empty(); + } + return java.util.Optional.of(HoodieSchema.createUUID()); + } + + @Override + public java.util.Optional visit(LogicalTypeAnnotation.StringLogicalTypeAnnotation stringLogicalType) { + // STRING must be BINARY + if (primitiveType != BINARY) { + return java.util.Optional.empty(); + } + return java.util.Optional.of(HoodieSchema.create(HoodieSchemaType.STRING)); + } + + @Override + public java.util.Optional visit(LogicalTypeAnnotation.EnumLogicalTypeAnnotation enumLogicalType) { + // ENUM must be BINARY + if (primitiveType != BINARY) { + return java.util.Optional.empty(); + } + return java.util.Optional.of(HoodieSchema.create(HoodieSchemaType.STRING)); + } + + // Return empty for other logical types that don't have direct HoodieSchema equivalents + // They will fall back to the basic type conversion + }).orElse(null); } /** @@ -568,10 +627,10 @@ private boolean isElementType(Type repeatedType, String parentName) { ); } - private static Schema optional(Schema original) { + private static HoodieSchema optional(HoodieSchema original) { // null is first in the union because Parquet's default is always null - return Schema.createUnion(Arrays.asList( - Schema.create(Schema.Type.NULL), + return HoodieSchema.createUnion(Arrays.asList( + HoodieSchema.create(HoodieSchemaType.NULL), original)); } diff --git a/hudi-hadoop-common/src/main/java/org/apache/parquet/avro/HoodieAvroParquetSchemaConverter.java b/hudi-hadoop-common/src/main/java/org/apache/parquet/avro/HoodieAvroParquetSchemaConverter.java index a06b9bb0ef40d..b664cfbf66a59 100644 --- a/hudi-hadoop-common/src/main/java/org/apache/parquet/avro/HoodieAvroParquetSchemaConverter.java +++ b/hudi-hadoop-common/src/main/java/org/apache/parquet/avro/HoodieAvroParquetSchemaConverter.java @@ -19,10 +19,10 @@ package org.apache.parquet.avro; +import org.apache.hudi.common.schema.HoodieSchema; import org.apache.hudi.common.util.ReflectionUtils; import lombok.extern.slf4j.Slf4j; -import org.apache.avro.Schema; import org.apache.hadoop.conf.Configuration; import org.apache.parquet.schema.MessageType; @@ -50,7 +50,7 @@ public static HoodieAvroParquetSchemaConverter getAvroSchemaConverter(Configurat } } - public abstract MessageType convert(Schema schema); + public abstract MessageType convert(HoodieSchema schema); - public abstract Schema convert(MessageType schema); + public abstract HoodieSchema convert(MessageType schema); } diff --git a/hudi-hadoop-common/src/main/java/org/apache/parquet/avro/HoodieAvroReadSupport.java b/hudi-hadoop-common/src/main/java/org/apache/parquet/avro/HoodieAvroReadSupport.java index d06cfae1ede37..9292baf65dbfb 100644 --- a/hudi-hadoop-common/src/main/java/org/apache/parquet/avro/HoodieAvroReadSupport.java +++ b/hudi-hadoop-common/src/main/java/org/apache/parquet/avro/HoodieAvroReadSupport.java @@ -18,9 +18,9 @@ package org.apache.parquet.avro; +import org.apache.hudi.common.schema.HoodieSchema; import org.apache.hudi.common.util.Option; -import org.apache.avro.Schema; import org.apache.avro.generic.GenericData; import org.apache.hadoop.conf.Configuration; import org.apache.parquet.conf.ParquetConfiguration; @@ -86,10 +86,10 @@ public ReadContext init(ParquetConfiguration configuration, Map String requestedProjectionString = configuration.get(AVRO_REQUESTED_PROJECTION); if (requestedProjectionString != null) { - Schema avroRequestedProjection = new Schema.Parser().parse(requestedProjectionString); + HoodieSchema avroRequestedProjection = HoodieSchema.parse(requestedProjectionString); Configuration conf = new Configuration(); configuration.forEach(entry -> conf.set(entry.getKey(), entry.getValue())); - projection = new AvroSchemaConverter(conf).convert(avroRequestedProjection); + projection = new AvroSchemaConverter(conf).convert(avroRequestedProjection.toAvroSchema()); } String avroReadSchema = configuration.get("parquet.avro.read.schema"); diff --git a/hudi-hadoop-common/src/main/java/org/apache/parquet/avro/NativeAvroSchemaConverter.java b/hudi-hadoop-common/src/main/java/org/apache/parquet/avro/NativeAvroSchemaConverter.java index 509a8afb3ea83..abf76dddc0e9d 100644 --- a/hudi-hadoop-common/src/main/java/org/apache/parquet/avro/NativeAvroSchemaConverter.java +++ b/hudi-hadoop-common/src/main/java/org/apache/parquet/avro/NativeAvroSchemaConverter.java @@ -19,7 +19,8 @@ package org.apache.parquet.avro; -import org.apache.avro.Schema; +import org.apache.hudi.common.schema.HoodieSchema; + import org.apache.hadoop.conf.Configuration; import org.apache.parquet.schema.MessageType; @@ -35,12 +36,12 @@ public NativeAvroSchemaConverter(Configuration configuration) { } @Override - public MessageType convert(Schema schema) { - return avroSchemaConverter.convert(schema); + public MessageType convert(HoodieSchema schema) { + return avroSchemaConverter.convert(schema.toAvroSchema()); } @Override - public Schema convert(MessageType schema) { - return avroSchemaConverter.convert(schema); + public HoodieSchema convert(MessageType schema) { + return HoodieSchema.fromAvroSchema(avroSchemaConverter.convert(schema)); } } diff --git a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/TestTableSchemaResolver.java b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/TestTableSchemaResolver.java index 5a572f315a9c1..956b116ecd8d6 100644 --- a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/TestTableSchemaResolver.java +++ b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/TestTableSchemaResolver.java @@ -202,7 +202,7 @@ void testHasOperationFieldFileInspectionOrdering() throws IOException { FileFormatUtils fileFormatUtils = mock(FileFormatUtils.class); StoragePath parquetPath = new StoragePath("/tmp/hudi_table/partition1/baseFile1.parquet"); when(ioFactory.getFileFormatUtils(parquetPath)).thenReturn(fileFormatUtils); - when(fileFormatUtils.readAvroSchema(any(), eq(parquetPath))).thenReturn(null); + when(fileFormatUtils.readSchema(any(), eq(parquetPath))).thenReturn(null); ioFactoryMockedStatic.when(() -> HoodieIOFactory.getIOFactory(any())).thenReturn(ioFactory); // mock log file schema reading to return the expected schema tableSchemaResolverMockedStatic.when(() -> TableSchemaResolver.readSchemaFromLogFile(any(), eq(new StoragePath("/tmp/hudi_table/" + logFileWriteStat.getPath())))) diff --git a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/util/TestAvroOrcUtils.java b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/util/TestAvroOrcUtils.java index de7968b3ce010..f2d4db28809d2 100644 --- a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/util/TestAvroOrcUtils.java +++ b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/util/TestAvroOrcUtils.java @@ -18,10 +18,11 @@ package org.apache.hudi.common.util; +import org.apache.hudi.common.schema.HoodieSchema; import org.apache.hudi.common.testutils.HoodieCommonTestHarness; -import org.apache.avro.Schema; import org.apache.orc.TypeDescription; +import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.MethodSource; @@ -29,7 +30,7 @@ import java.util.Arrays; import java.util.List; -import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.AVRO_SCHEMA; +import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.HOODIE_SCHEMA; import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA; import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.TRIP_SCHEMA; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -38,8 +39,9 @@ * Tests {@link AvroOrcUtils}. */ public class TestAvroOrcUtils extends HoodieCommonTestHarness { - public static final TypeDescription ORC_SCHEMA = AvroOrcUtils.createOrcSchema(new Schema.Parser().parse(TRIP_EXAMPLE_SCHEMA)); - public static final TypeDescription ORC_TRIP_SCHEMA = AvroOrcUtils.createOrcSchema(new Schema.Parser().parse(TRIP_SCHEMA)); + + public static final TypeDescription ORC_SCHEMA = AvroOrcUtils.createOrcSchema(HoodieSchema.parse(TRIP_EXAMPLE_SCHEMA)); + public static final TypeDescription ORC_TRIP_SCHEMA = AvroOrcUtils.createOrcSchema(HoodieSchema.parse(TRIP_SCHEMA)); public static List testCreateOrcSchemaArgs() { // the ORC schema is constructed in the order as AVRO_SCHEMA: @@ -60,7 +62,7 @@ public static List testCreateOrcSchemaArgs() { // Tests the types FIXED, UNION String structField = "{\"type\":\"record\", \"name\":\"fare\",\"fields\": " + "[{\"name\": \"amount\",\"type\": \"double\"},{\"name\": \"currency\", \"type\": \"string\"}]}"; - Schema avroSchemaWithMoreTypes = new Schema.Parser().parse( + HoodieSchema schemaWithMoreTypes = HoodieSchema.parse( "{\"type\": \"record\"," + "\"name\": \"triprec\"," + "\"fields\": [ " + "{\"name\" : \"age\", \"type\":{\"type\": \"fixed\", \"size\": 16, \"name\": \"fixedField\" }}," + "{\"name\" : \"height\", \"type\": [\"int\", \"null\"] }," @@ -70,15 +72,45 @@ public static List testCreateOrcSchemaArgs() { "struct,fare:struct>"); return Arrays.asList( - Arguments.of(AVRO_SCHEMA, orcSchema), - Arguments.of(avroSchemaWithMoreTypes, orcSchemaWithMoreTypes) + Arguments.of(HOODIE_SCHEMA, orcSchema), + Arguments.of(schemaWithMoreTypes, orcSchemaWithMoreTypes) ); } @ParameterizedTest @MethodSource("testCreateOrcSchemaArgs") - public void testCreateOrcSchema(Schema avroSchema, TypeDescription orcSchema) { + public void testCreateOrcSchema(HoodieSchema avroSchema, TypeDescription orcSchema) { TypeDescription convertedSchema = AvroOrcUtils.createOrcSchema(avroSchema); assertEquals(orcSchema, convertedSchema); } + + /** + * Tests that LocalTimestamp types are converted to ORC Long (not Timestamp) to preserve old behavior. + * This ensures backward compatibility with the pre-HoodieSchema refactoring implementation + * where LocalTimestamp logical types were not explicitly handled and fell through to LONG conversion. + */ + @Test + public void testLocalTimestampConvertedToLong() { + // Create HoodieSchemas for all timestamp types + HoodieSchema timestampMillis = HoodieSchema.createTimestampMillis(); + HoodieSchema timestampMicros = HoodieSchema.createTimestampMicros(); + HoodieSchema localTimestampMillis = HoodieSchema.createLocalTimestampMillis(); + HoodieSchema localTimestampMicros = HoodieSchema.createLocalTimestampMicros(); + + // UTC-adjusted timestamps should convert to ORC Timestamp + TypeDescription orcTimestampMillis = AvroOrcUtils.createOrcSchema(timestampMillis); + TypeDescription orcTimestampMicros = AvroOrcUtils.createOrcSchema(timestampMicros); + assertEquals(TypeDescription.Category.TIMESTAMP, orcTimestampMillis.getCategory(), + "TimestampMillis should convert to ORC Timestamp"); + assertEquals(TypeDescription.Category.TIMESTAMP, orcTimestampMicros.getCategory(), + "TimestampMicros should convert to ORC Timestamp"); + + // Local timestamps should convert to ORC Long (old behavior) + TypeDescription orcLocalTimestampMillis = AvroOrcUtils.createOrcSchema(localTimestampMillis); + TypeDescription orcLocalTimestampMicros = AvroOrcUtils.createOrcSchema(localTimestampMicros); + assertEquals(TypeDescription.Category.LONG, orcLocalTimestampMillis.getCategory(), + "LocalTimestampMillis should convert to ORC Long (preserving old behavior)"); + assertEquals(TypeDescription.Category.LONG, orcLocalTimestampMicros.getCategory(), + "LocalTimestampMicros should convert to ORC Long (preserving old behavior)"); + } } diff --git a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/util/TestParquetUtils.java b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/util/TestParquetUtils.java index ac1e91944de9b..ab64c1c1992a3 100644 --- a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/util/TestParquetUtils.java +++ b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/util/TestParquetUtils.java @@ -18,7 +18,6 @@ package org.apache.hudi.common.util; -import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.avro.HoodieAvroWriteSupport; import org.apache.hudi.common.bloom.BloomFilter; import org.apache.hudi.common.bloom.BloomFilterFactory; @@ -29,6 +28,7 @@ import org.apache.hudi.common.schema.HoodieSchema; import org.apache.hudi.common.schema.HoodieSchemaField; import org.apache.hudi.common.schema.HoodieSchemaType; +import org.apache.hudi.common.schema.HoodieSchemaUtils; import org.apache.hudi.common.testutils.HoodieCommonTestHarness; import org.apache.hudi.common.testutils.HoodieTestUtils; import org.apache.hudi.common.util.collection.ClosableIterator; @@ -37,8 +37,6 @@ import org.apache.hudi.stats.HoodieColumnRangeMetadata; import org.apache.hudi.storage.StoragePath; -import org.apache.avro.JsonProperties; -import org.apache.avro.Schema; import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericRecord; import org.apache.hadoop.fs.Path; @@ -66,7 +64,7 @@ import java.util.UUID; import java.util.stream.Collectors; -import static org.apache.hudi.avro.HoodieAvroUtils.METADATA_FIELD_SCHEMA; +import static org.apache.hudi.common.schema.HoodieSchemaUtils.METADATA_FIELD_SCHEMA; import static org.apache.hudi.metadata.HoodieIndexVersion.V1; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -157,7 +155,7 @@ public void testFetchRecordKeyPartitionPathFromParquet(String typeCode) throws E } String filePath = Paths.get(basePath, "test.parquet").toUri().toString(); - Schema schema = HoodieAvroUtils.getRecordKeyPartitionPathSchema(); + HoodieSchema schema = HoodieSchemaUtils.getRecordKeyPartitionPathSchema(); writeParquetFile(typeCode, filePath, rowKeys, schema, true, partitionPath); // Read and verify @@ -187,7 +185,7 @@ public void testFetchRecordKeyPartitionPathVirtualKeysFromParquet() throws Excep } String filePath = Paths.get(basePath, "test.parquet").toUri().toString(); - Schema schema = getSchemaWithFields(Arrays.asList(new String[] {"abc", "def"})); + HoodieSchema schema = getSchemaWithFields(Arrays.asList(new String[] {"abc", "def"})); writeParquetFile(BloomFilterTypeCode.SIMPLE.name(), filePath, rowKeys, schema, true, partitionPath, false, "abc", "def"); @@ -256,7 +254,7 @@ public void testReadColumnStatsFromMetadata() throws Exception { BloomFilter filter = BloomFilterFactory .createBloomFilter(1000, 0.0001, 10000, BloomFilterTypeCode.SIMPLE.name()); HoodieAvroWriteSupport writeSupport = - new HoodieAvroWriteSupport(new AvroSchemaConverter().convert(schema.toAvroSchema()), schema.toAvroSchema(), Option.of(filter), new Properties()); + new HoodieAvroWriteSupport(new AvroSchemaConverter().convert(schema.toAvroSchema()), schema, Option.of(filter), new Properties()); try (ParquetWriter writer = new ParquetWriter(new Path(filePath), writeSupport, CompressionCodecName.GZIP, 120 * 1024 * 1024, ParquetWriter.DEFAULT_PAGE_SIZE)) { valueList.forEach(entry -> { @@ -297,7 +295,7 @@ public void testReadColumnStatsFromMetadata() throws Exception { } private HoodieSchema getSchema(String recordKeyField, String partitionPathField, String dataField) { - List toBeAddedFields = new ArrayList<>(); + List toBeAddedFields = new ArrayList<>(3); HoodieSchemaField recordKeySchemaField = HoodieSchemaField.of(recordKeyField, HoodieSchema.createNullable(HoodieSchemaType.STRING), "", HoodieSchema.NULL_VALUE); @@ -309,8 +307,7 @@ private HoodieSchema getSchema(String recordKeyField, String partitionPathField, toBeAddedFields.add(recordKeySchemaField); toBeAddedFields.add(partitionPathSchemaField); toBeAddedFields.add(dataSchemaField); - HoodieSchema recordSchema = HoodieSchema.createRecord("HoodieRecord", "", "", false, toBeAddedFields); - return recordSchema; + return HoodieSchema.createRecord("HoodieRecord", "", "", false, toBeAddedFields); } private void validateColumnRangeMetadata(HoodieColumnRangeMetadata metadata, @@ -329,25 +326,25 @@ private void validateColumnRangeMetadata(HoodieColumnRangeMetadata metadata, } private void writeParquetFile(String typeCode, String filePath, List rowKeys) throws Exception { - writeParquetFile(typeCode, filePath, rowKeys, HoodieAvroUtils.getRecordKeySchema(), false, ""); + writeParquetFile(typeCode, filePath, rowKeys, HoodieSchemaUtils.getRecordKeySchema(), false, ""); } - private void writeParquetFile(String typeCode, String filePath, List rowKeys, Schema schema, boolean addPartitionPathField, String partitionPath) throws Exception { + private void writeParquetFile(String typeCode, String filePath, List rowKeys, HoodieSchema schema, boolean addPartitionPathField, String partitionPath) throws Exception { writeParquetFile(typeCode, filePath, rowKeys, schema, addPartitionPathField, partitionPath, true, null, null); } - private void writeParquetFile(String typeCode, String filePath, List rowKeys, Schema schema, boolean addPartitionPathField, String partitionPathValue, + private void writeParquetFile(String typeCode, String filePath, List rowKeys, HoodieSchema schema, boolean addPartitionPathField, String partitionPathValue, boolean useMetaFields, String recordFieldName, String partitionFieldName) throws Exception { // Write out a parquet file BloomFilter filter = BloomFilterFactory .createBloomFilter(1000, 0.0001, 10000, typeCode); HoodieAvroWriteSupport writeSupport = - new HoodieAvroWriteSupport(new AvroSchemaConverter().convert(schema), schema, Option.of(filter), new Properties()); + new HoodieAvroWriteSupport(new AvroSchemaConverter().convert(schema.toAvroSchema()), schema, Option.of(filter), new Properties()); ParquetWriter writer = new ParquetWriter(new Path(filePath), writeSupport, CompressionCodecName.GZIP, 120 * 1024 * 1024, ParquetWriter.DEFAULT_PAGE_SIZE); for (String rowKey : rowKeys) { - GenericRecord rec = new GenericData.Record(schema); + GenericRecord rec = new GenericData.Record(schema.toAvroSchema()); rec.put(useMetaFields ? HoodieRecord.RECORD_KEY_METADATA_FIELD : recordFieldName, rowKey); if (addPartitionPathField) { rec.put(useMetaFields ? HoodieRecord.PARTITION_PATH_METADATA_FIELD : partitionFieldName, partitionPathValue); @@ -358,17 +355,14 @@ private void writeParquetFile(String typeCode, String filePath, List row writer.close(); } - private static Schema getSchemaWithFields(List fields) { - List toBeAddedFields = new ArrayList<>(); - Schema recordSchema = Schema.createRecord("HoodieRecordKey", "", "", false); - + private static HoodieSchema getSchemaWithFields(List fields) { + List toBeAddedFields = new ArrayList<>(fields.size()); for (String field: fields) { - Schema.Field schemaField = - new Schema.Field(field, METADATA_FIELD_SCHEMA, "", JsonProperties.NULL_VALUE); + HoodieSchemaField schemaField = + HoodieSchemaField.of(field, METADATA_FIELD_SCHEMA, "", HoodieSchema.NULL_VALUE); toBeAddedFields.add(schemaField); } - recordSchema.setFields(toBeAddedFields); - return recordSchema; + return HoodieSchema.createRecord("HoodieRecordKey", "", "", false, toBeAddedFields); } class TestBaseKeyGen extends BaseKeyGenerator { @@ -404,7 +398,7 @@ public List getPartitionPathFields() { } @Test - public void testReadSchemaHash() throws Exception { + public void testReadMessageTypeHash() throws Exception { // Given: Create a parquet file with a specific schema List rowKeys = Arrays.asList("row1", "row2", "row3"); String filePath = Paths.get(basePath, "test_schema_hash.parquet").toUri().toString(); @@ -425,7 +419,7 @@ public void testReadSchemaHash() throws Exception { } @Test - public void testReadSchemaHash_DifferentSchemas() throws Exception { + public void testReadMessageTypeHash_DifferentSchemas() throws Exception { // Given: Create two parquet files with different schemas List rowKeys = Arrays.asList("row1", "row2"); @@ -447,7 +441,7 @@ public void testReadSchemaHash_DifferentSchemas() throws Exception { } @Test - public void testReadSchemaHash_NonExistentFile() throws Exception { + public void testReadMessageTypeHash_NonExistentFile() throws Exception { // Given: Non-existent file path StoragePath nonExistentPath = new StoragePath("/non/existent/file.parquet"); @@ -469,7 +463,7 @@ public void testReadSchemaHash_MatchesDirectSchemaRead() throws Exception { // When: Reading schema hash vs direct schema read Integer schemaHashFromUtils = ParquetUtils.readSchemaHash(HoodieTestUtils.getStorage(filePath), storagePath); - MessageType directSchema = parquetUtils.readSchema(HoodieTestUtils.getStorage(filePath), storagePath); + MessageType directSchema = parquetUtils.readMessageType(HoodieTestUtils.getStorage(filePath), storagePath); Integer directSchemaHash = directSchema.hashCode(); // Then: Hash from utility method should match direct schema hash @@ -479,7 +473,7 @@ public void testReadSchemaHash_MatchesDirectSchemaRead() throws Exception { private void writeParquetFileWithExtendedSchema(String filePath, List rowKeys) throws Exception { // Create an extended schema with an additional field - List fields = new ArrayList<>(); + List fields = new ArrayList<>(4); fields.add(HoodieSchemaField.of("_row_key", HoodieSchema.create(HoodieSchemaType.STRING), "", (Object) null)); fields.add(HoodieSchemaField.of("time", HoodieSchema.create(HoodieSchemaType.LONG), "", (Object) null)); fields.add(HoodieSchemaField.of("number", HoodieSchema.create(HoodieSchemaType.LONG), "", (Object) null)); @@ -489,7 +483,7 @@ private void writeParquetFileWithExtendedSchema(String filePath, List ro BloomFilter filter = BloomFilterFactory.createBloomFilter(1000, 0.0001, -1, BloomFilterTypeCode.SIMPLE.name()); HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport( - new AvroSchemaConverter().convert(extendedSchema.toAvroSchema()), extendedSchema.toAvroSchema(), Option.of(filter), new Properties()); + new AvroSchemaConverter().convert(extendedSchema.toAvroSchema()), extendedSchema, Option.of(filter), new Properties()); ParquetWriter writer = new ParquetWriter(new Path(filePath), writeSupport, CompressionCodecName.GZIP, 120 * 1024 * 1024, ParquetWriter.DEFAULT_PAGE_SIZE); diff --git a/hudi-hadoop-common/src/test/java/org/apache/hudi/io/hadoop/TestHoodieBaseParquetWriter.java b/hudi-hadoop-common/src/test/java/org/apache/hudi/io/hadoop/TestHoodieBaseParquetWriter.java index fdc994705d71d..e4e86abef939f 100644 --- a/hudi-hadoop-common/src/test/java/org/apache/hudi/io/hadoop/TestHoodieBaseParquetWriter.java +++ b/hudi-hadoop-common/src/test/java/org/apache/hudi/io/hadoop/TestHoodieBaseParquetWriter.java @@ -24,6 +24,7 @@ import org.apache.hudi.common.bloom.BloomFilterFactory; import org.apache.hudi.common.bloom.BloomFilterTypeCode; import org.apache.hudi.common.config.HoodieParquetConfig; +import org.apache.hudi.common.schema.HoodieSchema; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.testutils.HoodieTestUtils; import org.apache.hudi.common.util.Option; @@ -32,7 +33,6 @@ import lombok.Getter; import lombok.Setter; -import org.apache.avro.Schema; import org.apache.avro.generic.IndexedRecord; import org.apache.parquet.avro.AvroSchemaConverter; import org.apache.parquet.hadoop.ParquetWriter; @@ -79,8 +79,8 @@ public void testCanWrite() throws IOException { BloomFilterTypeCode.DYNAMIC_V0.name()); StorageConfiguration conf = HoodieTestUtils.getDefaultStorageConfWithDefaults(); - Schema schema = new Schema.Parser().parse(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA); - HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(new AvroSchemaConverter().convert(schema), + HoodieSchema schema = HoodieTestDataGenerator.HOODIE_SCHEMA; + HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(new AvroSchemaConverter().convert(schema.toAvroSchema()), schema, Option.of(filter), new Properties()); long maxFileSize = 2 * 1024 * 1024; diff --git a/hudi-hadoop-common/src/test/java/org/apache/hudi/io/hadoop/TestHoodieHFileReaderWriter.java b/hudi-hadoop-common/src/test/java/org/apache/hudi/io/hadoop/TestHoodieHFileReaderWriter.java index 4decc40e27704..12056d7a8b5a9 100644 --- a/hudi-hadoop-common/src/test/java/org/apache/hudi/io/hadoop/TestHoodieHFileReaderWriter.java +++ b/hudi-hadoop-common/src/test/java/org/apache/hudi/io/hadoop/TestHoodieHFileReaderWriter.java @@ -47,7 +47,6 @@ import org.apache.hudi.storage.HoodieStorage; import org.apache.hudi.storage.StoragePath; -import org.apache.avro.Schema; import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.IndexedRecord; @@ -157,7 +156,7 @@ protected void verifyHFileReader(byte[] content, @Override protected HoodieAvroHFileWriter createWriter( - Schema avroSchema, boolean populateMetaFields) throws Exception { + HoodieSchema schema, boolean populateMetaFields) throws Exception { String instantTime = "000"; HoodieStorage storage = HoodieTestUtils.getStorage(getFilePath()); Properties props = new Properties(); @@ -168,7 +167,7 @@ protected HoodieAvroHFileWriter createWriter( when(partitionSupplier.get()).thenReturn(10); return (HoodieAvroHFileWriter) HoodieFileWriterFactory.getFileWriter( - instantTime, getFilePath(), storage, HoodieStorageConfig.newBuilder().fromProperties(props).build(), HoodieSchema.fromAvroSchema(avroSchema), + instantTime, getFilePath(), storage, HoodieStorageConfig.newBuilder().fromProperties(props).build(), schema, mockTaskContextSupplier, HoodieRecord.HoodieRecordType.AVRO); } @@ -201,7 +200,7 @@ protected void verifySchema(HoodieStorage storage, String schemaPath) throws IOE @MethodSource("populateMetaFieldsAndTestAvroWithMeta") public void testWriteReadHFileWithMetaFields(boolean populateMetaFields, boolean testAvroWithMeta) throws Exception { HoodieSchema schema = getSchemaFromResource(TestHoodieOrcReaderWriter.class, "/exampleSchemaWithMetaFields.avsc"); - HoodieAvroHFileWriter writer = createWriter(schema.toAvroSchema(), populateMetaFields); + HoodieAvroHFileWriter writer = createWriter(schema, populateMetaFields); List keys = new ArrayList<>(); Map recordMap = new TreeMap<>(); for (int i = 0; i < 100; i++) { diff --git a/hudi-hadoop-common/src/test/java/org/apache/hudi/io/hadoop/TestHoodieOrcReaderWriter.java b/hudi-hadoop-common/src/test/java/org/apache/hudi/io/hadoop/TestHoodieOrcReaderWriter.java index 522e3d7e65a6d..ebbd9d45e9991 100644 --- a/hudi-hadoop-common/src/test/java/org/apache/hudi/io/hadoop/TestHoodieOrcReaderWriter.java +++ b/hudi-hadoop-common/src/test/java/org/apache/hudi/io/hadoop/TestHoodieOrcReaderWriter.java @@ -26,6 +26,7 @@ import org.apache.hudi.common.config.HoodieStorageConfig; import org.apache.hudi.common.engine.TaskContextSupplier; import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType; +import org.apache.hudi.common.schema.HoodieSchema; import org.apache.hudi.common.testutils.HoodieTestUtils; import org.apache.hudi.io.storage.HoodieAvroFileReader; import org.apache.hudi.io.storage.HoodieIOFactory; @@ -35,7 +36,6 @@ import org.apache.hudi.storage.StorageConfiguration; import org.apache.hudi.storage.StoragePath; -import org.apache.avro.Schema; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.orc.CompressionKind; @@ -62,7 +62,7 @@ protected StoragePath getFilePath() { @Override protected HoodieAvroOrcWriter createWriter( - Schema avroSchema, boolean populateMetaFields) throws Exception { + HoodieSchema schema, boolean populateMetaFields) throws Exception { BloomFilter filter = BloomFilterFactory.createBloomFilter(1000, 0.00001, -1, BloomFilterTypeCode.SIMPLE.name()); StorageConfiguration conf = HoodieTestUtils.getDefaultStorageConfWithDefaults(); int orcStripSize = Integer.parseInt(HoodieStorageConfig.ORC_STRIPE_SIZE.defaultValue()); @@ -74,7 +74,7 @@ protected HoodieAvroOrcWriter createWriter( when(mockTaskContextSupplier.getPartitionIdSupplier()).thenReturn(partitionSupplier); when(partitionSupplier.get()).thenReturn(10); String instantTime = "000"; - return new HoodieAvroOrcWriter(instantTime, getFilePath(), config, avroSchema, mockTaskContextSupplier); + return new HoodieAvroOrcWriter(instantTime, getFilePath(), config, schema, mockTaskContextSupplier); } @Override diff --git a/hudi-hadoop-common/src/test/java/org/apache/hudi/io/hadoop/TestHoodieReaderWriterBase.java b/hudi-hadoop-common/src/test/java/org/apache/hudi/io/hadoop/TestHoodieReaderWriterBase.java index 4b8a6e66d5a4e..dd3533a4da213 100644 --- a/hudi-hadoop-common/src/test/java/org/apache/hudi/io/hadoop/TestHoodieReaderWriterBase.java +++ b/hudi-hadoop-common/src/test/java/org/apache/hudi/io/hadoop/TestHoodieReaderWriterBase.java @@ -35,7 +35,6 @@ import org.apache.hudi.storage.HoodieStorage; import org.apache.hudi.storage.StoragePath; -import org.apache.avro.Schema; import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.IndexedRecord; @@ -76,7 +75,7 @@ public abstract class TestHoodieReaderWriterBase { protected abstract StoragePath getFilePath(); protected abstract HoodieAvroFileWriter createWriter( - Schema avroSchema, boolean populateMetaFields) throws Exception; + HoodieSchema schema, boolean populateMetaFields) throws Exception; protected abstract HoodieAvroFileReader createReader( HoodieStorage storage) throws Exception; @@ -134,7 +133,7 @@ public void testWriteReadComplexRecord() throws Exception { String schemaPath = "/exampleSchemaWithUDT.avsc"; HoodieSchema schema = getSchemaFromResource(TestHoodieReaderWriterBase.class, schemaPath); HoodieSchema udtSchema = schema.getField("driver").get().schema().getTypes().get(1); - HoodieAvroFileWriter writer = createWriter(schema.toAvroSchema(), true); + HoodieAvroFileWriter writer = createWriter(schema, true); for (int i = 0; i < NUM_RECORDS; i++) { GenericRecord record = new GenericData.Record(schema.toAvroSchema()); String key = "key" + String.format("%02d", i); @@ -181,7 +180,7 @@ public void testReaderFilterRowKeys() throws Exception { protected void writeFileWithSimpleSchema() throws Exception { HoodieSchema schema = getHoodieSchemaFromResource(TestHoodieReaderWriterBase.class, "/exampleSchema.avsc"); - HoodieAvroFileWriter writer = createWriter(schema.getAvroSchema(), true); + HoodieAvroFileWriter writer = createWriter(schema, true); for (int i = 0; i < NUM_RECORDS; i++) { GenericRecord record = new GenericData.Record(schema.getAvroSchema()); String key = "key" + String.format("%02d", i); @@ -196,7 +195,7 @@ protected void writeFileWithSimpleSchema() throws Exception { private void writeFileWithSchemaWithMeta() throws Exception { HoodieSchema schema = getSchemaFromResource(TestHoodieReaderWriterBase.class, "/exampleSchemaWithMetaFields.avsc"); - HoodieAvroFileWriter writer = createWriter(schema.toAvroSchema(), true); + HoodieAvroFileWriter writer = createWriter(schema, true); for (int i = 0; i < NUM_RECORDS; i++) { GenericRecord record = new GenericData.Record(schema.toAvroSchema()); String key = "key" + String.format("%02d", i); diff --git a/hudi-hadoop-common/src/test/java/org/apache/hudi/io/hadoop/TestOrcReaderIterator.java b/hudi-hadoop-common/src/test/java/org/apache/hudi/io/hadoop/TestOrcReaderIterator.java index ede59441a2794..5c0a8802a1f4d 100644 --- a/hudi-hadoop-common/src/test/java/org/apache/hudi/io/hadoop/TestOrcReaderIterator.java +++ b/hudi-hadoop-common/src/test/java/org/apache/hudi/io/hadoop/TestOrcReaderIterator.java @@ -64,7 +64,7 @@ public void clearTempFile() { public void testOrcIteratorReadData() throws Exception { final Configuration conf = new Configuration(); HoodieSchema schema = getSchemaFromResource(TestOrcReaderIterator.class, "/simple-test.avsc"); - TypeDescription orcSchema = AvroOrcUtils.createOrcSchema(schema.toAvroSchema()); + TypeDescription orcSchema = AvroOrcUtils.createOrcSchema(schema); OrcFile.WriterOptions options = OrcFile.writerOptions(conf).setSchema(orcSchema).compress(CompressionKind.ZLIB); try (Writer writer = OrcFile.createWriter(filePath, options)) { VectorizedRowBatch batch = orcSchema.createRowBatch(); @@ -84,7 +84,7 @@ public void testOrcIteratorReadData() throws Exception { Reader reader = OrcFile.createReader(filePath, OrcFile.readerOptions(conf)); RecordReader recordReader = reader.rows(new Reader.Options(conf).schema(orcSchema)); - try (ClosableIterator iterator = new OrcReaderIterator<>(recordReader, schema.toAvroSchema(), orcSchema)) { + try (ClosableIterator iterator = new OrcReaderIterator<>(recordReader, schema, orcSchema)) { int recordCount = 0; while (iterator.hasNext()) { GenericRecord record = iterator.next(); diff --git a/hudi-hadoop-common/src/test/java/org/apache/parquet/avro/TestAvroSchemaConverter.java b/hudi-hadoop-common/src/test/java/org/apache/parquet/avro/TestAvroSchemaConverter.java index 8e9e61ee4a9b7..50977e573b6c1 100644 --- a/hudi-hadoop-common/src/test/java/org/apache/parquet/avro/TestAvroSchemaConverter.java +++ b/hudi-hadoop-common/src/test/java/org/apache/parquet/avro/TestAvroSchemaConverter.java @@ -20,10 +20,10 @@ package org.apache.parquet.avro; import org.apache.hudi.common.schema.HoodieSchema; +import org.apache.hudi.common.schema.HoodieSchemaCompatibility; +import org.apache.hudi.common.schema.HoodieSchemaField; +import org.apache.hudi.common.schema.HoodieSchemaType; -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; @@ -37,12 +37,8 @@ import java.util.Arrays; import java.util.Collections; +import java.util.List; -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; @@ -60,6 +56,7 @@ 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.assertTrue; import static org.junit.jupiter.api.Assertions.fail; public class TestAvroSchemaConverter { @@ -110,45 +107,45 @@ public static void setupConf() { + " 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(HoodieSchema schema, String schemaString) throws Exception { + testAvroToParquetConversion(new Configuration(false), schema, schemaString); } - private void testAvroToParquetConversion(Configuration conf, Schema avroSchema, String schemaString) + private void testAvroToParquetConversion(Configuration conf, HoodieSchema schema, String schemaString) throws Exception { HoodieAvroParquetSchemaConverter avroSchemaConverter = getAvroSchemaConverter(conf); - MessageType schema = avroSchemaConverter.convert(avroSchema); + MessageType messageType = avroSchemaConverter.convert(schema); MessageType expectedMT = MessageTypeParser.parseMessageType(schemaString); - assertEquals(expectedMT.toString(), schema.toString()); + assertEquals(expectedMT.toString(), messageType.toString()); } - private void testParquetToAvroConversion(Schema avroSchema, String schemaString) throws Exception { - testParquetToAvroConversion(new Configuration(false), avroSchema, schemaString); + private void testParquetToAvroConversion(HoodieSchema schema, String schemaString) throws Exception { + testParquetToAvroConversion(new Configuration(false), schema, schemaString); } - private void testParquetToAvroConversion(Configuration conf, Schema avroSchema, String schemaString) + private void testParquetToAvroConversion(Configuration conf, HoodieSchema schema, String schemaString) throws Exception { HoodieAvroParquetSchemaConverter avroSchemaConverter = getAvroSchemaConverter(conf); - Schema schema = avroSchemaConverter.convert(MessageTypeParser.parseMessageType(schemaString)); - assertEquals(avroSchema.toString(), schema.toString()); + HoodieSchema convertedSchema = avroSchemaConverter.convert(MessageTypeParser.parseMessageType(schemaString)); + assertEquals(schema.toString(), convertedSchema.toString()); } - private void testRoundTripConversion(Schema avroSchema, String schemaString) throws Exception { - testRoundTripConversion(new Configuration(), avroSchema, schemaString); + private void testRoundTripConversion(HoodieSchema schema, String schemaString) throws Exception { + testRoundTripConversion(new Configuration(), schema, schemaString); } - private void testRoundTripConversion(Configuration conf, Schema avroSchema, String schemaString) throws Exception { + private void testRoundTripConversion(Configuration conf, HoodieSchema schema, String schemaString) throws Exception { HoodieAvroParquetSchemaConverter avroSchemaConverter = getAvroSchemaConverter(conf); - MessageType schema = avroSchemaConverter.convert(avroSchema); + MessageType messageType = avroSchemaConverter.convert(schema); MessageType expectedMT = MessageTypeParser.parseMessageType(schemaString); - assertEquals(expectedMT.toString(), schema.toString()); - Schema convertedAvroSchema = avroSchemaConverter.convert(expectedMT); - assertEquals(avroSchema.toString(), convertedAvroSchema.toString()); + assertEquals(expectedMT.toString(), messageType.toString()); + HoodieSchema convertedSchema = avroSchemaConverter.convert(expectedMT); + assertEquals(schema.toString(), convertedSchema.toString()); } @Test() public void testTopLevelMustBeARecord() { - assertThrows("expected to throw", IllegalArgumentException.class, () -> getAvroSchemaConverter(new Configuration()).convert(Schema.create(INT))); + assertThrows("expected to throw", IllegalArgumentException.class, () -> getAvroSchemaConverter(new Configuration()).convert(HoodieSchema.create(HoodieSchemaType.INT))); } @Test @@ -156,7 +153,7 @@ public void testAllTypes() throws Exception { HoodieSchema schema = getSchemaFromResource(TestAvroSchemaConverter.class, "/parquet-java/all.avsc"); testAvroToParquetConversion( NEW_BEHAVIOR, - schema.toAvroSchema(), + schema, "message org.apache.parquet.avro.myrecord {\n" // Avro nulls are not encoded, unless they are null unions + " required boolean myboolean;\n" @@ -210,7 +207,7 @@ public void testAllTypes() throws Exception { public void testAllTypesOldListBehavior() throws Exception { HoodieSchema schema = getSchemaFromResource(TestAvroSchemaConverter.class, "/parquet-java/all.avsc"); testAvroToParquetConversion( - schema.toAvroSchema(), + schema, "message org.apache.parquet.avro.myrecord {\n" // Avro nulls are not encoded, unless they are null unions + " required boolean myboolean;\n" @@ -256,14 +253,14 @@ public void testAllTypesOldListBehavior() throws Exception { public void testAllTypesParquetToAvro() throws Exception { HoodieSchema schema = getSchemaFromResource(TestAvroSchemaConverter.class, "/parquet-java/allFromParquetNewBehavior.avsc"); // Cannot use round-trip assertion because enum is lost - testParquetToAvroConversion(NEW_BEHAVIOR, schema.toAvroSchema(), ALL_PARQUET_SCHEMA); + testParquetToAvroConversion(NEW_BEHAVIOR, schema, ALL_PARQUET_SCHEMA); } @Test public void testAllTypesParquetToAvroOldBehavior() throws Exception { HoodieSchema schema = getSchemaFromResource(TestAvroSchemaConverter.class, "/parquet-java/allFromParquetOldBehavior.avsc"); // Cannot use round-trip assertion because enum is lost - testParquetToAvroConversion(schema.toAvroSchema(), ALL_PARQUET_SCHEMA); + testParquetToAvroConversion(schema, ALL_PARQUET_SCHEMA); } @Test @@ -281,18 +278,17 @@ public void testParquetMapWithNonStringKeyFails() throws Exception { @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))); + HoodieSchema optionalInt = HoodieSchema.createNullable(HoodieSchemaType.INT); + HoodieSchema schema = HoodieSchema.createRecord("record1", null, null, false, + Collections.singletonList(HoodieSchemaField.of("myint", optionalInt, null, HoodieSchema.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))); + HoodieSchema optionalIntMap = HoodieSchema.createMap(HoodieSchema.createNullable(HoodieSchema.create(HoodieSchemaType.INT))); + HoodieSchema schema = HoodieSchema.createRecord("record1", null, null, false, + Collections.singletonList(HoodieSchemaField.of("myintmap", optionalIntMap, null, null))); testRoundTripConversion( schema, "message record1 {\n" + " required group myintmap (MAP) {\n" @@ -306,9 +302,9 @@ public void testOptionalMapValue() throws Exception { @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))); + HoodieSchema optionalIntArray = HoodieSchema.createArray(HoodieSchema.createNullable(HoodieSchemaType.INT)); + HoodieSchema schema = HoodieSchema.createRecord("record1", null, null, false, + Collections.singletonList(HoodieSchemaField.of("myintarray", optionalIntArray, null, null))); testRoundTripConversion( NEW_BEHAVIOR, schema, @@ -322,10 +318,12 @@ public void testOptionalArrayElement() throws Exception { @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))); + HoodieSchema multipleTypes = HoodieSchema.createUnion( + Arrays.asList(HoodieSchema.create(HoodieSchemaType.NULL), + HoodieSchema.create(HoodieSchemaType.INT), + HoodieSchema.create(HoodieSchemaType.FLOAT))); + HoodieSchema schema = HoodieSchema.createRecord("record2", null, null, false, + Collections.singletonList(HoodieSchemaField.of("myunion", multipleTypes, null, HoodieSchema.NULL_VALUE))); // Avro union is modelled using optional data members of the different // types. This does not translate back into an Avro union @@ -340,15 +338,13 @@ public void testUnionOfTwoTypes() throws Exception { @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)); + HoodieSchema optionalString = HoodieSchema.createNullable(HoodieSchema.create(HoodieSchemaType.STRING)); + List innerRecordFields = Arrays.asList( + HoodieSchemaField.of("s1", optionalString, null, HoodieSchema.NULL_VALUE), + HoodieSchemaField.of("s2", optionalString, null, HoodieSchema.NULL_VALUE)); + HoodieSchema innerRecord = HoodieSchema.createRecord("element", null, null, false, innerRecordFields); + HoodieSchema schema = HoodieSchema.createRecord("HasArray", null, null, false, + Collections.singletonList(HoodieSchemaField.of("myarray", HoodieSchema.createArray(HoodieSchema.createNullable(innerRecord)), null, null))); testRoundTripConversion( NEW_BEHAVIOR, @@ -366,15 +362,13 @@ public void testArrayOfOptionalRecords() throws Exception { @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)); + HoodieSchema optionalString = HoodieSchema.createNullable(HoodieSchemaType.STRING); + List innerRecordFields = Arrays.asList( + HoodieSchemaField.of("s1", optionalString, null, HoodieSchema.NULL_VALUE), + HoodieSchemaField.of("s2", optionalString, null, HoodieSchema.NULL_VALUE)); + HoodieSchema innerRecord = HoodieSchema.createRecord("InnerRecord", null, null, false, innerRecordFields); + HoodieSchema schema = HoodieSchema.createRecord("HasArray", null, null, false, + Collections.singletonList(HoodieSchemaField.of("myarray", HoodieSchema.createArray(HoodieSchema.createNullable(innerRecord)), null, null))); // Cannot use round-trip assertion because InnerRecord optional is removed testAvroToParquetConversion( @@ -390,11 +384,9 @@ public void testArrayOfOptionalRecordsOldBehavior() throws Exception { @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)); + HoodieSchema listOfLists = HoodieSchema.createNullable(HoodieSchema.createArray(HoodieSchema.createArray(HoodieSchema.create(HoodieSchemaType.INT)))); + HoodieSchema schema = HoodieSchema.createRecord("AvroCompatListInList", null, null, false, + Collections.singletonList(HoodieSchemaField.of("listOfLists", listOfLists, null, HoodieSchema.NULL_VALUE))); testRoundTripConversion( schema, @@ -418,11 +410,9 @@ public void testOldAvroListOfLists() throws Exception { @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)); + HoodieSchema listOfLists = HoodieSchema.createNullable(HoodieSchema.createArray(HoodieSchema.createArray(HoodieSchema.create(HoodieSchemaType.INT)))); + HoodieSchema schema = HoodieSchema.createRecord("ThriftCompatListInList", null, null, false, + Collections.singletonList(HoodieSchemaField.of("listOfLists", listOfLists, null, HoodieSchema.NULL_VALUE))); // Cannot use round-trip assertion because repeated group names differ testParquetToAvroConversion( @@ -450,11 +440,9 @@ 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)); + HoodieSchema listOfLists = HoodieSchema.createNullable(HoodieSchema.createArray(HoodieSchema.createArray(HoodieSchema.create(HoodieSchemaType.INT)))); + HoodieSchema schema = HoodieSchema.createRecord("UnknownTwoLevelListInList", null, null, false, + Collections.singletonList(HoodieSchemaField.of("listOfLists", listOfLists, null, HoodieSchema.NULL_VALUE))); // Cannot use round-trip assertion because repeated group names differ testParquetToAvroConversion( @@ -479,9 +467,9 @@ public void testUnknownTwoLevelListOfLists() throws Exception { @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))); + HoodieSchema map = HoodieSchema.createMap(HoodieSchema.create(HoodieSchemaType.INT)); + HoodieSchema schema = HoodieSchema.createRecord("myrecord", null, null, false, + Collections.singletonList(HoodieSchemaField.of("mymap", map, null, null))); String parquetSchema = "message myrecord {\n" + " required group mymap (MAP) {\n" + " repeated group map {\n" + " required binary key (UTF8);\n" @@ -496,18 +484,18 @@ public void testParquetMapWithoutMapKeyValueAnnotation() throws Exception { @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))); + HoodieSchema decimal = HoodieSchema.createDecimal(9, 2); + HoodieSchema schema = HoodieSchema.createRecord("myrecord", null, null, false, + Collections.singletonList(HoodieSchemaField.of("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))); + HoodieSchema decimal = HoodieSchema.createDecimal("dec", null, null, 9, 2, 8); + HoodieSchema schema = HoodieSchema.createRecord("myrecord", null, null, false, + Collections.singletonList(HoodieSchemaField.of("dec", decimal, null, null))); testRoundTripConversion( schema, "message myrecord {\n" + " required fixed_len_byte_array(8) dec (DECIMAL(9,2));\n" + "}\n"); @@ -515,8 +503,8 @@ public void testDecimalFixedType() throws Exception { @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))); + HoodieSchema expected = HoodieSchema.createRecord("myrecord", null, null, false, + Collections.singletonList(HoodieSchemaField.of("dec", HoodieSchema.create(HoodieSchemaType.INT), null, null))); // the decimal portion is lost because it isn't valid in Avro testParquetToAvroConversion( @@ -525,8 +513,9 @@ public void testDecimalIntegerType() throws Exception { @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))); + HoodieSchema expected = HoodieSchema.createRecord( + "myrecord", null, null, false, + Collections.singletonList(HoodieSchemaField.of("dec", HoodieSchema.create(HoodieSchemaType.LONG), null, null))); // the decimal portion is lost because it isn't valid in Avro testParquetToAvroConversion( @@ -538,10 +527,9 @@ 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))); + HoodieSchema int96schema = HoodieSchema.createFixed("INT96", "INT96 represented as byte[12]", null, 12); + HoodieSchema schema = HoodieSchema.createRecord("myrecord", null, null, false, + Collections.singletonList(HoodieSchemaField.of("int96_field", int96schema, null, null))); testParquetToAvroConversion(enableInt96ReadingConfig, schema, "message myrecord {\n" + " required int96 int96_field;\n" @@ -550,8 +538,6 @@ public void testParquetInt96AsFixed12AvroType() throws Exception { @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"); @@ -563,9 +549,10 @@ public void testParquetInt96DefaultFail() throws Exception { @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))); + HoodieSchema date = HoodieSchema.createDate(); + HoodieSchema expected = HoodieSchema.createRecord( + "myrecord", null, null, false, + Collections.singletonList(HoodieSchemaField.of("date", date, null, null))); testRoundTripConversion(expected, "message myrecord {\n" + " required int32 date (DATE);\n" + "}\n"); @@ -587,9 +574,10 @@ public void testDateType() throws Exception { @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))); + HoodieSchema timeMillis = HoodieSchema.createTimeMillis(); + HoodieSchema expected = HoodieSchema.createRecord( + "myrecord", null, null, false, + Collections.singletonList(HoodieSchemaField.of("time", timeMillis, null, null))); testRoundTripConversion( expected, "message myrecord {\n" + " required int32 time (TIME(MILLIS,true));\n" + "}\n"); @@ -612,9 +600,10 @@ public void testTimeMillisType() throws Exception { @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))); + HoodieSchema timeMicros = HoodieSchema.createTimeMicros(); + HoodieSchema expected = HoodieSchema.createRecord( + "myrecord", null, null, false, + Collections.singletonList(HoodieSchemaField.of("time", timeMicros, null, null))); testRoundTripConversion( expected, "message myrecord {\n" + " required int64 time (TIME(MICROS,true));\n" + "}\n"); @@ -637,14 +626,15 @@ public void testTimeMicrosType() throws Exception { @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))); + HoodieSchema timestampMillis = HoodieSchema.createTimestampMillis(); + HoodieSchema expected = HoodieSchema.createRecord( + "myrecord", null, null, false, + Collections.singletonList(HoodieSchemaField.of("timestamp", timestampMillis, null, null))); testRoundTripConversion( expected, "message myrecord {\n" + " required int64 timestamp (TIMESTAMP(MILLIS,true));\n" + "}\n"); - final Schema converted = getAvroSchemaConverter(new Configuration()) + final HoodieSchema converted = getAvroSchemaConverter(new Configuration()) .convert(Types.buildMessage() .addField(Types.primitive(INT64, Type.Repetition.REQUIRED) .as(LogicalTypeAnnotation.timestampType( @@ -656,8 +646,8 @@ public void testTimestampMillisType() throws Exception { "local-timestamp-millis", converted .getField("timestamp_type") + .get() .schema() - .getLogicalType() .getName()); for (PrimitiveTypeName primitive : @@ -678,9 +668,10 @@ public void testTimestampMillisType() throws Exception { @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))); + HoodieSchema localTimestampMillis = HoodieSchema.createLocalTimestampMillis(); + HoodieSchema expected = HoodieSchema.createRecord( + "myrecord", null, null, false, + Collections.singletonList(HoodieSchemaField.of("timestamp", localTimestampMillis, null, null))); testRoundTripConversion( expected, "message myrecord {\n" + " required int64 timestamp (TIMESTAMP(MILLIS,false));\n" + "}\n"); @@ -703,9 +694,10 @@ public void testLocalTimestampMillisType() throws Exception { @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))); + HoodieSchema timestampMicros = HoodieSchema.createTimestampMicros(); + HoodieSchema expected = HoodieSchema.createRecord( + "myrecord", null, null, false, + Collections.singletonList(HoodieSchemaField.of("timestamp", timestampMicros, null, null))); testRoundTripConversion( expected, "message myrecord {\n" + " required int64 timestamp (TIMESTAMP(MICROS,true));\n" + "}\n"); @@ -725,7 +717,7 @@ public void testTimestampMicrosType() throws Exception { () -> getAvroSchemaConverter(new Configuration()).convert(message(type))); } - final Schema converted = getAvroSchemaConverter(new Configuration()) + final HoodieSchema converted = getAvroSchemaConverter(new Configuration()) .convert(Types.buildMessage() .addField(Types.primitive(INT64, Type.Repetition.REQUIRED) .as(LogicalTypeAnnotation.timestampType( @@ -738,16 +730,17 @@ public void testTimestampMicrosType() throws Exception { "local-timestamp-micros", converted .getField("timestamp_type") + .get() .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))); + HoodieSchema localTimestampMicros = HoodieSchema.createLocalTimestampMicros(); + HoodieSchema expected = HoodieSchema.createRecord( + "myrecord", null, null, false, + Collections.singletonList(HoodieSchemaField.of("timestamp", localTimestampMicros, null, null))); testRoundTripConversion( expected, "message myrecord {\n" + " required int64 timestamp (TIMESTAMP(MICROS,false));\n" + "}\n"); @@ -770,10 +763,14 @@ public void testLocalTimestampMicrosType() throws Exception { @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)); + HoodieSchema innerA1 = HoodieSchema.createRecord("a1", null, "a12", false, + Collections.singletonList(HoodieSchemaField.of("a4", HoodieSchema.create(HoodieSchemaType.FLOAT)))); + HoodieSchema outerA1 = HoodieSchema.createRecord("a1", null,null, false, + Arrays.asList( + HoodieSchemaField.of("a2", HoodieSchema.create(HoodieSchemaType.FLOAT)), + HoodieSchemaField.of("a1", HoodieSchema.createNullable(innerA1), null, HoodieSchema.NULL_VALUE))); + HoodieSchema schema = HoodieSchema.createRecord("Message", null, null, false, + Collections.singletonList(HoodieSchemaField.of("a1", HoodieSchema.createNullable(outerA1), null, HoodieSchema.NULL_VALUE))); String parquetSchema = "message Message {\n" + " optional group a1 {\n" @@ -790,14 +787,24 @@ public void testReuseNameInNestedStructure() throws Exception { @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)); + HoodieSchema a2 = HoodieSchema.createRecord( + "a2", null, null, false, + Collections.singletonList(HoodieSchemaField.of("a4", HoodieSchema.create(HoodieSchemaType.FLOAT)))); + HoodieSchema a22 = HoodieSchema.createRecord( + "a2", "a22", null, Arrays.asList( + HoodieSchemaField.of("a4", HoodieSchema.create(HoodieSchemaType.FLOAT)), + HoodieSchemaField.of("a5", HoodieSchema.create(HoodieSchemaType.FLOAT)))); + + HoodieSchema a1 = HoodieSchema.createRecord("a1", null, null, false, + Collections.singletonList(HoodieSchemaField.of("a2", HoodieSchema.createNullable(a2), null, HoodieSchema.NULL_VALUE))); + HoodieSchema a3 = HoodieSchema.createRecord("a3", null, null, false, + Collections.singletonList(HoodieSchemaField.of("a2", HoodieSchema.createNullable(a22), null, HoodieSchema.NULL_VALUE))); + + HoodieSchema schema = HoodieSchema.createRecord("Message", null, null, false, + Arrays.asList( + HoodieSchemaField.of("a1", HoodieSchema.createNullable(a1), null, HoodieSchema.NULL_VALUE), + HoodieSchemaField.of("a3", HoodieSchema.createNullable(a3), null, HoodieSchema.NULL_VALUE))); - Schema schema = record("Message", optionalField("a1", a1), optionalField("a3", a3)); String parquetSchema = "message Message {\n" + " optional group a1 {\n" @@ -819,33 +826,33 @@ public void testReuseNameInNestedStructureAtSameLevel() throws Exception { @Test public void testUUIDType() throws Exception { - Schema fromAvro = Schema.createRecord( + HoodieSchema fromSchema = HoodieSchema.createRecord( "myrecord", null, null, false, - Arrays.asList( - new Schema.Field("uuid", LogicalTypes.uuid().addToSchema(Schema.create(STRING)), null, null))); + Collections.singletonList( + HoodieSchemaField.of("uuid", HoodieSchema.createUUID(), null, null))); String parquet = "message myrecord {\n" + " required binary uuid (STRING);\n" + "}\n"; - Schema toAvro = Schema.createRecord( + HoodieSchema toSchema = HoodieSchema.createRecord( "myrecord", null, null, false, - Arrays.asList(new Schema.Field("uuid", Schema.create(STRING), null, null))); + Collections.singletonList(HoodieSchemaField.of("uuid", HoodieSchema.create(HoodieSchemaType.STRING), null, null))); - testAvroToParquetConversion(fromAvro, parquet); - testParquetToAvroConversion(toAvro, parquet); + testAvroToParquetConversion(fromSchema, parquet); + testParquetToAvroConversion(toSchema, parquet); - assertEquals( - COMPATIBLE, checkReaderWriterCompatibility(fromAvro, toAvro).getType()); + assertTrue(HoodieSchemaCompatibility.areSchemasCompatible(fromSchema, toSchema)); } @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))); + HoodieSchema uuid = HoodieSchema.createUUID(); + HoodieSchema expected = HoodieSchema.createRecord( + "myrecord", null, null, false, + Collections.singletonList(HoodieSchemaField.of("uuid", uuid, null, null))); testRoundTripConversion( conf(AvroWriteSupport.WRITE_PARQUET_UUID, true), @@ -866,7 +873,7 @@ public void testAvroFixed12AsParquetInt96Type() throws Exception { "mynestedrecord.mymap"); testAvroToParquetConversion( conf, - schema.toAvroSchema(), + schema, "message org.apache.parquet.avro.fixedToInt96 {\n" + " required int96 int96;\n" + " required fixed_len_byte_array(12) notanint96;\n" @@ -889,11 +896,7 @@ public void testAvroFixed12AsParquetInt96Type() throws Exception { 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.toAvroSchema())); - } - - public static Schema optional(Schema original) { - return Schema.createUnion(Arrays.asList(Schema.create(Schema.Type.NULL), original)); + () -> getAvroSchemaConverter(conf).convert(schema)); } public static MessageType message(PrimitiveType primitive) { @@ -921,32 +924,6 @@ public static void assertThrows(String message, Class expec } } - 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); diff --git a/hudi-hadoop-common/src/test/java/org/apache/parquet/schema/TestSchemaRepairEquivalence.java b/hudi-hadoop-common/src/test/java/org/apache/parquet/schema/TestSchemaRepairEquivalence.java index 75fe9ffde7d61..4f7aa2b305558 100644 --- a/hudi-hadoop-common/src/test/java/org/apache/parquet/schema/TestSchemaRepairEquivalence.java +++ b/hudi-hadoop-common/src/test/java/org/apache/parquet/schema/TestSchemaRepairEquivalence.java @@ -19,6 +19,8 @@ package org.apache.parquet.schema; +import org.apache.hudi.common.schema.HoodieSchema; + import org.apache.avro.LogicalTypes; import org.apache.avro.Schema; import org.apache.avro.SchemaBuilder; @@ -52,17 +54,17 @@ private void assertRepairEquivalence(Schema requestedAvro, Schema tableAvro) { Schema repairedAvro = AvroSchemaRepair.repairLogicalTypes(requestedAvro, tableAvro); // Convert to Parquet schemas - MessageType requestedParquet = converter.convert(requestedAvro); - MessageType tableParquet = converter.convert(tableAvro); + MessageType requestedParquet = converter.convert(HoodieSchema.fromAvroSchema(requestedAvro)); + MessageType tableParquet = converter.convert(HoodieSchema.fromAvroSchema(tableAvro)); // Apply Parquet repair MessageType repairedParquet = SchemaRepair.repairLogicalTypes(requestedParquet, tableParquet); // Convert repaired Parquet back to Avro - Schema repairedParquetAsAvro = converter.convert(repairedParquet); + HoodieSchema repairedParquetAsSchema = converter.convert(repairedParquet); // Verify equivalence - assertEquals(repairedAvro, repairedParquetAsAvro, + assertEquals(repairedAvro, repairedParquetAsSchema.toAvroSchema(), "SchemaRepair and AvroSchemaRepair should produce equivalent results"); } diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HiveHoodieReaderContext.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HiveHoodieReaderContext.java index 38cd64c13f37d..1ebf3044d5103 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HiveHoodieReaderContext.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HiveHoodieReaderContext.java @@ -134,8 +134,8 @@ private ClosableIterator getFileRecordIterator(StoragePath filePa // Read file schema and repair logical types if needed HoodieSchema fileSchema; if (isParquetOrOrc) { - Schema avroFileSchema = HoodieIOFactory.getIOFactory(storage).getFileFormatUtils(filePath).readAvroSchema(storage, filePath); - Schema repairedAvroSchema = AvroSchemaRepair.repairLogicalTypes(avroFileSchema, dataSchema.toAvroSchema()); + HoodieSchema rawFileSchema = HoodieIOFactory.getIOFactory(storage).getFileFormatUtils(filePath).readSchema(storage, filePath); + Schema repairedAvroSchema = AvroSchemaRepair.repairLogicalTypes(rawFileSchema.toAvroSchema(), dataSchema.toAvroSchema()); fileSchema = HoodieSchema.fromAvroSchema(repairedAvroSchema); } else { fileSchema = dataSchema; diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieHFileRecordReader.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieHFileRecordReader.java index b58fd4550086b..9ff7d83788080 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieHFileRecordReader.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieHFileRecordReader.java @@ -32,7 +32,6 @@ import org.apache.hudi.storage.StorageConfiguration; import org.apache.hudi.storage.StoragePath; -import org.apache.avro.Schema; import org.apache.avro.generic.IndexedRecord; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.ArrayWritable; @@ -54,7 +53,7 @@ public class HoodieHFileRecordReader implements RecordReader> recordIterator; - private final Schema schema; + private final HoodieSchema schema; public HoodieHFileRecordReader(Configuration conf, InputSplit split, JobConf job) throws IOException { FileSplit fileSplit = (FileSplit) split; @@ -64,15 +63,14 @@ public HoodieHFileRecordReader(Configuration conf, InputSplit split, JobConf job reader = HoodieIOFactory.getIOFactory(HoodieStorageUtils.getStorage(path, storageConf)).getReaderFactory(HoodieRecord.HoodieRecordType.AVRO) .getFileReader(hoodieConfig, path, HoodieFileFormat.HFILE, Option.empty()); - //TODO boundary for now to revisit in later pr to use HoodieSchema - schema = reader.getSchema().getAvroSchema(); + schema = reader.getSchema(); valueObj = new ArrayWritable(Writable.class, new Writable[schema.getFields().size()]); } @Override public boolean next(NullWritable key, ArrayWritable value) throws IOException { if (recordIterator == null) { - recordIterator = reader.getRecordIterator(HoodieSchema.fromAvroSchema(schema)); + recordIterator = reader.getRecordIterator(schema); } if (!recordIterator.hasNext()) { @@ -80,7 +78,7 @@ public boolean next(NullWritable key, ArrayWritable value) throws IOException { } IndexedRecord record = recordIterator.next().getData(); - ArrayWritable aWritable = (ArrayWritable) HoodieRealtimeRecordReaderUtils.avroToArrayWritable(record, schema); + ArrayWritable aWritable = (ArrayWritable) HoodieRealtimeRecordReaderUtils.avroToArrayWritable(record, schema.toAvroSchema()); value.set(aWritable.get()); count++; return true; 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 4052098a07ff5..3c635634c4765 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 @@ -18,8 +18,8 @@ package org.apache.hudi.hadoop.avro; -import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.common.schema.HoodieSchema; +import org.apache.hudi.common.schema.HoodieSchemaUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.hadoop.HoodieColumnProjectionUtils; import org.apache.hudi.hadoop.utils.HoodieRealtimeRecordReaderUtils; @@ -53,11 +53,11 @@ public class HoodieAvroParquetReader extends RecordReader { private final ParquetRecordReader parquetRecordReader; - private Schema baseSchema; + private HoodieSchema baseSchema; public HoodieAvroParquetReader(InputSplit inputSplit, Configuration conf, Option internalSchemaOption, Option dataSchema) throws IOException { if (dataSchema.isPresent()) { - baseSchema = dataSchema.get(); + baseSchema = HoodieSchema.fromAvroSchema(dataSchema.get()); } else { // get base schema ParquetMetadata fileFooter = @@ -68,19 +68,19 @@ public HoodieAvroParquetReader(InputSplit inputSplit, Configuration conf, Option if (internalSchemaOption.isPresent()) { // do schema reconciliation in case there exists read column which is not in the file schema. InternalSchema mergedInternalSchema = new InternalSchemaMerger( - InternalSchemaConverter.convert(HoodieSchema.fromAvroSchema(baseSchema)), + InternalSchemaConverter.convert(baseSchema), internalSchemaOption.get(), true, true).mergeSchema(); - baseSchema = InternalSchemaConverter.convert(mergedInternalSchema, baseSchema.getFullName()).getAvroSchema(); + baseSchema = InternalSchemaConverter.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); + HoodieSchema filterSchema = HoodieSchemaUtils.generateProjectionSchema(baseSchema, readColNames); + AvroReadSupport.setAvroReadSchema(conf, filterSchema.toAvroSchema()); + AvroReadSupport.setRequestedProjection(conf, filterSchema.toAvroSchema()); } } parquetRecordReader = new ParquetRecordReader<>(new AvroReadSupport<>(), getFilter(conf)); @@ -104,7 +104,7 @@ public Void getCurrentKey() throws IOException, InterruptedException { @Override public ArrayWritable getCurrentValue() throws IOException, InterruptedException { GenericRecord record = parquetRecordReader.getCurrentValue(); - return (ArrayWritable) HoodieRealtimeRecordReaderUtils.avroToArrayWritable(record, baseSchema, true); + return (ArrayWritable) HoodieRealtimeRecordReaderUtils.avroToArrayWritable(record, baseSchema.toAvroSchema(), true); } @Override 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 c0444a6092838..5708475839a7a 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 @@ -66,7 +66,7 @@ import java.util.Set; import java.util.stream.Collectors; -import static org.apache.hudi.avro.HoodieAvroUtils.isMetadataField; +import static org.apache.hudi.common.schema.HoodieSchemaUtils.isMetadataField; /** * Helper class to serialize hive writable type to avro record. diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/cdc/CDCFileGroupIterator.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/cdc/CDCFileGroupIterator.scala index 7585d9d93e65d..eb4f5828fa6a2 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/cdc/CDCFileGroupIterator.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/cdc/CDCFileGroupIterator.scala @@ -149,7 +149,7 @@ class CDCFileGroupIterator(split: HoodieCDCFileGroupSplit, private lazy val tableSchemaOpt = if (schema != null) { val hadoopConf = storage.getConf.unwrapAs(classOf[Configuration]) - val parquetSchema = getAvroSchemaConverter(hadoopConf).convert(schema.getAvroSchema) + val parquetSchema = getAvroSchemaConverter(hadoopConf).convert(schema) org.apache.hudi.common.util.Option.of(parquetSchema) } else { org.apache.hudi.common.util.Option.empty[org.apache.parquet.schema.MessageType]() diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedFileFormat.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedFileFormat.scala index 4dd3f66551de5..4cf0e6930fc32 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedFileFormat.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedFileFormat.scala @@ -25,7 +25,7 @@ import org.apache.hudi.client.utils.SparkInternalSchemaConverter import org.apache.hudi.common.config.{HoodieMemoryConfig, TypedProperties} import org.apache.hudi.common.fs.FSUtils import org.apache.hudi.common.model.HoodieFileFormat -import org.apache.hudi.common.schema.{HoodieSchema, HoodieSchemaUtils} +import org.apache.hudi.common.schema.HoodieSchemaUtils import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient, ParquetTableSchemaResolver} import org.apache.hudi.common.table.read.HoodieFileGroupReader import org.apache.hudi.common.util.{Option => HOption} @@ -92,11 +92,11 @@ class HoodieFileGroupReaderBasedFileFormat(tablePath: String, private lazy val tableSchemaAsMessageType: HOption[MessageType] = { HOption.ofNullable( - ParquetTableSchemaResolver.convertAvroSchemaToParquet(schema.getAvroSchema, new Configuration()) + ParquetTableSchemaResolver.convertAvroSchemaToParquet(schema, new Configuration()) ) } - private lazy val hasTimestampMillisFieldInTableSchema = AvroSchemaRepair.hasTimestampMillisField(schema.getAvroSchema) + private lazy val hasTimestampMillisFieldInTableSchema = AvroSchemaRepair.hasTimestampMillisField(schema.toAvroSchema) private lazy val supportBatchWithTableSchema = HoodieSparkUtils.gteqSpark3_5 || !hasTimestampMillisFieldInTableSchema override def shortName(): String = "HudiFileGroup" diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/SparkHelpers.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/SparkHelpers.scala index a944a7d43661d..1aaf818a32b1f 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/SparkHelpers.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/SparkHelpers.scala @@ -23,12 +23,12 @@ import org.apache.hudi.common.bloom.{BloomFilter, BloomFilterFactory} import org.apache.hudi.common.config.{HoodieParquetConfig, HoodieStorageConfig} import org.apache.hudi.common.config.HoodieStorageConfig.{BLOOM_FILTER_DYNAMIC_MAX_ENTRIES, BLOOM_FILTER_FPP_VALUE, BLOOM_FILTER_NUM_ENTRIES_VALUE, BLOOM_FILTER_TYPE} import org.apache.hudi.common.model.{HoodieFileFormat, HoodieRecord} +import org.apache.hudi.common.schema.HoodieSchema import org.apache.hudi.common.util.Option import org.apache.hudi.io.storage.HoodieIOFactory import org.apache.hudi.io.storage.hadoop.HoodieAvroParquetWriter import org.apache.hudi.storage.{HoodieStorage, StorageConfiguration, StoragePath} -import org.apache.avro.Schema import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.FileSystem import org.apache.parquet.avro.HoodieAvroParquetSchemaConverter.getAvroSchemaConverter @@ -51,7 +51,7 @@ object SparkHelpers { val sourceRecords = HoodieIOFactory.getIOFactory(storage) .getFileFormatUtils(HoodieFileFormat.PARQUET) .readAvroRecords(storage, sourceFile).asScala - val schema: Schema = sourceRecords.head.getSchema + val schema: HoodieSchema = HoodieSchema.fromAvroSchema(sourceRecords.head.getSchema) val filter: BloomFilter = BloomFilterFactory.createBloomFilter( BLOOM_FILTER_NUM_ENTRIES_VALUE.defaultValue.toInt, BLOOM_FILTER_FPP_VALUE.defaultValue.toDouble, BLOOM_FILTER_DYNAMIC_MAX_ENTRIES.defaultValue.toInt, BLOOM_FILTER_TYPE.defaultValue); diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestHoodieIndex.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestHoodieIndex.java index 9734c7d2492b9..7c6169a781837 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestHoodieIndex.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestHoodieIndex.java @@ -30,6 +30,7 @@ import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.schema.HoodieSchemaUtils; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieInstant; @@ -91,7 +92,6 @@ import scala.Tuple2; -import static org.apache.hudi.avro.HoodieAvroUtils.addMetadataFields; import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; import static org.apache.hudi.common.testutils.HoodieTestUtils.SIMPLE_RECORD_SCHEMA; import static org.apache.hudi.common.testutils.HoodieTestUtils.TIMELINE_FACTORY; @@ -416,7 +416,7 @@ public void testTagLocationAndFetchRecordLocations(IndexType indexType, boolean } // We create three parquet files, each having one record (two different partitions) - HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(metaClient, addMetadataFields(SIMPLE_RECORD_SCHEMA.toAvroSchema()), metadataWriter); + HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(metaClient, HoodieSchemaUtils.addMetadataFields(SIMPLE_RECORD_SCHEMA), metadataWriter); final String fileId1 = "fileID1"; final String fileId2 = "fileID2"; final String fileId3 = "fileID3"; diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestOrcBootstrap.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestOrcBootstrap.java index d3acadf29471a..46f1fbb795e76 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestOrcBootstrap.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestOrcBootstrap.java @@ -446,7 +446,7 @@ private static JavaRDD generateInputBatch(JavaSparkContext jsc, HoodieSchema hoodieSchema = AvroOrcUtils.createSchemaWithDefaultValue(orcSchema, "test_orc_record", null, true); - Iterator recIterator = new OrcReaderIterator(recordReader, hoodieSchema.toAvroSchema(), orcSchema); + Iterator recIterator = new OrcReaderIterator(recordReader, hoodieSchema, orcSchema); return StreamSupport.stream(Spliterators.spliteratorUnknownSize(recIterator, 0), false).map(gr -> { String key = gr.get("_row_key").toString(); diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestSparkBinaryCopyClusteringAndValidationMeta.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestSparkBinaryCopyClusteringAndValidationMeta.java index 75988064d3d90..e3cf43fd8a7be 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestSparkBinaryCopyClusteringAndValidationMeta.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestSparkBinaryCopyClusteringAndValidationMeta.java @@ -27,7 +27,6 @@ import org.apache.hudi.common.bloom.BloomFilter; import org.apache.hudi.common.bloom.BloomFilterFactory; import org.apache.hudi.common.bloom.BloomFilterTypeCode; -import org.apache.hudi.common.schema.HoodieSchema; import org.apache.hudi.common.config.HoodieConfig; import org.apache.hudi.common.config.HoodieParquetConfig; import org.apache.hudi.common.engine.LocalTaskContextSupplier; @@ -36,6 +35,7 @@ import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieReplaceCommitMetadata; import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.schema.HoodieSchema; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; @@ -260,8 +260,8 @@ public void testSupportBinaryStreamCopy() throws IOException { Assertions.assertFalse(strategy.supportBinaryStreamCopy(groups, new HashMap<>())); } - private String makeTestFile(String fileName, HoodieSchema hoodieSchema, MessageType messageType, BloomFilter filter) throws IOException { - HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(messageType, hoodieSchema.getAvroSchema(), Option.of(filter), new Properties()); + private String makeTestFile(String fileName, HoodieSchema schema, MessageType messageType, BloomFilter filter) throws IOException { + HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(messageType, schema, Option.of(filter), new Properties()); StoragePath filePath = new StoragePath(tempDir.resolve(fileName).toAbsolutePath().toString()); HoodieConfig hoodieConfig = new HoodieConfig(); hoodieConfig.setValue("hoodie.base.path", basePath); diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java index 93ae04e5df29b..e0e6f4c7fc708 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java @@ -379,7 +379,7 @@ public void testLogFileCountsAfterCompaction(IndexType indexType) throws Excepti try (HoodieTableMetadataWriter metadataWriter = SparkHoodieBackedTableMetadataWriter.create( writeClient.getEngineContext().getStorageConf(), config, writeClient.getEngineContext())) { HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable - .of(metaClient, HoodieTestDataGenerator.AVRO_SCHEMA_WITH_METADATA_FIELDS, metadataWriter); + .of(metaClient, HoodieTestDataGenerator.HOODIE_SCHEMA_WITH_METADATA_FIELDS, metadataWriter); Set allPartitions = lastCommit.updatedRecords.stream() .map(record -> record.getPartitionPath()) @@ -499,7 +499,7 @@ public void testLogBlocksCountsAfterLogCompaction(boolean populateMetaFields, St try (HoodieTableMetadataWriter metadataWriter = SparkHoodieBackedTableMetadataWriter.create( writeClient.getEngineContext().getStorageConf(), config, writeClient.getEngineContext())) { HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable - .of(metaClient, HoodieTestDataGenerator.AVRO_SCHEMA_WITH_METADATA_FIELDS, metadataWriter); + .of(metaClient, HoodieTestDataGenerator.HOODIE_SCHEMA_WITH_METADATA_FIELDS, metadataWriter); Set allPartitions = lastCommit.updatedRecords.stream() .map(record -> record.getPartitionPath()) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestParquetReaderCompatibility.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestParquetReaderCompatibility.scala index 7bf63134120c4..21674026d3593 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestParquetReaderCompatibility.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestParquetReaderCompatibility.scala @@ -311,8 +311,7 @@ class TestParquetReaderCompatibility extends HoodieSparkWriterTestBase { private def getListType(hadoopConf: Configuration, path: StoragePath): String = { val reader = HoodieIOFactory.getIOFactory(HoodieStorageUtils.getStorage(path, HadoopFSUtils.getStorageConf(hadoopConf))).getReaderFactory(HoodieRecordType.AVRO).getFileReader(DEFAULT_HUDI_CONFIG_FOR_READER, path) - //TODO boundary to revisit in later pr to use HoodieSchema directly - val schema = ParquetTableSchemaResolver.convertAvroSchemaToParquet(reader.getSchema.toAvroSchema, hadoopConf) + val schema = ParquetTableSchemaResolver.convertAvroSchemaToParquet(reader.getSchema, hadoopConf) val list = schema.getFields.asScala.find(_.getName == TestParquetReaderCompatibility.listFieldName).get val groupType = list.asGroupType() diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestRepairsProcedure.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestRepairsProcedure.scala index 0126ad52e92b7..0a71f63fbf9c2 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestRepairsProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestRepairsProcedure.scala @@ -508,7 +508,7 @@ class TestRepairsProcedure extends HoodieSparkProcedureTestBase { // generate 200 records val schema: HoodieSchema = HoodieSchemaUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema) - val testTable: HoodieSparkWriteableTestTable = HoodieSparkWriteableTestTable.of(metaClient, schema.toAvroSchema) + val testTable: HoodieSparkWriteableTestTable = HoodieSparkWriteableTestTable.of(metaClient, schema) val testUtil = new SchemaTestUtil val hoodieRecords1 = testUtil.generateHoodieTestRecords(0, 100, schema) diff --git a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestCluster.java b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestCluster.java index e0c642a5de633..f54925114e568 100644 --- a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestCluster.java +++ b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestCluster.java @@ -222,7 +222,7 @@ private void generateParquetData(Path filePath, boolean isParquetSchemaSimple) org.apache.parquet.schema.MessageType parquetSchema = new AvroSchemaConverter().convert(schema.toAvroSchema()); BloomFilter filter = BloomFilterFactory.createBloomFilter(1000, 0.0001, -1, BloomFilterTypeCode.SIMPLE.name()); - HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(parquetSchema, schema.toAvroSchema(), Option.of(filter), new Properties()); + HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(parquetSchema, schema, Option.of(filter), new Properties()); ParquetWriter writer = new ParquetWriter(filePath, writeSupport, CompressionCodecName.GZIP, 120 * 1024 * 1024, ParquetWriter.DEFAULT_PAGE_SIZE, ParquetWriter.DEFAULT_PAGE_SIZE, ParquetWriter.DEFAULT_IS_DICTIONARY_ENABLED, ParquetWriter.DEFAULT_IS_VALIDATING_ENABLED, ParquetWriter.DEFAULT_WRITER_VERSION, dfsCluster.getFileSystem().getConf()); diff --git a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestUtil.java b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestUtil.java index 6c5ecb0376d32..ebe139dba5f98 100644 --- a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestUtil.java +++ b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestUtil.java @@ -49,7 +49,6 @@ import org.apache.hudi.common.testutils.InProcessTimeGenerator; import org.apache.hudi.common.testutils.SchemaTestUtil; import org.apache.hudi.common.testutils.minicluster.ZookeeperTestService; -import org.apache.hudi.io.util.FileIOUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.hadoop.fs.HadoopFSUtils; @@ -59,12 +58,12 @@ import org.apache.hudi.hive.ddl.HiveQueryDDLExecutor; import org.apache.hudi.hive.ddl.QueryBasedDDLExecutor; import org.apache.hudi.hive.util.IMetaStoreClientUtil; +import org.apache.hudi.io.util.FileIOUtils; import org.apache.hudi.storage.HoodieInstantWriter; import org.apache.hudi.storage.HoodieStorage; import org.apache.hudi.storage.StoragePath; import org.apache.hudi.storage.HoodieStorageUtils; -import org.apache.avro.Schema; import org.apache.avro.generic.IndexedRecord; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; @@ -403,7 +402,7 @@ public static void createCOWTableWithSchema(String instantTime, String schemaFil Path filePath = new Path(partPath.toString() + "/" + FSUtils.makeBaseFileName(instantTime, "1-0-1", fileId, HoodieTableConfig.BASE_FILE_FORMAT.defaultValue().getFileExtension())); HoodieSchema schema = SchemaTestUtil.getSchemaFromResource(HiveTestUtil.class, schemaFileName); - generateParquetDataWithSchema(filePath, schema.toAvroSchema()); + generateParquetDataWithSchema(filePath, schema); HoodieWriteStat writeStat = new HoodieWriteStat(); writeStat.setFileId(fileId); writeStat.setPath(filePath.toString()); @@ -644,7 +643,7 @@ private static void generateParquetData(Path filePath, boolean isParquetSchemaSi org.apache.parquet.schema.MessageType parquetSchema = new AvroSchemaConverter().convert(schema.toAvroSchema()); BloomFilter filter = BloomFilterFactory.createBloomFilter(1000, 0.0001, -1, BloomFilterTypeCode.SIMPLE.name()); - HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(parquetSchema, schema.toAvroSchema(), Option.of(filter), new Properties()); + HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(parquetSchema, schema, Option.of(filter), new Properties()); ParquetWriter writer = new ParquetWriter(filePath, writeSupport, CompressionCodecName.GZIP, 120 * 1024 * 1024, ParquetWriter.DEFAULT_PAGE_SIZE, ParquetWriter.DEFAULT_PAGE_SIZE, ParquetWriter.DEFAULT_IS_DICTIONARY_ENABLED, ParquetWriter.DEFAULT_IS_VALIDATING_ENABLED, ParquetWriter.DEFAULT_WRITER_VERSION, fileSystem.getConf()); @@ -667,7 +666,7 @@ private static void generateParquetData(Path filePath, String schemaPath, String org.apache.parquet.schema.MessageType parquetSchema = new AvroSchemaConverter().convert(schema.toAvroSchema()); BloomFilter filter = BloomFilterFactory.createBloomFilter(1000, 0.0001, -1, BloomFilterTypeCode.SIMPLE.name()); - HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(parquetSchema, schema.toAvroSchema(), Option.of(filter), new Properties()); + HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(parquetSchema, schema, Option.of(filter), new Properties()); ParquetWriter writer = new ParquetWriter(filePath, writeSupport, CompressionCodecName.GZIP, 120 * 1024 * 1024, ParquetWriter.DEFAULT_PAGE_SIZE, ParquetWriter.DEFAULT_PAGE_SIZE, ParquetWriter.DEFAULT_IS_DICTIONARY_ENABLED, ParquetWriter.DEFAULT_IS_VALIDATING_ENABLED, ParquetWriter.DEFAULT_WRITER_VERSION, fileSystem.getConf()); @@ -683,9 +682,9 @@ private static void generateParquetData(Path filePath, String schemaPath, String writer.close(); } - private static void generateParquetDataWithSchema(Path filePath, Schema schema) + private static void generateParquetDataWithSchema(Path filePath, HoodieSchema schema) throws IOException { - org.apache.parquet.schema.MessageType parquetSchema = new AvroSchemaConverter().convert(schema); + org.apache.parquet.schema.MessageType parquetSchema = new AvroSchemaConverter().convert(schema.toAvroSchema()); BloomFilter filter = BloomFilterFactory.createBloomFilter(1000, 0.0001, -1, BloomFilterTypeCode.SIMPLE.name()); HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(parquetSchema, schema, Option.of(filter), new Properties()); @@ -693,7 +692,7 @@ private static void generateParquetDataWithSchema(Path filePath, Schema schema) ParquetWriter.DEFAULT_PAGE_SIZE, ParquetWriter.DEFAULT_PAGE_SIZE, ParquetWriter.DEFAULT_IS_DICTIONARY_ENABLED, ParquetWriter.DEFAULT_IS_VALIDATING_ENABLED, ParquetWriter.DEFAULT_WRITER_VERSION, fileSystem.getConf()); - List testRecords = SchemaTestUtil.generateTestRecordsForSchema(HoodieSchema.fromAvroSchema(schema)); + List testRecords = SchemaTestUtil.generateTestRecordsForSchema(schema); testRecords.forEach(s -> { try { writer.write(s); diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/UtilitiesTestBase.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/UtilitiesTestBase.java index 5903698565dc0..a29090a1a7b12 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/UtilitiesTestBase.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/UtilitiesTestBase.java @@ -22,6 +22,8 @@ import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.schema.HoodieSchema; +import org.apache.hudi.common.schema.HoodieSchemaField; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.testutils.HoodieTestUtils; @@ -36,6 +38,7 @@ import org.apache.hudi.hive.ddl.JDBCExecutor; import org.apache.hudi.hive.ddl.QueryBasedDDLExecutor; import org.apache.hudi.hive.testutils.HiveTestService; +import org.apache.hudi.internal.schema.HoodieSchemaException; import org.apache.hudi.storage.HoodieStorage; import org.apache.hudi.storage.StoragePath; import org.apache.hudi.storage.HoodieStorageUtils; @@ -531,8 +534,9 @@ private static void addAvroRecord( final TypeDescription type = orcSchema.getChildren().get(c); Object fieldValue = record.get(thisField); - Schema.Field avroField = record.getSchema().getField(thisField); - AvroOrcUtils.addToVector(type, colVector, avroField.schema(), fieldValue, batch.size); + HoodieSchemaField field = HoodieSchema.fromAvroSchema(record.getSchema()).getField(thisField) + .orElseThrow(() -> new HoodieSchemaException("Could not find field: " + thisField)); + AvroOrcUtils.addToVector(type, colVector, field.schema(), fieldValue, batch.size); } } }