[HUDI-69] Support Spark Datasource for MOR table - RDD approach (#1848)
- This PR implements Spark Datasource for MOR table in the RDD approach. - Implemented SnapshotRelation - Implemented HudiMergeOnReadRDD - Implemented separate Iterator to handle merge and unmerge record reader. - Added TestMORDataSource to verify this feature. - Clean up test file name, add tests for mixed query type tests - We can now revert the change made in DefaultSource Co-authored-by: Vinoth Chandar <vchandar@confluent.io>
This commit is contained in:
@@ -634,6 +634,10 @@ public class HoodieTestDataGenerator {
|
||||
return generateUniqueUpdatesStream(instantTime, n, TRIP_EXAMPLE_SCHEMA).collect(Collectors.toList());
|
||||
}
|
||||
|
||||
public List<HoodieRecord> generateUniqueUpdatesAsPerSchema(String instantTime, Integer n, String schemaStr) {
|
||||
return generateUniqueUpdatesStream(instantTime, n, schemaStr).collect(Collectors.toList());
|
||||
}
|
||||
|
||||
/**
|
||||
* Generates deduped delete of keys previously inserted, randomly distributed across the keys above.
|
||||
*
|
||||
@@ -745,6 +749,17 @@ public class HoodieTestDataGenerator {
|
||||
return result.stream();
|
||||
}
|
||||
|
||||
/**
|
||||
* Generates deduped delete records previously inserted, randomly distributed across the keys above.
|
||||
*
|
||||
* @param instantTime Commit Timestamp
|
||||
* @param n Number of unique records
|
||||
* @return List of hoodie records for delete
|
||||
*/
|
||||
public List<HoodieRecord> generateUniqueDeleteRecords(String instantTime, Integer n) {
|
||||
return generateUniqueDeleteRecordStream(instantTime, n).collect(Collectors.toList());
|
||||
}
|
||||
|
||||
public boolean deleteExistingKeyIfPresent(HoodieKey key) {
|
||||
Map<Integer, KeyPartition> existingKeys = existingKeysBySchema.get(TRIP_EXAMPLE_SCHEMA);
|
||||
Integer numExistingKeys = numKeysBySchema.get(TRIP_EXAMPLE_SCHEMA);
|
||||
|
||||
Reference in New Issue
Block a user