[HUDI-2176, 2178, 2179] Adding virtual key support to COW table (#3306)
This commit is contained in:
committed by
GitHub
parent
5353243449
commit
61148c1c43
@@ -22,6 +22,7 @@ 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.model.HoodieRecord;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericData;
|
||||
@@ -52,7 +53,7 @@ public class TestHoodieAvroWriteSupport {
|
||||
1000, 0.0001, 10000,
|
||||
BloomFilterTypeCode.SIMPLE.name());
|
||||
HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(
|
||||
new AvroSchemaConverter().convert(schema), schema, filter);
|
||||
new AvroSchemaConverter().convert(schema), schema, Option.of(filter));
|
||||
ParquetWriter writer = new ParquetWriter(new Path(filePath), writeSupport, CompressionCodecName.GZIP,
|
||||
120 * 1024 * 1024, ParquetWriter.DEFAULT_PAGE_SIZE);
|
||||
for (String rowKey : rowKeys) {
|
||||
|
||||
@@ -90,6 +90,7 @@ public class HoodieTestDataGenerator {
|
||||
public static final int DEFAULT_PARTITION_DEPTH = 3;
|
||||
public static final String TRIP_SCHEMA_PREFIX = "{\"type\": \"record\"," + "\"name\": \"triprec\"," + "\"fields\": [ "
|
||||
+ "{\"name\": \"timestamp\",\"type\": \"long\"}," + "{\"name\": \"_row_key\", \"type\": \"string\"},"
|
||||
+ "{\"name\": \"partition_path\", \"type\": \"string\"},"
|
||||
+ "{\"name\": \"rider\", \"type\": \"string\"}," + "{\"name\": \"driver\", \"type\": \"string\"},"
|
||||
+ "{\"name\": \"begin_lat\", \"type\": \"double\"}," + "{\"name\": \"begin_lon\", \"type\": \"double\"},"
|
||||
+ "{\"name\": \"end_lat\", \"type\": \"double\"}," + "{\"name\": \"end_lon\", \"type\": \"double\"},";
|
||||
@@ -123,7 +124,7 @@ public class HoodieTestDataGenerator {
|
||||
+ "{\"name\":\"driver\",\"type\":\"string\"},{\"name\":\"fare\",\"type\":\"double\"},{\"name\": \"_hoodie_is_deleted\", \"type\": \"boolean\", \"default\": false}]}";
|
||||
|
||||
public static final String NULL_SCHEMA = Schema.create(Schema.Type.NULL).toString();
|
||||
public static final String TRIP_HIVE_COLUMN_TYPES = "bigint,string,string,string,double,double,double,double,int,bigint,float,binary,int,bigint,decimal(10,6),"
|
||||
public static final String TRIP_HIVE_COLUMN_TYPES = "bigint,string,string,string,string,double,double,double,double,int,bigint,float,binary,int,bigint,decimal(10,6),"
|
||||
+ "map<string,string>,struct<amount:double,currency:string>,array<struct<amount:double,currency:string>>,boolean";
|
||||
|
||||
|
||||
@@ -208,7 +209,7 @@ public class HoodieTestDataGenerator {
|
||||
public static RawTripTestPayload generateRandomValue(
|
||||
HoodieKey key, String instantTime, boolean isFlattened) throws IOException {
|
||||
GenericRecord rec = generateGenericRecord(
|
||||
key.getRecordKey(), "rider-" + instantTime, "driver-" + instantTime, 0,
|
||||
key.getRecordKey(), key.getPartitionPath(), "rider-" + instantTime, "driver-" + instantTime, 0,
|
||||
false, isFlattened);
|
||||
return new RawTripTestPayload(rec.toString(), key.getRecordKey(), key.getPartitionPath(), TRIP_EXAMPLE_SCHEMA);
|
||||
}
|
||||
@@ -230,7 +231,7 @@ public class HoodieTestDataGenerator {
|
||||
* Generates a new avro record of the above schema format for a delete.
|
||||
*/
|
||||
public static RawTripTestPayload generateRandomDeleteValue(HoodieKey key, String instantTime) throws IOException {
|
||||
GenericRecord rec = generateGenericRecord(key.getRecordKey(), "rider-" + instantTime, "driver-" + instantTime, 0,
|
||||
GenericRecord rec = generateGenericRecord(key.getRecordKey(), key.getPartitionPath(), "rider-" + instantTime, "driver-" + instantTime, 0,
|
||||
true, false);
|
||||
return new RawTripTestPayload(Option.of(rec.toString()), key.getRecordKey(), key.getPartitionPath(), TRIP_EXAMPLE_SCHEMA, true);
|
||||
}
|
||||
@@ -239,21 +240,22 @@ public class HoodieTestDataGenerator {
|
||||
* Generates a new avro record of the above schema format, retaining the key if optionally provided.
|
||||
*/
|
||||
public static HoodieAvroPayload generateAvroPayload(HoodieKey key, String instantTime) {
|
||||
GenericRecord rec = generateGenericRecord(key.getRecordKey(), "rider-" + instantTime, "driver-" + instantTime, 0);
|
||||
GenericRecord rec = generateGenericRecord(key.getRecordKey(), key.getPartitionPath(), "rider-" + instantTime, "driver-" + instantTime, 0);
|
||||
return new HoodieAvroPayload(Option.of(rec));
|
||||
}
|
||||
|
||||
public static GenericRecord generateGenericRecord(String rowKey, String riderName, String driverName,
|
||||
public static GenericRecord generateGenericRecord(String rowKey, String partitionPath, String riderName, String driverName,
|
||||
long timestamp) {
|
||||
return generateGenericRecord(rowKey, riderName, driverName, timestamp, false, false);
|
||||
return generateGenericRecord(rowKey, partitionPath, riderName, driverName, timestamp, false, false);
|
||||
}
|
||||
|
||||
public static GenericRecord generateGenericRecord(String rowKey, String riderName, String driverName,
|
||||
public static GenericRecord generateGenericRecord(String rowKey, String partitionPath, String riderName, String driverName,
|
||||
long timestamp, boolean isDeleteRecord,
|
||||
boolean isFlattened) {
|
||||
GenericRecord rec = new GenericData.Record(isFlattened ? FLATTENED_AVRO_SCHEMA : AVRO_SCHEMA);
|
||||
rec.put("_row_key", rowKey);
|
||||
rec.put("timestamp", timestamp);
|
||||
rec.put("partition_path", partitionPath);
|
||||
rec.put("rider", riderName);
|
||||
rec.put("driver", driverName);
|
||||
rec.put("begin_lat", RAND.nextDouble());
|
||||
@@ -807,7 +809,7 @@ public class HoodieTestDataGenerator {
|
||||
public List<GenericRecord> generateGenericRecords(int numRecords) {
|
||||
List<GenericRecord> list = new ArrayList<>();
|
||||
IntStream.range(0, numRecords).forEach(i -> {
|
||||
list.add(generateGenericRecord(UUID.randomUUID().toString(), UUID.randomUUID().toString(), UUID.randomUUID()
|
||||
list.add(generateGenericRecord(UUID.randomUUID().toString(), "0", UUID.randomUUID().toString(), UUID.randomUUID()
|
||||
.toString(), RAND.nextLong()));
|
||||
});
|
||||
return list;
|
||||
|
||||
@@ -40,7 +40,7 @@ public class TestAvroOrcUtils extends HoodieCommonTestHarness {
|
||||
// The following types are tested:
|
||||
// DATE, DECIMAL, LONG, INT, BYTES, ARRAY, RECORD, MAP, STRING, FLOAT, DOUBLE
|
||||
TypeDescription orcSchema = TypeDescription.fromString("struct<"
|
||||
+ "timestamp:bigint,_row_key:string,rider:string,driver:string,begin_lat:double,"
|
||||
+ "timestamp:bigint,_row_key:string,partition_path:string,rider:string,driver:string,begin_lat:double,"
|
||||
+ "begin_lon:double,end_lat:double,end_lon:double,"
|
||||
+ "distance_in_meters:int,seconds_since_epoch:bigint,weight:float,nation:binary,"
|
||||
+ "current_date:date,current_ts:bigint,height:decimal(10,6),"
|
||||
|
||||
@@ -23,11 +23,14 @@ 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.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.testutils.HoodieCommonTestHarness;
|
||||
import org.apache.hudi.common.testutils.HoodieTestUtils;
|
||||
import org.apache.hudi.keygen.BaseKeyGenerator;
|
||||
|
||||
import org.apache.avro.JsonProperties;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericData;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
@@ -50,6 +53,7 @@ import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.UUID;
|
||||
|
||||
import static org.apache.hudi.avro.HoodieAvroUtils.METADATA_FIELD_SCHEMA;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
@@ -151,6 +155,33 @@ public class TestParquetUtils extends HoodieCommonTestHarness {
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFetchRecordKeyPartitionPathVirtualKeysFromParquet() throws Exception {
|
||||
List<String> rowKeys = new ArrayList<>();
|
||||
List<HoodieKey> expected = new ArrayList<>();
|
||||
String partitionPath = "path1";
|
||||
for (int i = 0; i < 1000; i++) {
|
||||
String rowKey = UUID.randomUUID().toString();
|
||||
rowKeys.add(rowKey);
|
||||
expected.add(new HoodieKey(rowKey, partitionPath));
|
||||
}
|
||||
|
||||
String filePath = Paths.get(basePath, "test.parquet").toUri().toString();
|
||||
Schema schema = getSchemaWithFields(Arrays.asList(new String[]{"abc", "def"}));
|
||||
writeParquetFile(BloomFilterTypeCode.SIMPLE.name(), filePath, rowKeys, schema, true, partitionPath,
|
||||
false, "abc", "def");
|
||||
|
||||
// Read and verify
|
||||
List<HoodieKey> fetchedRows =
|
||||
parquetUtils.fetchRecordKeyPartitionPath(HoodieTestUtils.getDefaultHadoopConf(), new Path(filePath),
|
||||
Option.of(new TestBaseKeyGen("abc","def")));
|
||||
assertEquals(rowKeys.size(), fetchedRows.size(), "Total count does not match");
|
||||
|
||||
for (HoodieKey entry : fetchedRows) {
|
||||
assertTrue(expected.contains(entry), "Record key must be in the given filter");
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testReadCounts() throws Exception {
|
||||
String filePath = Paths.get(basePath, "test.parquet").toUri().toString();
|
||||
@@ -168,22 +199,73 @@ public class TestParquetUtils extends HoodieCommonTestHarness {
|
||||
}
|
||||
|
||||
private void writeParquetFile(String typeCode, String filePath, List<String> rowKeys, Schema 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<String> rowKeys, Schema 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, filter);
|
||||
new HoodieAvroWriteSupport(new AvroSchemaConverter().convert(schema), schema, Option.of(filter));
|
||||
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);
|
||||
rec.put(HoodieRecord.RECORD_KEY_METADATA_FIELD, rowKey);
|
||||
rec.put(useMetaFields ? HoodieRecord.RECORD_KEY_METADATA_FIELD : recordFieldName, rowKey);
|
||||
if (addPartitionPathField) {
|
||||
rec.put(HoodieRecord.PARTITION_PATH_METADATA_FIELD, partitionPath);
|
||||
rec.put(useMetaFields ? HoodieRecord.PARTITION_PATH_METADATA_FIELD : partitionFieldName, partitionPathValue);
|
||||
}
|
||||
writer.write(rec);
|
||||
writeSupport.add(rowKey);
|
||||
}
|
||||
writer.close();
|
||||
}
|
||||
|
||||
private static Schema getSchemaWithFields(List<String> fields) {
|
||||
List<Schema.Field> toBeAddedFields = new ArrayList<>();
|
||||
Schema recordSchema = Schema.createRecord("HoodieRecordKey", "", "", false);
|
||||
|
||||
for (String field: fields) {
|
||||
Schema.Field schemaField =
|
||||
new Schema.Field(field, METADATA_FIELD_SCHEMA, "", JsonProperties.NULL_VALUE);
|
||||
toBeAddedFields.add(schemaField);
|
||||
}
|
||||
recordSchema.setFields(toBeAddedFields);
|
||||
return recordSchema;
|
||||
}
|
||||
|
||||
class TestBaseKeyGen extends BaseKeyGenerator {
|
||||
|
||||
private String recordKeyField;
|
||||
private String partitionField;
|
||||
|
||||
public TestBaseKeyGen(String recordKeyField, String partitionField) {
|
||||
super(new TypedProperties());
|
||||
this.recordKeyField = recordKeyField;
|
||||
this.partitionField = partitionField;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getRecordKey(GenericRecord record) {
|
||||
return record.get(recordKeyField).toString();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getPartitionPath(GenericRecord record) {
|
||||
return record.get(partitionField).toString();
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<String> getRecordKeyFields() {
|
||||
return Arrays.asList(new String[]{recordKeyField});
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<String> getPartitionPathFields() {
|
||||
return Arrays.asList(new String[]{partitionField});
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user