jonvex commented on code in PR #10263: URL: https://github.com/apache/hudi/pull/10263#discussion_r1420893953
########## hudi-common/src/test/java/org/apache/hudi/common/testutils/reader/DataGenerationPlan.java: ########## @@ -0,0 +1,122 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.common.testutils.reader; + +import java.util.List; + +/** + * The blueprint of records that will be generated + * by the data generator. + */ +public class DataGenerationPlan { + // The values for "_row_key" field. + private final List<String> recordKeys; + // The partition path for all records. + private final String partitionPath; + // The ordering field. + private final long timestamp; + // The operation type of the record. + private final OperationType operationType; + private final String instantTime; + + public enum OperationType { + INSERT, + UPDATE, + DELETE + } + + public DataGenerationPlan(List<String> recordKeys, + String partitionPath, Review Comment: We seem to be limited to only a single partition path? What if we want multiple partitions in a write? ########## hudi-common/src/test/java/org/apache/hudi/common/testutils/reader/HoodieFileSliceTestUtils.java: ########## @@ -0,0 +1,437 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.common.testutils.reader; + +import org.apache.hudi.avro.HoodieAvroWriteSupport; +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.config.TypedProperties; +import org.apache.hudi.common.engine.LocalTaskContextSupplier; +import org.apache.hudi.common.model.DeleteRecord; +import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.HoodieAvroIndexedRecord; +import org.apache.hudi.common.model.HoodieBaseFile; +import org.apache.hudi.common.model.HoodieFileGroupId; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieLogFile; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.table.log.HoodieLogFormat; +import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock; +import org.apache.hudi.common.table.log.block.HoodieCDCDataBlock; +import org.apache.hudi.common.table.log.block.HoodieDataBlock; +import org.apache.hudi.common.table.log.block.HoodieDeleteBlock; +import org.apache.hudi.common.table.log.block.HoodieHFileDataBlock; +import org.apache.hudi.common.table.log.block.HoodieLogBlock; +import org.apache.hudi.common.table.log.block.HoodieParquetDataBlock; +import org.apache.hudi.common.testutils.HoodieTestDataGenerator; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.io.storage.HoodieAvroParquetWriter; +import org.apache.hudi.io.storage.HoodieParquetConfig; + +import org.apache.avro.Schema; +import org.apache.avro.generic.IndexedRecord; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.io.compress.Compression; +import org.apache.parquet.avro.AvroSchemaConverter; +import org.apache.parquet.hadoop.ParquetWriter; +import org.apache.parquet.hadoop.metadata.CompressionCodecName; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.UUID; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import static org.apache.hudi.common.table.log.block.HoodieLogBlock.HoodieLogBlockType.DELETE_BLOCK; +import static org.apache.hudi.common.table.log.block.HoodieLogBlock.HoodieLogBlockType.PARQUET_DATA_BLOCK; +import static org.apache.hudi.common.testutils.FileCreateUtils.baseFileName; +import static org.apache.hudi.common.testutils.FileCreateUtils.logFileName; +import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.AVRO_SCHEMA; +import static org.apache.hudi.common.testutils.reader.DataGenerationPlan.OperationType.DELETE; +import static org.apache.hudi.common.testutils.reader.DataGenerationPlan.OperationType.INSERT; + +public class HoodieFileSliceTestUtils { + public static final String FORWARD_SLASH = "/"; + public static final String PERIOD = "."; + public static final String PARQUET = ".parquet"; + public static final String ROW_KEY = "_row_key"; + public static final String PARTITION_PATH = "partition_path"; + public static final String RIDER = "rider"; + public static final String DRIVER = "driver"; + public static final String TIMESTAMP = "timestamp"; + public static final HoodieTestDataGenerator DATA_GEN = + new HoodieTestDataGenerator(0xDEED); + + // We use a number to represent a record key, and a (start, end) range + // to represent a set of record keys between start <= k <= end. + public static class KeyRange { + public int start; + public int end; + + public KeyRange(int start, int end) { + this.start = start; + this.end = end; + } + } + + private static Path generateBaseFilePath( + String basePath, + String fileId, + String instantTime + ) { + return new Path( + basePath + FORWARD_SLASH + + baseFileName(instantTime, fileId, PARQUET)); + } + + private static Path generateLogFilePath( + String basePath, + String fileId, + String instantTime, + int version) { + return new Path( + basePath + FORWARD_SLASH + logFileName( + instantTime, fileId, version)); + } + + // Note: + // "start < end" means start <= k <= end. + // "start == end" means k = start. + // "start > end" means no keys. + private static List<String> generateKeys(KeyRange range) { + List<String> keys = new ArrayList<>(); + if (range.start == range.end) { + keys.add(String.valueOf(range.start)); + } else { + keys = IntStream + .rangeClosed(range.start, range.end) + .boxed() + .map(String::valueOf).collect(Collectors.toList()); + } + return keys; + } + + private static List<IndexedRecord> generateRecords(DataGenerationPlan plan) { + List<IndexedRecord> records = new ArrayList<>(); + List<String> keys = plan.getRecordKeys(); + for (String key : keys) { + records.add(DATA_GEN.generateGenericRecord( Review Comment: Can we add this logic into the datagenerator itself? ########## hudi-common/src/test/java/org/apache/hudi/common/testutils/reader/HoodieFileSliceTestUtils.java: ########## @@ -0,0 +1,437 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.common.testutils.reader; + +import org.apache.hudi.avro.HoodieAvroWriteSupport; +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.config.TypedProperties; +import org.apache.hudi.common.engine.LocalTaskContextSupplier; +import org.apache.hudi.common.model.DeleteRecord; +import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.HoodieAvroIndexedRecord; +import org.apache.hudi.common.model.HoodieBaseFile; +import org.apache.hudi.common.model.HoodieFileGroupId; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieLogFile; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.table.log.HoodieLogFormat; +import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock; +import org.apache.hudi.common.table.log.block.HoodieCDCDataBlock; +import org.apache.hudi.common.table.log.block.HoodieDataBlock; +import org.apache.hudi.common.table.log.block.HoodieDeleteBlock; +import org.apache.hudi.common.table.log.block.HoodieHFileDataBlock; +import org.apache.hudi.common.table.log.block.HoodieLogBlock; +import org.apache.hudi.common.table.log.block.HoodieParquetDataBlock; +import org.apache.hudi.common.testutils.HoodieTestDataGenerator; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.io.storage.HoodieAvroParquetWriter; +import org.apache.hudi.io.storage.HoodieParquetConfig; + +import org.apache.avro.Schema; +import org.apache.avro.generic.IndexedRecord; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.io.compress.Compression; +import org.apache.parquet.avro.AvroSchemaConverter; +import org.apache.parquet.hadoop.ParquetWriter; +import org.apache.parquet.hadoop.metadata.CompressionCodecName; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.UUID; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import static org.apache.hudi.common.table.log.block.HoodieLogBlock.HoodieLogBlockType.DELETE_BLOCK; +import static org.apache.hudi.common.table.log.block.HoodieLogBlock.HoodieLogBlockType.PARQUET_DATA_BLOCK; +import static org.apache.hudi.common.testutils.FileCreateUtils.baseFileName; +import static org.apache.hudi.common.testutils.FileCreateUtils.logFileName; +import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.AVRO_SCHEMA; +import static org.apache.hudi.common.testutils.reader.DataGenerationPlan.OperationType.DELETE; +import static org.apache.hudi.common.testutils.reader.DataGenerationPlan.OperationType.INSERT; + +public class HoodieFileSliceTestUtils { + public static final String FORWARD_SLASH = "/"; + public static final String PERIOD = "."; Review Comment: Not a fan of this. What is your reasoning for needing this? ########## hudi-common/src/test/java/org/apache/hudi/common/testutils/reader/HoodieFileSliceTestUtils.java: ########## @@ -0,0 +1,437 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.common.testutils.reader; + +import org.apache.hudi.avro.HoodieAvroWriteSupport; +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.config.TypedProperties; +import org.apache.hudi.common.engine.LocalTaskContextSupplier; +import org.apache.hudi.common.model.DeleteRecord; +import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.HoodieAvroIndexedRecord; +import org.apache.hudi.common.model.HoodieBaseFile; +import org.apache.hudi.common.model.HoodieFileGroupId; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieLogFile; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.table.log.HoodieLogFormat; +import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock; +import org.apache.hudi.common.table.log.block.HoodieCDCDataBlock; +import org.apache.hudi.common.table.log.block.HoodieDataBlock; +import org.apache.hudi.common.table.log.block.HoodieDeleteBlock; +import org.apache.hudi.common.table.log.block.HoodieHFileDataBlock; +import org.apache.hudi.common.table.log.block.HoodieLogBlock; +import org.apache.hudi.common.table.log.block.HoodieParquetDataBlock; +import org.apache.hudi.common.testutils.HoodieTestDataGenerator; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.io.storage.HoodieAvroParquetWriter; +import org.apache.hudi.io.storage.HoodieParquetConfig; + +import org.apache.avro.Schema; +import org.apache.avro.generic.IndexedRecord; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.io.compress.Compression; +import org.apache.parquet.avro.AvroSchemaConverter; +import org.apache.parquet.hadoop.ParquetWriter; +import org.apache.parquet.hadoop.metadata.CompressionCodecName; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.UUID; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import static org.apache.hudi.common.table.log.block.HoodieLogBlock.HoodieLogBlockType.DELETE_BLOCK; +import static org.apache.hudi.common.table.log.block.HoodieLogBlock.HoodieLogBlockType.PARQUET_DATA_BLOCK; +import static org.apache.hudi.common.testutils.FileCreateUtils.baseFileName; +import static org.apache.hudi.common.testutils.FileCreateUtils.logFileName; +import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.AVRO_SCHEMA; +import static org.apache.hudi.common.testutils.reader.DataGenerationPlan.OperationType.DELETE; +import static org.apache.hudi.common.testutils.reader.DataGenerationPlan.OperationType.INSERT; + +public class HoodieFileSliceTestUtils { + public static final String FORWARD_SLASH = "/"; + public static final String PERIOD = "."; + public static final String PARQUET = ".parquet"; + public static final String ROW_KEY = "_row_key"; + public static final String PARTITION_PATH = "partition_path"; + public static final String RIDER = "rider"; + public static final String DRIVER = "driver"; + public static final String TIMESTAMP = "timestamp"; + public static final HoodieTestDataGenerator DATA_GEN = + new HoodieTestDataGenerator(0xDEED); + + // We use a number to represent a record key, and a (start, end) range + // to represent a set of record keys between start <= k <= end. + public static class KeyRange { + public int start; + public int end; + + public KeyRange(int start, int end) { + this.start = start; + this.end = end; + } + } + + private static Path generateBaseFilePath( + String basePath, + String fileId, + String instantTime + ) { + return new Path( + basePath + FORWARD_SLASH + + baseFileName(instantTime, fileId, PARQUET)); + } + + private static Path generateLogFilePath( + String basePath, + String fileId, + String instantTime, + int version) { + return new Path( + basePath + FORWARD_SLASH + logFileName( + instantTime, fileId, version)); + } + + // Note: + // "start < end" means start <= k <= end. + // "start == end" means k = start. + // "start > end" means no keys. + private static List<String> generateKeys(KeyRange range) { + List<String> keys = new ArrayList<>(); + if (range.start == range.end) { + keys.add(String.valueOf(range.start)); + } else { + keys = IntStream + .rangeClosed(range.start, range.end) + .boxed() + .map(String::valueOf).collect(Collectors.toList()); + } + return keys; + } + + private static List<IndexedRecord> generateRecords(DataGenerationPlan plan) { + List<IndexedRecord> records = new ArrayList<>(); + List<String> keys = plan.getRecordKeys(); + for (String key : keys) { + records.add(DATA_GEN.generateGenericRecord( + key, + plan.getPartitionPath(), + RIDER + PERIOD + UUID.randomUUID().toString(), + DRIVER + PERIOD + UUID.randomUUID().toString(), + plan.getTimestamp(), + plan.getOperationType() == DELETE, + false + )); + } + return records; + } + + private static HoodieDataBlock getDataBlock( + HoodieLogBlock.HoodieLogBlockType dataBlockType, + List<IndexedRecord> records, + Map<HoodieLogBlock.HeaderMetadataType, String> header, + Path logFilePath + ) { + return createDataBlock( + dataBlockType, + records.stream().map(HoodieAvroIndexedRecord::new) + .collect(Collectors.toList()), + header, + logFilePath); + } + + private static HoodieDataBlock createDataBlock( + HoodieLogBlock.HoodieLogBlockType dataBlockType, + List<HoodieRecord> records, + Map<HoodieLogBlock.HeaderMetadataType, String> header, + Path pathForReader + ) { + switch (dataBlockType) { + case CDC_DATA_BLOCK: + return new HoodieCDCDataBlock( + records, + header, + HoodieRecord.RECORD_KEY_METADATA_FIELD); + case AVRO_DATA_BLOCK: + return new HoodieAvroDataBlock( + records, + false, + header, + HoodieRecord.RECORD_KEY_METADATA_FIELD); + case HFILE_DATA_BLOCK: + return new HoodieHFileDataBlock( + records, + header, + Compression.Algorithm.GZ, + pathForReader); + case PARQUET_DATA_BLOCK: + return new HoodieParquetDataBlock( + records, + false, + header, + HoodieRecord.RECORD_KEY_METADATA_FIELD, + CompressionCodecName.GZIP, + 0.1, + true); + default: + throw new RuntimeException( + "Unknown data block type " + dataBlockType); + } + } + + public static HoodieDeleteBlock getDeleteBlock( + List<IndexedRecord> records, + Map<HoodieLogBlock.HeaderMetadataType, String> header, + Schema schema, + Properties props + ) { + List<HoodieRecord> hoodieRecords = records.stream() + .map(r -> { + String rowKey = (String) r.get(r.getSchema().getField(ROW_KEY).pos()); + String partitionPath = (String) r.get(r.getSchema().getField(PARTITION_PATH).pos()); + return new HoodieAvroIndexedRecord(new HoodieKey(rowKey, partitionPath), r); + }) + .collect(Collectors.toList()); + return new HoodieDeleteBlock( + hoodieRecords.stream().map( + r -> Pair.of(DeleteRecord.create( + r.getKey(), r.getOrderingValue(schema, props)), -1L)) + .collect(Collectors.toList()), + false, + header + ); + } + + public static HoodieBaseFile createBaseFile( + String baseFilePath, + List<IndexedRecord> records, + Schema schema, + String baseInstantTime + ) throws IOException { + Configuration hadoopConf = new Configuration(); + + BloomFilter filter = BloomFilterFactory.createBloomFilter( + 1000, + 0.0001, + 10000, + BloomFilterTypeCode.DYNAMIC_V0.name()); + HoodieAvroWriteSupport<IndexedRecord> writeSupport = new HoodieAvroWriteSupport<>( + new AvroSchemaConverter().convert(schema), + schema, + Option.of(filter), + new Properties()); + HoodieParquetConfig<HoodieAvroWriteSupport> parquetConfig = new HoodieParquetConfig( + writeSupport, + CompressionCodecName.GZIP, + ParquetWriter.DEFAULT_BLOCK_SIZE, + ParquetWriter.DEFAULT_PAGE_SIZE, + 1024 * 1024 * 1024, + hadoopConf, + 0.1, + true); + Review Comment: Can you explain what we are doing here? Lots of hard coded numbers ########## hudi-common/src/test/java/org/apache/hudi/common/testutils/reader/HoodieFileSliceTestUtils.java: ########## @@ -0,0 +1,437 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.common.testutils.reader; + +import org.apache.hudi.avro.HoodieAvroWriteSupport; +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.config.TypedProperties; +import org.apache.hudi.common.engine.LocalTaskContextSupplier; +import org.apache.hudi.common.model.DeleteRecord; +import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.HoodieAvroIndexedRecord; +import org.apache.hudi.common.model.HoodieBaseFile; +import org.apache.hudi.common.model.HoodieFileGroupId; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieLogFile; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.table.log.HoodieLogFormat; +import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock; +import org.apache.hudi.common.table.log.block.HoodieCDCDataBlock; +import org.apache.hudi.common.table.log.block.HoodieDataBlock; +import org.apache.hudi.common.table.log.block.HoodieDeleteBlock; +import org.apache.hudi.common.table.log.block.HoodieHFileDataBlock; +import org.apache.hudi.common.table.log.block.HoodieLogBlock; +import org.apache.hudi.common.table.log.block.HoodieParquetDataBlock; +import org.apache.hudi.common.testutils.HoodieTestDataGenerator; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.io.storage.HoodieAvroParquetWriter; +import org.apache.hudi.io.storage.HoodieParquetConfig; + +import org.apache.avro.Schema; +import org.apache.avro.generic.IndexedRecord; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.io.compress.Compression; +import org.apache.parquet.avro.AvroSchemaConverter; +import org.apache.parquet.hadoop.ParquetWriter; +import org.apache.parquet.hadoop.metadata.CompressionCodecName; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.UUID; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import static org.apache.hudi.common.table.log.block.HoodieLogBlock.HoodieLogBlockType.DELETE_BLOCK; +import static org.apache.hudi.common.table.log.block.HoodieLogBlock.HoodieLogBlockType.PARQUET_DATA_BLOCK; +import static org.apache.hudi.common.testutils.FileCreateUtils.baseFileName; +import static org.apache.hudi.common.testutils.FileCreateUtils.logFileName; +import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.AVRO_SCHEMA; +import static org.apache.hudi.common.testutils.reader.DataGenerationPlan.OperationType.DELETE; +import static org.apache.hudi.common.testutils.reader.DataGenerationPlan.OperationType.INSERT; + +public class HoodieFileSliceTestUtils { + public static final String FORWARD_SLASH = "/"; + public static final String PERIOD = "."; + public static final String PARQUET = ".parquet"; + public static final String ROW_KEY = "_row_key"; + public static final String PARTITION_PATH = "partition_path"; + public static final String RIDER = "rider"; + public static final String DRIVER = "driver"; + public static final String TIMESTAMP = "timestamp"; + public static final HoodieTestDataGenerator DATA_GEN = + new HoodieTestDataGenerator(0xDEED); + + // We use a number to represent a record key, and a (start, end) range + // to represent a set of record keys between start <= k <= end. + public static class KeyRange { + public int start; + public int end; + + public KeyRange(int start, int end) { + this.start = start; + this.end = end; + } + } + + private static Path generateBaseFilePath( + String basePath, + String fileId, + String instantTime + ) { + return new Path( + basePath + FORWARD_SLASH + + baseFileName(instantTime, fileId, PARQUET)); + } + + private static Path generateLogFilePath( + String basePath, + String fileId, + String instantTime, + int version) { + return new Path( + basePath + FORWARD_SLASH + logFileName( + instantTime, fileId, version)); + } + + // Note: + // "start < end" means start <= k <= end. + // "start == end" means k = start. + // "start > end" means no keys. + private static List<String> generateKeys(KeyRange range) { + List<String> keys = new ArrayList<>(); + if (range.start == range.end) { + keys.add(String.valueOf(range.start)); + } else { + keys = IntStream + .rangeClosed(range.start, range.end) + .boxed() + .map(String::valueOf).collect(Collectors.toList()); + } + return keys; + } + + private static List<IndexedRecord> generateRecords(DataGenerationPlan plan) { + List<IndexedRecord> records = new ArrayList<>(); + List<String> keys = plan.getRecordKeys(); + for (String key : keys) { + records.add(DATA_GEN.generateGenericRecord( + key, + plan.getPartitionPath(), + RIDER + PERIOD + UUID.randomUUID().toString(), + DRIVER + PERIOD + UUID.randomUUID().toString(), + plan.getTimestamp(), + plan.getOperationType() == DELETE, + false + )); + } + return records; + } + + private static HoodieDataBlock getDataBlock( + HoodieLogBlock.HoodieLogBlockType dataBlockType, + List<IndexedRecord> records, + Map<HoodieLogBlock.HeaderMetadataType, String> header, + Path logFilePath + ) { + return createDataBlock( + dataBlockType, + records.stream().map(HoodieAvroIndexedRecord::new) + .collect(Collectors.toList()), + header, + logFilePath); + } + + private static HoodieDataBlock createDataBlock( + HoodieLogBlock.HoodieLogBlockType dataBlockType, + List<HoodieRecord> records, + Map<HoodieLogBlock.HeaderMetadataType, String> header, + Path pathForReader + ) { + switch (dataBlockType) { + case CDC_DATA_BLOCK: + return new HoodieCDCDataBlock( + records, + header, + HoodieRecord.RECORD_KEY_METADATA_FIELD); + case AVRO_DATA_BLOCK: + return new HoodieAvroDataBlock( + records, + false, + header, + HoodieRecord.RECORD_KEY_METADATA_FIELD); + case HFILE_DATA_BLOCK: + return new HoodieHFileDataBlock( + records, + header, + Compression.Algorithm.GZ, + pathForReader); + case PARQUET_DATA_BLOCK: + return new HoodieParquetDataBlock( + records, + false, + header, + HoodieRecord.RECORD_KEY_METADATA_FIELD, + CompressionCodecName.GZIP, + 0.1, + true); + default: + throw new RuntimeException( + "Unknown data block type " + dataBlockType); + } + } + + public static HoodieDeleteBlock getDeleteBlock( + List<IndexedRecord> records, + Map<HoodieLogBlock.HeaderMetadataType, String> header, + Schema schema, + Properties props + ) { + List<HoodieRecord> hoodieRecords = records.stream() + .map(r -> { + String rowKey = (String) r.get(r.getSchema().getField(ROW_KEY).pos()); + String partitionPath = (String) r.get(r.getSchema().getField(PARTITION_PATH).pos()); + return new HoodieAvroIndexedRecord(new HoodieKey(rowKey, partitionPath), r); + }) + .collect(Collectors.toList()); + return new HoodieDeleteBlock( + hoodieRecords.stream().map( + r -> Pair.of(DeleteRecord.create( + r.getKey(), r.getOrderingValue(schema, props)), -1L)) + .collect(Collectors.toList()), + false, + header + ); + } + + public static HoodieBaseFile createBaseFile( + String baseFilePath, + List<IndexedRecord> records, + Schema schema, + String baseInstantTime + ) throws IOException { + Configuration hadoopConf = new Configuration(); + + BloomFilter filter = BloomFilterFactory.createBloomFilter( + 1000, + 0.0001, + 10000, + BloomFilterTypeCode.DYNAMIC_V0.name()); + HoodieAvroWriteSupport<IndexedRecord> writeSupport = new HoodieAvroWriteSupport<>( + new AvroSchemaConverter().convert(schema), + schema, + Option.of(filter), + new Properties()); + HoodieParquetConfig<HoodieAvroWriteSupport> parquetConfig = new HoodieParquetConfig( + writeSupport, + CompressionCodecName.GZIP, + ParquetWriter.DEFAULT_BLOCK_SIZE, + ParquetWriter.DEFAULT_PAGE_SIZE, + 1024 * 1024 * 1024, + hadoopConf, + 0.1, + true); + + try (HoodieAvroParquetWriter writer = new HoodieAvroParquetWriter( + new Path(baseFilePath), + parquetConfig, + baseInstantTime, + new LocalTaskContextSupplier(), + true)) { + for (IndexedRecord record : records) { + writer.writeAvro( + (String) record.get(schema.getField(ROW_KEY).pos()), record); + } + } + return new HoodieBaseFile(baseFilePath); + } + + public static HoodieLogFile createLogFile( + FileSystem fileSystem, + String logFilePath, + List<IndexedRecord> records, + Schema schema, + String fileId, + String logInstantTime, + int version, + HoodieLogBlock.HoodieLogBlockType blockType + ) throws InterruptedException, IOException { + try (HoodieLogFormat.Writer writer = + HoodieLogFormat.newWriterBuilder() + .onParentPath(new Path(logFilePath).getParent()) + .withFileExtension(HoodieLogFile.DELTA_EXTENSION) + .withFileId(fileId) + .withDeltaCommit(logInstantTime) + .withLogVersion(version) + .withFs(fileSystem).build()) { + Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>(); + header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, logInstantTime); + header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); + // TODO: use global properties instead. + TypedProperties properties = new TypedProperties(); + properties.setProperty( + "hoodie.datasource.write.precombine.field", "timestamp"); Review Comment: should use static TIMESTAMP var ########## hudi-common/src/test/java/org/apache/hudi/common/testutils/reader/HoodieFileSliceTestUtils.java: ########## @@ -0,0 +1,437 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.common.testutils.reader; + +import org.apache.hudi.avro.HoodieAvroWriteSupport; +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.config.TypedProperties; +import org.apache.hudi.common.engine.LocalTaskContextSupplier; +import org.apache.hudi.common.model.DeleteRecord; +import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.HoodieAvroIndexedRecord; +import org.apache.hudi.common.model.HoodieBaseFile; +import org.apache.hudi.common.model.HoodieFileGroupId; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieLogFile; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.table.log.HoodieLogFormat; +import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock; +import org.apache.hudi.common.table.log.block.HoodieCDCDataBlock; +import org.apache.hudi.common.table.log.block.HoodieDataBlock; +import org.apache.hudi.common.table.log.block.HoodieDeleteBlock; +import org.apache.hudi.common.table.log.block.HoodieHFileDataBlock; +import org.apache.hudi.common.table.log.block.HoodieLogBlock; +import org.apache.hudi.common.table.log.block.HoodieParquetDataBlock; +import org.apache.hudi.common.testutils.HoodieTestDataGenerator; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.io.storage.HoodieAvroParquetWriter; +import org.apache.hudi.io.storage.HoodieParquetConfig; + +import org.apache.avro.Schema; +import org.apache.avro.generic.IndexedRecord; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.io.compress.Compression; +import org.apache.parquet.avro.AvroSchemaConverter; +import org.apache.parquet.hadoop.ParquetWriter; +import org.apache.parquet.hadoop.metadata.CompressionCodecName; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.UUID; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import static org.apache.hudi.common.table.log.block.HoodieLogBlock.HoodieLogBlockType.DELETE_BLOCK; +import static org.apache.hudi.common.table.log.block.HoodieLogBlock.HoodieLogBlockType.PARQUET_DATA_BLOCK; +import static org.apache.hudi.common.testutils.FileCreateUtils.baseFileName; +import static org.apache.hudi.common.testutils.FileCreateUtils.logFileName; +import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.AVRO_SCHEMA; +import static org.apache.hudi.common.testutils.reader.DataGenerationPlan.OperationType.DELETE; +import static org.apache.hudi.common.testutils.reader.DataGenerationPlan.OperationType.INSERT; + +public class HoodieFileSliceTestUtils { + public static final String FORWARD_SLASH = "/"; + public static final String PERIOD = "."; + public static final String PARQUET = ".parquet"; + public static final String ROW_KEY = "_row_key"; + public static final String PARTITION_PATH = "partition_path"; + public static final String RIDER = "rider"; + public static final String DRIVER = "driver"; + public static final String TIMESTAMP = "timestamp"; + public static final HoodieTestDataGenerator DATA_GEN = + new HoodieTestDataGenerator(0xDEED); + + // We use a number to represent a record key, and a (start, end) range + // to represent a set of record keys between start <= k <= end. + public static class KeyRange { + public int start; + public int end; + + public KeyRange(int start, int end) { + this.start = start; + this.end = end; + } + } + + private static Path generateBaseFilePath( + String basePath, + String fileId, + String instantTime + ) { + return new Path( + basePath + FORWARD_SLASH + + baseFileName(instantTime, fileId, PARQUET)); + } + + private static Path generateLogFilePath( + String basePath, + String fileId, + String instantTime, + int version) { + return new Path( + basePath + FORWARD_SLASH + logFileName( + instantTime, fileId, version)); + } + + // Note: + // "start < end" means start <= k <= end. + // "start == end" means k = start. + // "start > end" means no keys. + private static List<String> generateKeys(KeyRange range) { + List<String> keys = new ArrayList<>(); + if (range.start == range.end) { + keys.add(String.valueOf(range.start)); + } else { + keys = IntStream + .rangeClosed(range.start, range.end) + .boxed() + .map(String::valueOf).collect(Collectors.toList()); + } + return keys; + } + + private static List<IndexedRecord> generateRecords(DataGenerationPlan plan) { + List<IndexedRecord> records = new ArrayList<>(); + List<String> keys = plan.getRecordKeys(); + for (String key : keys) { + records.add(DATA_GEN.generateGenericRecord( + key, + plan.getPartitionPath(), + RIDER + PERIOD + UUID.randomUUID().toString(), + DRIVER + PERIOD + UUID.randomUUID().toString(), + plan.getTimestamp(), + plan.getOperationType() == DELETE, + false + )); + } + return records; + } + + private static HoodieDataBlock getDataBlock( + HoodieLogBlock.HoodieLogBlockType dataBlockType, + List<IndexedRecord> records, + Map<HoodieLogBlock.HeaderMetadataType, String> header, + Path logFilePath + ) { + return createDataBlock( + dataBlockType, + records.stream().map(HoodieAvroIndexedRecord::new) + .collect(Collectors.toList()), + header, + logFilePath); + } + + private static HoodieDataBlock createDataBlock( + HoodieLogBlock.HoodieLogBlockType dataBlockType, + List<HoodieRecord> records, + Map<HoodieLogBlock.HeaderMetadataType, String> header, + Path pathForReader + ) { + switch (dataBlockType) { + case CDC_DATA_BLOCK: + return new HoodieCDCDataBlock( + records, + header, + HoodieRecord.RECORD_KEY_METADATA_FIELD); + case AVRO_DATA_BLOCK: + return new HoodieAvroDataBlock( + records, + false, + header, + HoodieRecord.RECORD_KEY_METADATA_FIELD); + case HFILE_DATA_BLOCK: + return new HoodieHFileDataBlock( + records, + header, + Compression.Algorithm.GZ, + pathForReader); + case PARQUET_DATA_BLOCK: + return new HoodieParquetDataBlock( + records, + false, + header, + HoodieRecord.RECORD_KEY_METADATA_FIELD, + CompressionCodecName.GZIP, + 0.1, + true); + default: + throw new RuntimeException( + "Unknown data block type " + dataBlockType); + } + } + + public static HoodieDeleteBlock getDeleteBlock( + List<IndexedRecord> records, + Map<HoodieLogBlock.HeaderMetadataType, String> header, + Schema schema, + Properties props + ) { + List<HoodieRecord> hoodieRecords = records.stream() + .map(r -> { + String rowKey = (String) r.get(r.getSchema().getField(ROW_KEY).pos()); + String partitionPath = (String) r.get(r.getSchema().getField(PARTITION_PATH).pos()); + return new HoodieAvroIndexedRecord(new HoodieKey(rowKey, partitionPath), r); + }) + .collect(Collectors.toList()); + return new HoodieDeleteBlock( + hoodieRecords.stream().map( + r -> Pair.of(DeleteRecord.create( + r.getKey(), r.getOrderingValue(schema, props)), -1L)) + .collect(Collectors.toList()), + false, + header + ); + } + + public static HoodieBaseFile createBaseFile( + String baseFilePath, + List<IndexedRecord> records, + Schema schema, + String baseInstantTime + ) throws IOException { + Configuration hadoopConf = new Configuration(); + + BloomFilter filter = BloomFilterFactory.createBloomFilter( + 1000, + 0.0001, + 10000, + BloomFilterTypeCode.DYNAMIC_V0.name()); + HoodieAvroWriteSupport<IndexedRecord> writeSupport = new HoodieAvroWriteSupport<>( + new AvroSchemaConverter().convert(schema), + schema, + Option.of(filter), + new Properties()); + HoodieParquetConfig<HoodieAvroWriteSupport> parquetConfig = new HoodieParquetConfig( + writeSupport, + CompressionCodecName.GZIP, + ParquetWriter.DEFAULT_BLOCK_SIZE, + ParquetWriter.DEFAULT_PAGE_SIZE, + 1024 * 1024 * 1024, + hadoopConf, + 0.1, + true); + + try (HoodieAvroParquetWriter writer = new HoodieAvroParquetWriter( + new Path(baseFilePath), + parquetConfig, + baseInstantTime, + new LocalTaskContextSupplier(), + true)) { + for (IndexedRecord record : records) { + writer.writeAvro( + (String) record.get(schema.getField(ROW_KEY).pos()), record); + } + } + return new HoodieBaseFile(baseFilePath); + } + + public static HoodieLogFile createLogFile( + FileSystem fileSystem, + String logFilePath, + List<IndexedRecord> records, + Schema schema, + String fileId, + String logInstantTime, + int version, + HoodieLogBlock.HoodieLogBlockType blockType + ) throws InterruptedException, IOException { + try (HoodieLogFormat.Writer writer = + HoodieLogFormat.newWriterBuilder() + .onParentPath(new Path(logFilePath).getParent()) + .withFileExtension(HoodieLogFile.DELTA_EXTENSION) + .withFileId(fileId) + .withDeltaCommit(logInstantTime) + .withLogVersion(version) + .withFs(fileSystem).build()) { + Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>(); + header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, logInstantTime); + header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); + // TODO: use global properties instead. Review Comment: do todo ########## hudi-common/src/test/java/org/apache/hudi/common/testutils/reader/DataGenerationPlan.java: ########## @@ -0,0 +1,122 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.common.testutils.reader; + +import java.util.List; + +/** + * The blueprint of records that will be generated + * by the data generator. + */ +public class DataGenerationPlan { + // The values for "_row_key" field. + private final List<String> recordKeys; Review Comment: Do we want to test complex keygen or other keygenerators? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
