[HUDI-76] Add CSV Source support for Hudi Delta Streamer
This commit is contained in:
@@ -74,20 +74,30 @@ public class HoodieTestDataGenerator {
|
||||
public static final String[] DEFAULT_PARTITION_PATHS =
|
||||
{DEFAULT_FIRST_PARTITION_PATH, DEFAULT_SECOND_PARTITION_PATH, DEFAULT_THIRD_PARTITION_PATH};
|
||||
public static final int DEFAULT_PARTITION_DEPTH = 3;
|
||||
public static final String TRIP_EXAMPLE_SCHEMA = "{\"type\": \"record\"," + "\"name\": \"triprec\"," + "\"fields\": [ "
|
||||
public static final String TRIP_SCHEMA_PREFIX = "{\"type\": \"record\"," + "\"name\": \"triprec\"," + "\"fields\": [ "
|
||||
+ "{\"name\": \"timestamp\",\"type\": \"double\"}," + "{\"name\": \"_row_key\", \"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\"},"
|
||||
+ "{\"name\": \"fare\",\"type\": {\"type\":\"record\", \"name\":\"fare\",\"fields\": ["
|
||||
+ "{\"name\": \"amount\",\"type\": \"double\"},{\"name\": \"currency\", \"type\": \"string\"}]}},"
|
||||
+ "{\"name\": \"_hoodie_is_deleted\", \"type\": \"boolean\", \"default\": false} ]}";
|
||||
+ "{\"name\": \"end_lat\", \"type\": \"double\"}," + "{\"name\": \"end_lon\", \"type\": \"double\"},";
|
||||
public static final String TRIP_SCHEMA_SUFFIX = "{\"name\": \"_hoodie_is_deleted\", \"type\": \"boolean\", \"default\": false} ]}";
|
||||
public static final String FARE_NESTED_SCHEMA = "{\"name\": \"fare\",\"type\": {\"type\":\"record\", \"name\":\"fare\",\"fields\": ["
|
||||
+ "{\"name\": \"amount\",\"type\": \"double\"},{\"name\": \"currency\", \"type\": \"string\"}]}},";
|
||||
public static final String FARE_FLATTENED_SCHEMA = "{\"name\": \"fare\", \"type\": \"double\"},"
|
||||
+ "{\"name\": \"currency\", \"type\": \"string\"},";
|
||||
|
||||
public static final String TRIP_EXAMPLE_SCHEMA =
|
||||
TRIP_SCHEMA_PREFIX + FARE_NESTED_SCHEMA + TRIP_SCHEMA_SUFFIX;
|
||||
public static final String TRIP_FLATTENED_SCHEMA =
|
||||
TRIP_SCHEMA_PREFIX + FARE_FLATTENED_SCHEMA + TRIP_SCHEMA_SUFFIX;
|
||||
|
||||
public static final String NULL_SCHEMA = Schema.create(Schema.Type.NULL).toString();
|
||||
public static final String TRIP_HIVE_COLUMN_TYPES = "double,string,string,string,double,double,double,double,"
|
||||
+ "struct<amount:double,currency:string>,boolean";
|
||||
|
||||
public static final Schema AVRO_SCHEMA = new Schema.Parser().parse(TRIP_EXAMPLE_SCHEMA);
|
||||
public static final Schema AVRO_SCHEMA_WITH_METADATA_FIELDS =
|
||||
HoodieAvroUtils.addMetadataFields(AVRO_SCHEMA);
|
||||
public static final Schema FLATTENED_AVRO_SCHEMA = new Schema.Parser().parse(TRIP_FLATTENED_SCHEMA);
|
||||
|
||||
private static final Random RAND = new Random(46474747);
|
||||
|
||||
@@ -115,10 +125,33 @@ public class HoodieTestDataGenerator {
|
||||
}
|
||||
|
||||
/**
|
||||
* Generates a new avro record of the above schema format, retaining the key if optionally provided.
|
||||
* Generates a new avro record of the above nested schema format,
|
||||
* retaining the key if optionally provided.
|
||||
*
|
||||
* @param key Hoodie key.
|
||||
* @param commitTime Commit time to use.
|
||||
* @return Raw paylaod of a test record.
|
||||
* @throws IOException
|
||||
*/
|
||||
public static TestRawTripPayload generateRandomValue(HoodieKey key, String commitTime) throws IOException {
|
||||
GenericRecord rec = generateGenericRecord(key.getRecordKey(), "rider-" + commitTime, "driver-" + commitTime, 0.0);
|
||||
return generateRandomValue(key, commitTime, false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Generates a new avro record with the specified schema (nested or flattened),
|
||||
* retaining the key if optionally provided.
|
||||
*
|
||||
* @param key Hoodie key.
|
||||
* @param commitTime Commit time to use.
|
||||
* @param isFlattened whether the schema of the record should be flattened.
|
||||
* @return Raw paylaod of a test record.
|
||||
* @throws IOException
|
||||
*/
|
||||
public static TestRawTripPayload generateRandomValue(
|
||||
HoodieKey key, String commitTime, boolean isFlattened) throws IOException {
|
||||
GenericRecord rec = generateGenericRecord(
|
||||
key.getRecordKey(), "rider-" + commitTime, "driver-" + commitTime, 0.0,
|
||||
false, isFlattened);
|
||||
return new TestRawTripPayload(rec.toString(), key.getRecordKey(), key.getPartitionPath(), TRIP_EXAMPLE_SCHEMA);
|
||||
}
|
||||
|
||||
@@ -127,7 +160,7 @@ public class HoodieTestDataGenerator {
|
||||
*/
|
||||
public static TestRawTripPayload generateRandomDeleteValue(HoodieKey key, String commitTime) throws IOException {
|
||||
GenericRecord rec = generateGenericRecord(key.getRecordKey(), "rider-" + commitTime, "driver-" + commitTime, 0.0,
|
||||
true);
|
||||
true, false);
|
||||
return new TestRawTripPayload(rec.toString(), key.getRecordKey(), key.getPartitionPath(), TRIP_EXAMPLE_SCHEMA);
|
||||
}
|
||||
|
||||
@@ -141,12 +174,13 @@ public class HoodieTestDataGenerator {
|
||||
|
||||
public static GenericRecord generateGenericRecord(String rowKey, String riderName, String driverName,
|
||||
double timestamp) {
|
||||
return generateGenericRecord(rowKey, riderName, driverName, timestamp, false);
|
||||
return generateGenericRecord(rowKey, riderName, driverName, timestamp, false, false);
|
||||
}
|
||||
|
||||
public static GenericRecord generateGenericRecord(String rowKey, String riderName, String driverName,
|
||||
double timestamp, boolean isDeleteRecord) {
|
||||
GenericRecord rec = new GenericData.Record(AVRO_SCHEMA);
|
||||
double 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("rider", riderName);
|
||||
@@ -156,10 +190,15 @@ public class HoodieTestDataGenerator {
|
||||
rec.put("end_lat", RAND.nextDouble());
|
||||
rec.put("end_lon", RAND.nextDouble());
|
||||
|
||||
GenericRecord fareRecord = new GenericData.Record(AVRO_SCHEMA.getField("fare").schema());
|
||||
fareRecord.put("amount", RAND.nextDouble() * 100);
|
||||
fareRecord.put("currency", "USD");
|
||||
rec.put("fare", fareRecord);
|
||||
if (isFlattened) {
|
||||
rec.put("fare", RAND.nextDouble() * 100);
|
||||
rec.put("currency", "USD");
|
||||
} else {
|
||||
GenericRecord fareRecord = new GenericData.Record(AVRO_SCHEMA.getField("fare").schema());
|
||||
fareRecord.put("amount", RAND.nextDouble() * 100);
|
||||
fareRecord.put("currency", "USD");
|
||||
rec.put("fare", fareRecord);
|
||||
}
|
||||
|
||||
if (isDeleteRecord) {
|
||||
rec.put("_hoodie_is_deleted", true);
|
||||
@@ -230,16 +269,31 @@ public class HoodieTestDataGenerator {
|
||||
}
|
||||
|
||||
/**
|
||||
* Generates new inserts, uniformly across the partition paths above. It also updates the list of existing keys.
|
||||
* Generates new inserts with nested schema, uniformly across the partition paths above.
|
||||
* It also updates the list of existing keys.
|
||||
*/
|
||||
public List<HoodieRecord> generateInserts(String commitTime, Integer n) {
|
||||
return generateInsertsStream(commitTime, n).collect(Collectors.toList());
|
||||
return generateInserts(commitTime, n, false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Generates new inserts, uniformly across the partition paths above.
|
||||
* It also updates the list of existing keys.
|
||||
*
|
||||
* @param commitTime Commit time to use.
|
||||
* @param n Number of records.
|
||||
* @param isFlattened whether the schema of the generated record is flattened
|
||||
* @return List of {@link HoodieRecord}s
|
||||
*/
|
||||
public List<HoodieRecord> generateInserts(String commitTime, Integer n, boolean isFlattened) {
|
||||
return generateInsertsStream(commitTime, n, isFlattened).collect(Collectors.toList());
|
||||
}
|
||||
|
||||
/**
|
||||
* Generates new inserts, uniformly across the partition paths above. It also updates the list of existing keys.
|
||||
*/
|
||||
public Stream<HoodieRecord> generateInsertsStream(String commitTime, Integer n) {
|
||||
public Stream<HoodieRecord> generateInsertsStream(
|
||||
String commitTime, Integer n, boolean isFlattened) {
|
||||
int currSize = getNumExistingKeys();
|
||||
|
||||
return IntStream.range(0, n).boxed().map(i -> {
|
||||
@@ -251,7 +305,7 @@ public class HoodieTestDataGenerator {
|
||||
existingKeys.put(currSize + i, kp);
|
||||
numExistingKeys++;
|
||||
try {
|
||||
return new HoodieRecord(key, generateRandomValue(key, commitTime));
|
||||
return new HoodieRecord(key, generateRandomValue(key, commitTime, isFlattened));
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException(e.getMessage(), e);
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user