[HUDI-377] Adding Delete() support to DeltaStreamer (#1073)
- Provides ability to perform hard deletes by writing delete marker records into the source data - if the record contains a special field _hoodie_delete_marker set to true, deletes are performed
This commit is contained in:
committed by
vinoth chandar
parent
726ae47ce2
commit
7031445eb3
@@ -57,6 +57,7 @@ import java.util.stream.Collectors;
|
||||
import java.util.stream.IntStream;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
|
||||
/**
|
||||
* Class to be used in tests to keep generating test inserts and updates against a corpus.
|
||||
* <p>
|
||||
@@ -73,14 +74,15 @@ 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 String TRIP_EXAMPLE_SCHEMA = "{\"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\": \"double\"}]}";
|
||||
public static String TRIP_EXAMPLE_SCHEMA = "{\"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\": \"double\"},"
|
||||
+ "{\"name\": \"_hoodie_is_deleted\", \"type\": \"boolean\", \"default\": false} ]}";
|
||||
public static String NULL_SCHEMA = Schema.create(Schema.Type.NULL).toString();
|
||||
public static String TRIP_HIVE_COLUMN_TYPES = "double,string,string,string,double,double,double,double,double";
|
||||
public static String TRIP_HIVE_COLUMN_TYPES = "double,string,string,string,double,double,double,double,double,boolean";
|
||||
public static Schema avroSchema = new Schema.Parser().parse(TRIP_EXAMPLE_SCHEMA);
|
||||
public static Schema avroSchemaWithMetadataFields = HoodieAvroUtils.addMetadataFields(avroSchema);
|
||||
|
||||
@@ -117,6 +119,15 @@ public class HoodieTestDataGenerator {
|
||||
return new TestRawTripPayload(rec.toString(), key.getRecordKey(), key.getPartitionPath(), TRIP_EXAMPLE_SCHEMA);
|
||||
}
|
||||
|
||||
/**
|
||||
* Generates a new avro record of the above schema format for a delete.
|
||||
*/
|
||||
public static TestRawTripPayload generateRandomDeleteValue(HoodieKey key, String commitTime) throws IOException {
|
||||
GenericRecord rec = generateGenericRecord(key.getRecordKey(), "rider-" + commitTime, "driver-" + commitTime, 0.0,
|
||||
true);
|
||||
return new TestRawTripPayload(rec.toString(), key.getRecordKey(), key.getPartitionPath(), TRIP_EXAMPLE_SCHEMA);
|
||||
}
|
||||
|
||||
/**
|
||||
* Generates a new avro record of the above schema format, retaining the key if optionally provided.
|
||||
*/
|
||||
@@ -126,7 +137,12 @@ public class HoodieTestDataGenerator {
|
||||
}
|
||||
|
||||
public static GenericRecord generateGenericRecord(String rowKey, String riderName, String driverName,
|
||||
double timestamp) {
|
||||
double timestamp) {
|
||||
return generateGenericRecord(rowKey, riderName, driverName, timestamp, false);
|
||||
}
|
||||
|
||||
public static GenericRecord generateGenericRecord(String rowKey, String riderName, String driverName,
|
||||
double timestamp, boolean isDeleteRecord) {
|
||||
GenericRecord rec = new GenericData.Record(avroSchema);
|
||||
rec.put("_row_key", rowKey);
|
||||
rec.put("timestamp", timestamp);
|
||||
@@ -137,12 +153,18 @@ public class HoodieTestDataGenerator {
|
||||
rec.put("end_lat", rand.nextDouble());
|
||||
rec.put("end_lon", rand.nextDouble());
|
||||
rec.put("fare", rand.nextDouble() * 100);
|
||||
if (isDeleteRecord) {
|
||||
rec.put("_hoodie_is_deleted", true);
|
||||
} else {
|
||||
rec.put("_hoodie_is_deleted", false);
|
||||
}
|
||||
return rec;
|
||||
}
|
||||
|
||||
public static void createCommitFile(String basePath, String commitTime, Configuration configuration) {
|
||||
Arrays.asList(HoodieTimeline.makeCommitFileName(commitTime), HoodieTimeline.makeInflightCommitFileName(commitTime),
|
||||
HoodieTimeline.makeRequestedCommitFileName(commitTime)).forEach(f -> {
|
||||
HoodieTimeline.makeRequestedCommitFileName(commitTime))
|
||||
.forEach(f -> {
|
||||
Path commitFile = new Path(
|
||||
basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + f);
|
||||
FSDataOutputStream os = null;
|
||||
@@ -176,7 +198,7 @@ public class HoodieTestDataGenerator {
|
||||
}
|
||||
|
||||
public static void createCompactionAuxiliaryMetadata(String basePath, HoodieInstant instant,
|
||||
Configuration configuration) throws IOException {
|
||||
Configuration configuration) throws IOException {
|
||||
Path commitFile =
|
||||
new Path(basePath + "/" + HoodieTableMetaClient.AUXILIARYFOLDER_NAME + "/" + instant.getFileName());
|
||||
FileSystem fs = FSUtils.getFs(basePath, configuration);
|
||||
@@ -332,7 +354,7 @@ public class HoodieTestDataGenerator {
|
||||
* list
|
||||
*
|
||||
* @param commitTime Commit Timestamp
|
||||
* @param n Number of updates (including dups)
|
||||
* @param n Number of updates (including dups)
|
||||
* @return list of hoodie record updates
|
||||
*/
|
||||
public List<HoodieRecord> generateUpdates(String commitTime, Integer n) throws IOException {
|
||||
@@ -349,7 +371,7 @@ public class HoodieTestDataGenerator {
|
||||
* Generates deduped updates of keys previously inserted, randomly distributed across the keys above.
|
||||
*
|
||||
* @param commitTime Commit Timestamp
|
||||
* @param n Number of unique records
|
||||
* @param n Number of unique records
|
||||
* @return list of hoodie record updates
|
||||
*/
|
||||
public List<HoodieRecord> generateUniqueUpdates(String commitTime, Integer n) {
|
||||
@@ -370,7 +392,7 @@ public class HoodieTestDataGenerator {
|
||||
* Generates deduped updates of keys previously inserted, randomly distributed across the keys above.
|
||||
*
|
||||
* @param commitTime Commit Timestamp
|
||||
* @param n Number of unique records
|
||||
* @param n Number of unique records
|
||||
* @return stream of hoodie record updates
|
||||
*/
|
||||
public Stream<HoodieRecord> generateUniqueUpdatesStream(String commitTime, Integer n) {
|
||||
@@ -418,11 +440,46 @@ public class HoodieTestDataGenerator {
|
||||
index = (index + 1) % numExistingKeys;
|
||||
kp = existingKeys.get(index);
|
||||
}
|
||||
existingKeys.remove(kp);
|
||||
numExistingKeys--;
|
||||
used.add(kp);
|
||||
return kp.key;
|
||||
});
|
||||
}
|
||||
|
||||
/**
|
||||
* Generates deduped delete records previously inserted, randomly distributed across the keys above.
|
||||
*
|
||||
* @param commitTime Commit Timestamp
|
||||
* @param n Number of unique records
|
||||
* @return stream of hoodie records for delete
|
||||
*/
|
||||
public Stream<HoodieRecord> generateUniqueDeleteRecordStream(String commitTime, Integer n) {
|
||||
final Set<KeyPartition> used = new HashSet<>();
|
||||
|
||||
if (n > numExistingKeys) {
|
||||
throw new IllegalArgumentException("Requested unique deletes is greater than number of available keys");
|
||||
}
|
||||
|
||||
return IntStream.range(0, n).boxed().map(i -> {
|
||||
int index = numExistingKeys == 1 ? 0 : rand.nextInt(numExistingKeys - 1);
|
||||
KeyPartition kp = existingKeys.get(index);
|
||||
// Find the available keyPartition starting from randomly chosen one.
|
||||
while (used.contains(kp)) {
|
||||
index = (index + 1) % numExistingKeys;
|
||||
kp = existingKeys.get(index);
|
||||
}
|
||||
existingKeys.remove(kp);
|
||||
numExistingKeys--;
|
||||
used.add(kp);
|
||||
try {
|
||||
return new HoodieRecord(kp.key, generateRandomDeleteValue(kp.key, commitTime));
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException(e.getMessage(), e);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
public String[] getPartitionPaths() {
|
||||
return partitionPaths;
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user