1
0

[HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment (#1150)

* [HUDI-288]: Add support for ingesting multiple kafka streams in a single DeltaStreamer deployment
This commit is contained in:
Pratyaksh Sharma
2020-04-08 04:40:26 +05:30
committed by GitHub
parent b5d093a21b
commit d610252d6b
26 changed files with 1184 additions and 78 deletions

View File

@@ -292,7 +292,8 @@ public class HoodieCommitArchiveLog {
archivedMetaWrapper.setActionType(ActionType.clean.name());
break;
}
case HoodieTimeline.COMMIT_ACTION: {
case HoodieTimeline.COMMIT_ACTION:
case HoodieTimeline.DELTA_COMMIT_ACTION: {
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
.fromBytes(commitTimeline.getInstantDetails(hoodieInstant).get(), HoodieCommitMetadata.class);
archivedMetaWrapper.setHoodieCommitMetadata(convertCommitMetadata(commitMetadata));
@@ -311,13 +312,6 @@ public class HoodieCommitArchiveLog {
archivedMetaWrapper.setActionType(ActionType.savepoint.name());
break;
}
case HoodieTimeline.DELTA_COMMIT_ACTION: {
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
.fromBytes(commitTimeline.getInstantDetails(hoodieInstant).get(), HoodieCommitMetadata.class);
archivedMetaWrapper.setHoodieCommitMetadata(convertCommitMetadata(commitMetadata));
archivedMetaWrapper.setActionType(ActionType.commit.name());
break;
}
case HoodieTimeline.COMPACTION_ACTION: {
HoodieCompactionPlan plan = CompactionUtils.getCompactionPlan(metaClient, hoodieInstant.getTimestamp());
archivedMetaWrapper.setHoodieCompactionPlan(plan);

View File

@@ -68,7 +68,7 @@ public class TestMultiFS extends HoodieClientTestHarness {
cleanupTestDataGenerator();
}
private HoodieWriteClient getHoodieWriteClient(HoodieWriteConfig config) throws Exception {
private HoodieWriteClient getHoodieWriteClient(HoodieWriteConfig config) {
return new HoodieWriteClient(jsc, config);
}
@@ -89,7 +89,7 @@ public class TestMultiFS extends HoodieClientTestHarness {
HoodieWriteConfig localConfig = getHoodieWriteConfig(tablePath);
try (HoodieWriteClient hdfsWriteClient = getHoodieWriteClient(cfg);
HoodieWriteClient localWriteClient = getHoodieWriteClient(localConfig);) {
HoodieWriteClient localWriteClient = getHoodieWriteClient(localConfig)) {
// Write generated data to hdfs (only inserts)
String readCommitTime = hdfsWriteClient.startCommit();

View File

@@ -41,6 +41,8 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.io.IOException;
import java.io.Serializable;
@@ -69,6 +71,7 @@ public class HoodieTestDataGenerator {
// based on examination of sample file, the schema produces the following per record size
public static final int SIZE_PER_RECORD = 50 * 1024;
private static Logger logger = LogManager.getLogger(HoodieTestDataGenerator.class);
public static final String DEFAULT_FIRST_PARTITION_PATH = "2016/03/15";
public static final String DEFAULT_SECOND_PARTITION_PATH = "2015/03/16";
public static final String DEFAULT_THIRD_PARTITION_PATH = "2015/03/17";
@@ -89,12 +92,18 @@ public class HoodieTestDataGenerator {
public static final String TIP_NESTED_SCHEMA = "{\"name\": \"tip_history\", \"type\": {\"type\": \"array\", \"items\": {\"type\": \"record\", \"name\": \"tip_history\", \"fields\": ["
+ "{\"name\": \"amount\", \"type\": \"double\"}, {\"name\": \"currency\", \"type\": \"string\"}]}}},";
public static final String MAP_TYPE_SCHEMA = "{\"name\": \"city_to_state\", \"type\": {\"type\": \"map\", \"values\": \"string\"}},";
public static final String TRIP_EXAMPLE_SCHEMA =
TRIP_SCHEMA_PREFIX + MAP_TYPE_SCHEMA + FARE_NESTED_SCHEMA + TIP_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 TRIP_SCHEMA = "{\"type\":\"record\",\"name\":\"tripUberRec\",\"fields\":["
+ "{\"name\":\"timestamp\",\"type\":\"double\"},{\"name\":\"_row_key\",\"type\":\"string\"},{\"name\":\"rider\",\"type\":\"string\"},"
+ "{\"name\":\"driver\",\"type\":\"string\"},{\"name\":\"fare\",\"type\":\"double\"},{\"name\": \"_hoodie_is_deleted\", \"type\": \"boolean\", \"default\": false}]}";
public static final String SHORT_TRIP_SCHEMA = "{\"type\":\"record\",\"name\":\"shortTripRec\",\"fields\":["
+ "{\"name\":\"timestamp\",\"type\":\"double\"},{\"name\":\"_row_key\",\"type\":\"string\"},{\"name\":\"rider\",\"type\":\"string\"},"
+ "{\"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 = "double,string,string,string,double,double,double,double,"
+ "map<string,string>,struct<amount:double,currency:string>,array<struct<amount:double,currency:string>>,boolean";
@@ -102,13 +111,17 @@ public class HoodieTestDataGenerator {
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 AVRO_SHORT_TRIP_SCHEMA = new Schema.Parser().parse(SHORT_TRIP_SCHEMA);
public static final Schema AVRO_TRIP_SCHEMA = new Schema.Parser().parse(TRIP_SCHEMA);
public static final Schema FLATTENED_AVRO_SCHEMA = new Schema.Parser().parse(TRIP_FLATTENED_SCHEMA);
private static final Random RAND = new Random(46474747);
private final Map<Integer, KeyPartition> existingKeys;
//Maintains all the existing keys schema wise
private final Map<String, Map<Integer, KeyPartition>> existingKeysBySchema;
private final String[] partitionPaths;
private int numExistingKeys;
//maintains the count of existing keys schema wise
private Map<String, Integer> numKeysBySchema;
public HoodieTestDataGenerator(String[] partitionPaths) {
this(partitionPaths, new HashMap<>());
@@ -120,7 +133,9 @@ public class HoodieTestDataGenerator {
public HoodieTestDataGenerator(String[] partitionPaths, Map<Integer, KeyPartition> keyPartitionMap) {
this.partitionPaths = Arrays.copyOf(partitionPaths, partitionPaths.length);
this.existingKeys = keyPartitionMap;
this.existingKeysBySchema = new HashMap<>();
existingKeysBySchema.put(TRIP_EXAMPLE_SCHEMA, keyPartitionMap);
numKeysBySchema = new HashMap<>();
}
public static void writePartitionMetadata(FileSystem fs, String[] partitionPaths, String basePath) {
@@ -129,6 +144,18 @@ public class HoodieTestDataGenerator {
}
}
public TestRawTripPayload generateRandomValueAsPerSchema(String schemaStr, HoodieKey key, String commitTime, boolean isFlattened) throws IOException {
if (TRIP_EXAMPLE_SCHEMA.equals(schemaStr)) {
return generateRandomValue(key, commitTime, isFlattened);
} else if (TRIP_SCHEMA.equals(schemaStr)) {
return generatePayloadForTripSchema(key, commitTime);
} else if (SHORT_TRIP_SCHEMA.equals(schemaStr)) {
return generatePayloadForShortTripSchema(key, commitTime);
}
return null;
}
/**
* Generates a new avro record of the above nested schema format,
* retaining the key if optionally provided.
@@ -160,6 +187,19 @@ public class HoodieTestDataGenerator {
return new TestRawTripPayload(rec.toString(), key.getRecordKey(), key.getPartitionPath(), TRIP_EXAMPLE_SCHEMA);
}
/**
* Generates a new avro record with TRIP_SCHEMA, retaining the key if optionally provided.
*/
public TestRawTripPayload generatePayloadForTripSchema(HoodieKey key, String commitTime) throws IOException {
GenericRecord rec = generateRecordForTripSchema(key.getRecordKey(), "rider-" + commitTime, "driver-" + commitTime, 0.0);
return new TestRawTripPayload(rec.toString(), key.getRecordKey(), key.getPartitionPath(), TRIP_SCHEMA);
}
public TestRawTripPayload generatePayloadForShortTripSchema(HoodieKey key, String commitTime) throws IOException {
GenericRecord rec = generateRecordForShortTripSchema(key.getRecordKey(), "rider-" + commitTime, "driver-" + commitTime, 0.0);
return new TestRawTripPayload(rec.toString(), key.getRecordKey(), key.getPartitionPath(), SHORT_TRIP_SCHEMA);
}
/**
* Generates a new avro record of the above schema format for a delete.
*/
@@ -223,6 +263,31 @@ public class HoodieTestDataGenerator {
return rec;
}
/*
Generate random record using TRIP_SCHEMA
*/
public GenericRecord generateRecordForTripSchema(String rowKey, String riderName, String driverName, double timestamp) {
GenericRecord rec = new GenericData.Record(AVRO_TRIP_SCHEMA);
rec.put("_row_key", rowKey);
rec.put("timestamp", timestamp);
rec.put("rider", riderName);
rec.put("driver", driverName);
rec.put("fare", RAND.nextDouble() * 100);
rec.put("_hoodie_is_deleted", false);
return rec;
}
public GenericRecord generateRecordForShortTripSchema(String rowKey, String riderName, String driverName, double timestamp) {
GenericRecord rec = new GenericData.Record(AVRO_SHORT_TRIP_SCHEMA);
rec.put("_row_key", rowKey);
rec.put("timestamp", timestamp);
rec.put("rider", riderName);
rec.put("driver", driverName);
rec.put("fare", RAND.nextDouble() * 100);
rec.put("_hoodie_is_deleted", false);
return rec;
}
public static void createCommitFile(String basePath, String instantTime, Configuration configuration) {
Arrays.asList(HoodieTimeline.makeCommitFileName(instantTime), HoodieTimeline.makeInflightCommitFileName(instantTime),
HoodieTimeline.makeRequestedCommitFileName(instantTime))
@@ -283,6 +348,10 @@ public class HoodieTestDataGenerator {
}
}
public List<HoodieRecord> generateInsertsAsPerSchema(String commitTime, Integer n, String schemaStr) {
return generateInsertsStream(commitTime, n, schemaStr).collect(Collectors.toList());
}
/**
* Generates new inserts with nested schema, uniformly across the partition paths above.
* It also updates the list of existing keys.
@@ -301,15 +370,22 @@ public class HoodieTestDataGenerator {
* @return List of {@link HoodieRecord}s
*/
public List<HoodieRecord> generateInserts(String instantTime, Integer n, boolean isFlattened) {
return generateInsertsStream(instantTime, n, isFlattened).collect(Collectors.toList());
return generateInsertsStream(instantTime, n, isFlattened, TRIP_EXAMPLE_SCHEMA).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, String schemaStr) {
return generateInsertsStream(commitTime, n, false, schemaStr);
}
/**
* Generates new inserts, uniformly across the partition paths above. It also updates the list of existing keys.
*/
public Stream<HoodieRecord> generateInsertsStream(
String instantTime, Integer n, boolean isFlattened) {
int currSize = getNumExistingKeys();
String instantTime, Integer n, boolean isFlattened, String schemaStr) {
int currSize = getNumExistingKeys(schemaStr);
return IntStream.range(0, n).boxed().map(i -> {
String partitionPath = partitionPaths[RAND.nextInt(partitionPaths.length)];
@@ -317,16 +393,36 @@ public class HoodieTestDataGenerator {
KeyPartition kp = new KeyPartition();
kp.key = key;
kp.partitionPath = partitionPath;
existingKeys.put(currSize + i, kp);
numExistingKeys++;
populateKeysBySchema(schemaStr, currSize + i, kp);
incrementNumExistingKeysBySchema(schemaStr);
try {
return new HoodieRecord(key, generateRandomValue(key, instantTime, isFlattened));
return new HoodieRecord(key, generateRandomValueAsPerSchema(schemaStr, key, instantTime, isFlattened));
} catch (IOException e) {
throw new HoodieIOException(e.getMessage(), e);
}
});
}
/*
Takes care of populating keys schema wise
*/
private void populateKeysBySchema(String schemaStr, int i, KeyPartition kp) {
if (existingKeysBySchema.containsKey(schemaStr)) {
existingKeysBySchema.get(schemaStr).put(i, kp);
} else {
existingKeysBySchema.put(schemaStr, new HashMap<>());
existingKeysBySchema.get(schemaStr).put(i, kp);
}
}
private void incrementNumExistingKeysBySchema(String schemaStr) {
if (numKeysBySchema.containsKey(schemaStr)) {
numKeysBySchema.put(schemaStr, numKeysBySchema.get(schemaStr) + 1);
} else {
numKeysBySchema.put(schemaStr, 1);
}
}
public List<HoodieRecord> generateSameKeyInserts(String instantTime, List<HoodieRecord> origin) throws IOException {
List<HoodieRecord> copy = new ArrayList<>();
for (HoodieRecord r : origin) {
@@ -339,7 +435,7 @@ public class HoodieTestDataGenerator {
public List<HoodieRecord> generateInsertsWithHoodieAvroPayload(String instantTime, int limit) {
List<HoodieRecord> inserts = new ArrayList<>();
int currSize = getNumExistingKeys();
int currSize = getNumExistingKeys(TRIP_EXAMPLE_SCHEMA);
for (int i = 0; i < limit; i++) {
String partitionPath = partitionPaths[RAND.nextInt(partitionPaths.length)];
HoodieKey key = new HoodieKey(UUID.randomUUID().toString(), partitionPath);
@@ -349,8 +445,8 @@ public class HoodieTestDataGenerator {
KeyPartition kp = new KeyPartition();
kp.key = key;
kp.partitionPath = partitionPath;
existingKeys.put(currSize + i, kp);
numExistingKeys++;
populateKeysBySchema(TRIP_EXAMPLE_SCHEMA, currSize + i, kp);
incrementNumExistingKeysBySchema(TRIP_EXAMPLE_SCHEMA);
}
return inserts;
}
@@ -431,6 +527,8 @@ public class HoodieTestDataGenerator {
public List<HoodieRecord> generateUpdates(String instantTime, Integer n) throws IOException {
List<HoodieRecord> updates = new ArrayList<>();
for (int i = 0; i < n; i++) {
Map<Integer, KeyPartition> existingKeys = existingKeysBySchema.get(TRIP_EXAMPLE_SCHEMA);
Integer numExistingKeys = numKeysBySchema.get(TRIP_EXAMPLE_SCHEMA);
KeyPartition kp = existingKeys.get(RAND.nextInt(numExistingKeys - 1));
HoodieRecord record = generateUpdateRecord(kp.key, instantTime);
updates.add(record);
@@ -438,6 +536,10 @@ public class HoodieTestDataGenerator {
return updates;
}
public List<HoodieRecord> generateUpdatesAsPerSchema(String commitTime, Integer n, String schemaStr) {
return generateUniqueUpdatesStream(commitTime, n, schemaStr).collect(Collectors.toList());
}
/**
* Generates deduped updates of keys previously inserted, randomly distributed across the keys above.
*
@@ -446,7 +548,7 @@ public class HoodieTestDataGenerator {
* @return list of hoodie record updates
*/
public List<HoodieRecord> generateUniqueUpdates(String instantTime, Integer n) {
return generateUniqueUpdatesStream(instantTime, n).collect(Collectors.toList());
return generateUniqueUpdatesStream(instantTime, n, TRIP_EXAMPLE_SCHEMA).collect(Collectors.toList());
}
/**
@@ -466,8 +568,10 @@ public class HoodieTestDataGenerator {
* @param n Number of unique records
* @return stream of hoodie record updates
*/
public Stream<HoodieRecord> generateUniqueUpdatesStream(String instantTime, Integer n) {
public Stream<HoodieRecord> generateUniqueUpdatesStream(String instantTime, Integer n, String schemaStr) {
final Set<KeyPartition> used = new HashSet<>();
int numExistingKeys = numKeysBySchema.getOrDefault(schemaStr, 0);
Map<Integer, KeyPartition> existingKeys = existingKeysBySchema.get(schemaStr);
if (n > numExistingKeys) {
throw new IllegalArgumentException("Requested unique updates is greater than number of available keys");
}
@@ -480,9 +584,10 @@ public class HoodieTestDataGenerator {
index = (index + 1) % numExistingKeys;
kp = existingKeys.get(index);
}
logger.debug("key getting updated: " + kp.key.getRecordKey());
used.add(kp);
try {
return new HoodieRecord(kp.key, generateRandomValue(kp.key, instantTime));
return new HoodieRecord(kp.key, generateRandomValueAsPerSchema(schemaStr, kp.key, instantTime, false));
} catch (IOException e) {
throw new HoodieIOException(e.getMessage(), e);
}
@@ -497,6 +602,8 @@ public class HoodieTestDataGenerator {
*/
public Stream<HoodieKey> generateUniqueDeleteStream(Integer n) {
final Set<KeyPartition> used = new HashSet<>();
Map<Integer, KeyPartition> existingKeys = existingKeysBySchema.get(TRIP_EXAMPLE_SCHEMA);
Integer numExistingKeys = numKeysBySchema.get(TRIP_EXAMPLE_SCHEMA);
if (n > numExistingKeys) {
throw new IllegalArgumentException("Requested unique deletes is greater than number of available keys");
}
@@ -514,6 +621,7 @@ public class HoodieTestDataGenerator {
used.add(kp);
result.add(kp.key);
}
numKeysBySchema.put(TRIP_EXAMPLE_SCHEMA, numExistingKeys);
return result.stream();
}
@@ -526,6 +634,8 @@ public class HoodieTestDataGenerator {
*/
public Stream<HoodieRecord> generateUniqueDeleteRecordStream(String instantTime, Integer n) {
final Set<KeyPartition> used = new HashSet<>();
Map<Integer, KeyPartition> existingKeys = existingKeysBySchema.get(TRIP_EXAMPLE_SCHEMA);
Integer numExistingKeys = numKeysBySchema.get(TRIP_EXAMPLE_SCHEMA);
if (n > numExistingKeys) {
throw new IllegalArgumentException("Requested unique deletes is greater than number of available keys");
}
@@ -536,7 +646,7 @@ public class HoodieTestDataGenerator {
while (!existingKeys.containsKey(index)) {
index = (index + 1) % numExistingKeys;
}
// swap chosen index with last index and remove last entry.
// swap chosen index with last index and remove last entry.
KeyPartition kp = existingKeys.remove(index);
existingKeys.put(index, existingKeys.get(numExistingKeys - 1));
existingKeys.remove(numExistingKeys - 1);
@@ -548,6 +658,7 @@ public class HoodieTestDataGenerator {
throw new HoodieIOException(e.getMessage(), e);
}
}
numKeysBySchema.put(TRIP_EXAMPLE_SCHEMA, numExistingKeys);
return result.stream();
}
@@ -555,8 +666,8 @@ public class HoodieTestDataGenerator {
return partitionPaths;
}
public int getNumExistingKeys() {
return numExistingKeys;
public int getNumExistingKeys(String schemaStr) {
return numKeysBySchema.getOrDefault(schemaStr, 0);
}
public static class KeyPartition implements Serializable {
@@ -566,6 +677,6 @@ public class HoodieTestDataGenerator {
}
public void close() {
existingKeys.clear();
existingKeysBySchema.clear();
}
}