[HUDI-2502] Refactor index in hudi-client module (#3778)
- Refactor Index to reduce Line of Code and re-use across engines.
This commit is contained in:
@@ -156,7 +156,7 @@ public class TestHBaseIndex extends SparkClientFunctionalTestHarness {
|
||||
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
|
||||
// Test tagLocation without any entries in index
|
||||
JavaRDD<HoodieRecord> records1 = index.tagLocation(writeRecords, context(), hoodieTable);
|
||||
JavaRDD<HoodieRecord> records1 = tagLocation(index, writeRecords, hoodieTable);
|
||||
assertEquals(0, records1.filter(record -> record.isCurrentLocationKnown()).count());
|
||||
|
||||
// Insert 200 records
|
||||
@@ -165,7 +165,7 @@ public class TestHBaseIndex extends SparkClientFunctionalTestHarness {
|
||||
assertNoWriteErrors(writeStatues.collect());
|
||||
|
||||
// Now tagLocation for these records, hbaseIndex should not tag them since commit never occurred
|
||||
JavaRDD<HoodieRecord> records2 = index.tagLocation(writeRecords, context(), hoodieTable);
|
||||
JavaRDD<HoodieRecord> records2 = tagLocation(index, writeRecords, hoodieTable);
|
||||
assertEquals(0, records2.filter(record -> record.isCurrentLocationKnown()).count());
|
||||
|
||||
// Now commit this & update location of records inserted and validate no errors
|
||||
@@ -173,7 +173,7 @@ public class TestHBaseIndex extends SparkClientFunctionalTestHarness {
|
||||
// Now tagLocation for these records, hbaseIndex should tag them correctly
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
List<HoodieRecord> records3 = index.tagLocation(writeRecords, context(), hoodieTable).collect();
|
||||
List<HoodieRecord> records3 = tagLocation(index, writeRecords, hoodieTable).collect();
|
||||
assertEquals(numRecords, records3.stream().filter(record -> record.isCurrentLocationKnown()).count());
|
||||
assertEquals(numRecords, records3.stream().map(record -> record.getKey().getRecordKey()).distinct().count());
|
||||
assertEquals(numRecords, records3.stream().filter(record -> (record.getCurrentLocation() != null
|
||||
@@ -207,17 +207,17 @@ public class TestHBaseIndex extends SparkClientFunctionalTestHarness {
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
|
||||
JavaRDD<HoodieRecord> oldHoodieRecord = index.tagLocation(oldWriteRecords, context, hoodieTable);
|
||||
JavaRDD<HoodieRecord> oldHoodieRecord = tagLocation(index, oldWriteRecords, hoodieTable);
|
||||
assertEquals(0, oldHoodieRecord.filter(record -> record.isCurrentLocationKnown()).count());
|
||||
writeClient.startCommitWithTime(newCommitTime);
|
||||
JavaRDD<WriteStatus> writeStatues = writeClient.upsert(oldWriteRecords, newCommitTime);
|
||||
writeClient.commit(newCommitTime, writeStatues);
|
||||
assertNoWriteErrors(writeStatues.collect());
|
||||
index.updateLocation(writeStatues, context, hoodieTable);
|
||||
updateLocation(index, writeStatues, hoodieTable);
|
||||
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
List<HoodieRecord> taggedRecords = index.tagLocation(newWriteRecords, context, hoodieTable).collect();
|
||||
List<HoodieRecord> taggedRecords = tagLocation(index, newWriteRecords, hoodieTable).collect();
|
||||
assertEquals(numRecords * 2L, taggedRecords.stream().count());
|
||||
// Verify the number of deleted records
|
||||
assertEquals(numRecords, taggedRecords.stream().filter(record -> record.getKey().getPartitionPath().equals(oldPartitionPath)
|
||||
@@ -227,7 +227,7 @@ public class TestHBaseIndex extends SparkClientFunctionalTestHarness {
|
||||
|
||||
// not allowed path change test
|
||||
index = new SparkHoodieHBaseIndex<>(getConfig(false, false));
|
||||
List<HoodieRecord> notAllowPathChangeRecords = index.tagLocation(newWriteRecords, context, hoodieTable).collect();
|
||||
List<HoodieRecord> notAllowPathChangeRecords = tagLocation(index, newWriteRecords, hoodieTable).collect();
|
||||
assertEquals(numRecords, notAllowPathChangeRecords.stream().count());
|
||||
assertEquals(numRecords, taggedRecords.stream().filter(hoodieRecord -> hoodieRecord.isCurrentLocationKnown()
|
||||
&& hoodieRecord.getKey().getPartitionPath().equals(oldPartitionPath)).count());
|
||||
@@ -250,7 +250,7 @@ public class TestHBaseIndex extends SparkClientFunctionalTestHarness {
|
||||
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
|
||||
JavaRDD<WriteStatus> writeStatues = writeClient.upsert(writeRecords, newCommitTime);
|
||||
index.tagLocation(writeRecords, context(), hoodieTable);
|
||||
tagLocation(index, writeRecords, hoodieTable);
|
||||
|
||||
// Duplicate upsert and ensure correctness is maintained
|
||||
// We are trying to approximately imitate the case when the RDD is recomputed. For RDD creating, driver code is not
|
||||
@@ -266,7 +266,7 @@ public class TestHBaseIndex extends SparkClientFunctionalTestHarness {
|
||||
// Now tagLocation for these records, hbaseIndex should tag them correctly
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
List<HoodieRecord> taggedRecords = index.tagLocation(writeRecords, context(), hoodieTable).collect();
|
||||
List<HoodieRecord> taggedRecords = tagLocation(index, writeRecords, hoodieTable).collect();
|
||||
assertEquals(numRecords, taggedRecords.stream().filter(HoodieRecord::isCurrentLocationKnown).count());
|
||||
assertEquals(numRecords, taggedRecords.stream().map(record -> record.getKey().getRecordKey()).distinct().count());
|
||||
assertEquals(numRecords, taggedRecords.stream().filter(record -> (record.getCurrentLocation() != null
|
||||
@@ -295,22 +295,22 @@ public class TestHBaseIndex extends SparkClientFunctionalTestHarness {
|
||||
// first commit old record
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
List<HoodieRecord> beforeFirstTaggedRecords = index.tagLocation(oldWriteRecords, context, hoodieTable).collect();
|
||||
List<HoodieRecord> beforeFirstTaggedRecords = tagLocation(index, oldWriteRecords, hoodieTable).collect();
|
||||
JavaRDD<WriteStatus> oldWriteStatues = writeClient.upsert(oldWriteRecords, firstCommitTime);
|
||||
index.updateLocation(oldWriteStatues, context, hoodieTable);
|
||||
updateLocation(index, oldWriteStatues, hoodieTable);
|
||||
writeClient.commit(firstCommitTime, oldWriteStatues);
|
||||
List<HoodieRecord> afterFirstTaggedRecords = index.tagLocation(oldWriteRecords, context, hoodieTable).collect();
|
||||
List<HoodieRecord> afterFirstTaggedRecords = tagLocation(index, oldWriteRecords, hoodieTable).collect();
|
||||
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
final String secondCommitTime = writeClient.startCommit();
|
||||
List<HoodieRecord> beforeSecondTaggedRecords = index.tagLocation(newWriteRecords, context, hoodieTable).collect();
|
||||
List<HoodieRecord> beforeSecondTaggedRecords = tagLocation(index, newWriteRecords, hoodieTable).collect();
|
||||
JavaRDD<WriteStatus> newWriteStatues = writeClient.upsert(newWriteRecords, secondCommitTime);
|
||||
index.updateLocation(newWriteStatues, context, hoodieTable);
|
||||
updateLocation(index, newWriteStatues, hoodieTable);
|
||||
writeClient.commit(secondCommitTime, newWriteStatues);
|
||||
List<HoodieRecord> afterSecondTaggedRecords = index.tagLocation(newWriteRecords, context, hoodieTable).collect();
|
||||
List<HoodieRecord> afterSecondTaggedRecords = tagLocation(index, newWriteRecords, hoodieTable).collect();
|
||||
writeClient.rollback(secondCommitTime);
|
||||
List<HoodieRecord> afterRollback = index.tagLocation(newWriteRecords, context, hoodieTable).collect();
|
||||
List<HoodieRecord> afterRollback = tagLocation(index, newWriteRecords, hoodieTable).collect();
|
||||
|
||||
// Verify the first commit
|
||||
assertEquals(numRecords, beforeFirstTaggedRecords.stream().filter(record -> record.getCurrentLocation() == null).count());
|
||||
@@ -355,7 +355,7 @@ public class TestHBaseIndex extends SparkClientFunctionalTestHarness {
|
||||
writeClient.commit(newCommitTime, writeStatues);
|
||||
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
// Now tagLocation for these records, hbaseIndex should tag them
|
||||
List<HoodieRecord> records2 = index.tagLocation(writeRecords, context(), hoodieTable).collect();
|
||||
List<HoodieRecord> records2 = tagLocation(index, writeRecords, hoodieTable).collect();
|
||||
assertEquals(numRecords, records2.stream().filter(HoodieRecord::isCurrentLocationKnown).count());
|
||||
|
||||
// check tagged records are tagged with correct fileIds
|
||||
@@ -371,7 +371,7 @@ public class TestHBaseIndex extends SparkClientFunctionalTestHarness {
|
||||
hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
// Now tagLocation for these records, hbaseIndex should not tag them since it was a rolled
|
||||
// back commit
|
||||
List<HoodieRecord> records3 = index.tagLocation(writeRecords, context(), hoodieTable).collect();
|
||||
List<HoodieRecord> records3 = tagLocation(index, writeRecords, hoodieTable).collect();
|
||||
assertEquals(0, records3.stream().filter(HoodieRecord::isCurrentLocationKnown).count());
|
||||
assertEquals(0, records3.stream().filter(record -> record.getCurrentLocation() != null).count());
|
||||
}
|
||||
@@ -397,7 +397,7 @@ public class TestHBaseIndex extends SparkClientFunctionalTestHarness {
|
||||
|
||||
// verify location is tagged.
|
||||
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
JavaRDD<HoodieRecord> javaRDD0 = index.tagLocation(invalidWriteRecords, context(), hoodieTable);
|
||||
JavaRDD<HoodieRecord> javaRDD0 = tagLocation(index, invalidWriteRecords, hoodieTable);
|
||||
assert (javaRDD0.collect().size() == 1); // one record present
|
||||
assert (javaRDD0.filter(HoodieRecord::isCurrentLocationKnown).collect().size() == 1); // it is tagged
|
||||
assert (javaRDD0.collect().get(0).getCurrentLocation().getInstantTime().equals(invalidCommit));
|
||||
@@ -408,11 +408,11 @@ public class TestHBaseIndex extends SparkClientFunctionalTestHarness {
|
||||
// Now tagLocation for the valid records, hbaseIndex should tag them
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
JavaRDD<HoodieRecord> javaRDD1 = index.tagLocation(writeRecords, context(), hoodieTable);
|
||||
JavaRDD<HoodieRecord> javaRDD1 = tagLocation(index, writeRecords, hoodieTable);
|
||||
assert (javaRDD1.filter(HoodieRecord::isCurrentLocationKnown).collect().size() == 199);
|
||||
|
||||
// tagLocation for the invalid record - commit is not present in timeline due to rollback.
|
||||
JavaRDD<HoodieRecord> javaRDD2 = index.tagLocation(invalidWriteRecords, context(), hoodieTable);
|
||||
JavaRDD<HoodieRecord> javaRDD2 = tagLocation(index, invalidWriteRecords, hoodieTable);
|
||||
assert (javaRDD2.collect().size() == 1); // one record present
|
||||
assert (javaRDD2.filter(HoodieRecord::isCurrentLocationKnown).collect().size() == 0); // it is not tagged
|
||||
}
|
||||
@@ -442,7 +442,7 @@ public class TestHBaseIndex extends SparkClientFunctionalTestHarness {
|
||||
// Now tagLocation for the first set of rolledback records, hbaseIndex should tag them
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
JavaRDD<HoodieRecord> javaRDD1 = index.tagLocation(writeRecords1, context(), hoodieTable);
|
||||
JavaRDD<HoodieRecord> javaRDD1 = tagLocation(index, writeRecords1, hoodieTable);
|
||||
assert (javaRDD1.filter(HoodieRecord::isCurrentLocationKnown).collect().size() == 20);
|
||||
}
|
||||
|
||||
@@ -492,7 +492,7 @@ public class TestHBaseIndex extends SparkClientFunctionalTestHarness {
|
||||
// tagLocation for the first set of records (for the archived commit), hbaseIndex should tag them as valid
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
JavaRDD<HoodieRecord> javaRDD1 = index.tagLocation(writeRecords1, context(), hoodieTable);
|
||||
JavaRDD<HoodieRecord> javaRDD1 = tagLocation(index, writeRecords1, hoodieTable);
|
||||
assertEquals(20, javaRDD1.filter(HoodieRecord::isCurrentLocationKnown).collect().size());
|
||||
}
|
||||
|
||||
@@ -524,7 +524,7 @@ public class TestHBaseIndex extends SparkClientFunctionalTestHarness {
|
||||
assertNoWriteErrors(writeStatues.collect());
|
||||
|
||||
// Now tagLocation for these records, hbaseIndex should tag them
|
||||
index.tagLocation(writeRecords, context(), hoodieTable);
|
||||
tagLocation(index, writeRecords, hoodieTable);
|
||||
|
||||
// 3 batches should be executed given batchSize = 100 and parallelism = 1
|
||||
verify(table, times(3)).get((List<Get>) any());
|
||||
@@ -562,7 +562,7 @@ public class TestHBaseIndex extends SparkClientFunctionalTestHarness {
|
||||
// Get all the files generated
|
||||
int numberOfDataFileIds = (int) writeStatues.map(status -> status.getFileId()).distinct().count();
|
||||
|
||||
index.updateLocation(writeStatues, context(), hoodieTable);
|
||||
updateLocation(index, writeStatues, hoodieTable);
|
||||
// 3 batches should be executed given batchSize = 100 and <=numberOfDataFileIds getting updated,
|
||||
// so each fileId ideally gets updates
|
||||
verify(table, atMost(numberOfDataFileIds)).put((List<Put>) any());
|
||||
@@ -696,7 +696,7 @@ public class TestHBaseIndex extends SparkClientFunctionalTestHarness {
|
||||
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
|
||||
// Test tagLocation without any entries in index
|
||||
JavaRDD<HoodieRecord> records1 = index.tagLocation(writeRecords, context(), hoodieTable);
|
||||
JavaRDD<HoodieRecord> records1 = tagLocation(index, writeRecords, hoodieTable);
|
||||
assertEquals(0, records1.filter(record -> record.isCurrentLocationKnown()).count());
|
||||
// Insert 200 records
|
||||
writeClient.startCommitWithTime(newCommitTime);
|
||||
@@ -705,7 +705,7 @@ public class TestHBaseIndex extends SparkClientFunctionalTestHarness {
|
||||
|
||||
// Now tagLocation for these records, hbaseIndex should not tag them since it was a failed
|
||||
// commit
|
||||
JavaRDD<HoodieRecord> records2 = index.tagLocation(writeRecords, context(), hoodieTable);
|
||||
JavaRDD<HoodieRecord> records2 = tagLocation(index, writeRecords, hoodieTable);
|
||||
assertEquals(0, records2.filter(record -> record.isCurrentLocationKnown()).count());
|
||||
|
||||
// Now commit this & update location of records inserted and validate no errors
|
||||
@@ -713,7 +713,7 @@ public class TestHBaseIndex extends SparkClientFunctionalTestHarness {
|
||||
// Now tagLocation for these records, hbaseIndex should tag them correctly
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
List<HoodieRecord> records3 = index.tagLocation(writeRecords, context(), hoodieTable).collect();
|
||||
List<HoodieRecord> records3 = tagLocation(index, writeRecords, hoodieTable).collect();
|
||||
assertEquals(numRecords, records3.stream().filter(record -> record.isCurrentLocationKnown()).count());
|
||||
assertEquals(numRecords, records3.stream().map(record -> record.getKey().getRecordKey()).distinct().count());
|
||||
assertEquals(numRecords, records3.stream().filter(record -> (record.getCurrentLocation() != null
|
||||
@@ -736,7 +736,7 @@ public class TestHBaseIndex extends SparkClientFunctionalTestHarness {
|
||||
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
|
||||
// Test tagLocation without any entries in index
|
||||
JavaRDD<HoodieRecord> records1 = index.tagLocation(writeRecords, context(), hoodieTable);
|
||||
JavaRDD<HoodieRecord> records1 = tagLocation(index, writeRecords, hoodieTable);
|
||||
assertEquals(0, records1.filter(record -> record.isCurrentLocationKnown()).count());
|
||||
|
||||
// Insert records
|
||||
@@ -748,7 +748,7 @@ public class TestHBaseIndex extends SparkClientFunctionalTestHarness {
|
||||
// Now tagLocation for these records, hbaseIndex should tag them correctly
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
List<HoodieRecord> records2 = index.tagLocation(writeRecords, context(), hoodieTable).collect();
|
||||
List<HoodieRecord> records2 = tagLocation(index, writeRecords, hoodieTable).collect();
|
||||
assertEquals(numRecords, records2.stream().filter(record -> record.isCurrentLocationKnown()).count());
|
||||
assertEquals(numRecords, records2.stream().map(record -> record.getKey().getRecordKey()).distinct().count());
|
||||
assertEquals(numRecords, records2.stream().filter(record -> (record.getCurrentLocation() != null
|
||||
@@ -766,12 +766,12 @@ public class TestHBaseIndex extends SparkClientFunctionalTestHarness {
|
||||
// if not for this caching, due to RDD chaining/lineage, first time update is called again when subsequent update is called.
|
||||
// So caching here to break the chain and so future update does not re-trigger update of older Rdd.
|
||||
deleteWriteStatues.cache();
|
||||
JavaRDD<WriteStatus> deleteStatus = index.updateLocation(deleteWriteStatues, context(), hoodieTable);
|
||||
JavaRDD<WriteStatus> deleteStatus = updateLocation(index, deleteWriteStatues, hoodieTable);
|
||||
assertEquals(deleteStatus.count(), deleteWriteStatues.count());
|
||||
assertNoWriteErrors(deleteStatus.collect());
|
||||
|
||||
// Ensure no records can be tagged
|
||||
List<HoodieRecord> records3 = index.tagLocation(writeRecords, context(), hoodieTable).collect();
|
||||
List<HoodieRecord> records3 = tagLocation(index, writeRecords, hoodieTable).collect();
|
||||
assertEquals(0, records3.stream().filter(record -> record.isCurrentLocationKnown()).count());
|
||||
assertEquals(numRecords, records3.stream().map(record -> record.getKey().getRecordKey()).distinct().count());
|
||||
assertEquals(0, records3.stream().filter(record -> (record.getCurrentLocation() != null
|
||||
|
||||
@@ -135,7 +135,7 @@ public class TestHoodieIndex extends HoodieClientTestHarness {
|
||||
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
|
||||
// Test tagLocation without any entries in index
|
||||
JavaRDD<HoodieRecord> javaRDD = (JavaRDD<HoodieRecord>) index.tagLocation(writeRecords, context, hoodieTable);
|
||||
JavaRDD<HoodieRecord> javaRDD = tagLocation(index, writeRecords, hoodieTable);
|
||||
assert (javaRDD.filter(record -> record.isCurrentLocationKnown()).collect().size() == 0);
|
||||
|
||||
// Insert totalRecords records
|
||||
@@ -145,14 +145,14 @@ public class TestHoodieIndex extends HoodieClientTestHarness {
|
||||
|
||||
// Now tagLocation for these records, index should not tag them since it was a failed
|
||||
// commit
|
||||
javaRDD = (JavaRDD<HoodieRecord>) index.tagLocation(writeRecords, context, hoodieTable);
|
||||
javaRDD = tagLocation(index, writeRecords, hoodieTable);
|
||||
assert (javaRDD.filter(record -> record.isCurrentLocationKnown()).collect().size() == 0);
|
||||
// Now commit this & update location of records inserted and validate no errors
|
||||
writeClient.commit(newCommitTime, writeStatues);
|
||||
// Now tagLocation for these records, index should tag them correctly
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
javaRDD = (JavaRDD<HoodieRecord>) index.tagLocation(writeRecords, context, hoodieTable);
|
||||
javaRDD = tagLocation(index, writeRecords, hoodieTable);
|
||||
Map<String, String> recordKeyToPartitionPathMap = new HashMap();
|
||||
List<HoodieRecord> hoodieRecords = writeRecords.collect();
|
||||
hoodieRecords.forEach(entry -> recordKeyToPartitionPathMap.put(entry.getRecordKey(), entry.getPartitionPath()));
|
||||
@@ -185,7 +185,7 @@ public class TestHoodieIndex extends HoodieClientTestHarness {
|
||||
|
||||
writeClient.startCommitWithTime(newCommitTime);
|
||||
JavaRDD<WriteStatus> writeStatues = writeClient.upsert(writeRecords, newCommitTime);
|
||||
JavaRDD<HoodieRecord> javaRDD1 = (JavaRDD<HoodieRecord>) index.tagLocation(writeRecords, context, hoodieTable);
|
||||
JavaRDD<HoodieRecord> javaRDD1 = tagLocation(index, writeRecords, hoodieTable);
|
||||
|
||||
// Duplicate upsert and ensure correctness is maintained
|
||||
// We are trying to approximately imitate the case when the RDD is recomputed. For RDD creating, driver code is not
|
||||
@@ -201,7 +201,7 @@ public class TestHoodieIndex extends HoodieClientTestHarness {
|
||||
// Now tagLocation for these records, hbaseIndex should tag them correctly
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
JavaRDD<HoodieRecord> javaRDD = (JavaRDD<HoodieRecord>) index.tagLocation(writeRecords, context, hoodieTable);
|
||||
JavaRDD<HoodieRecord> javaRDD = tagLocation(index, writeRecords, hoodieTable);
|
||||
|
||||
Map<String, String> recordKeyToPartitionPathMap = new HashMap();
|
||||
List<HoodieRecord> hoodieRecords = writeRecords.collect();
|
||||
@@ -241,7 +241,7 @@ public class TestHoodieIndex extends HoodieClientTestHarness {
|
||||
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
|
||||
// Now tagLocation for these records, hbaseIndex should tag them
|
||||
JavaRDD<HoodieRecord> javaRDD = (JavaRDD<HoodieRecord>) index.tagLocation(writeRecords, context, hoodieTable);
|
||||
JavaRDD<HoodieRecord> javaRDD = tagLocation(index, writeRecords, hoodieTable);
|
||||
assert (javaRDD.filter(HoodieRecord::isCurrentLocationKnown).collect().size() == totalRecords);
|
||||
|
||||
// check tagged records are tagged with correct fileIds
|
||||
@@ -269,7 +269,7 @@ public class TestHoodieIndex extends HoodieClientTestHarness {
|
||||
hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
// Now tagLocation for these records, hbaseIndex should not tag them since it was a rolled
|
||||
// back commit
|
||||
javaRDD = (JavaRDD<HoodieRecord>) index.tagLocation(writeRecords, context, hoodieTable);
|
||||
javaRDD = tagLocation(index, writeRecords, hoodieTable);
|
||||
assert (javaRDD.filter(HoodieRecord::isCurrentLocationKnown).collect().size() == 0);
|
||||
assert (javaRDD.filter(record -> record.getCurrentLocation() != null).collect().size() == 0);
|
||||
}
|
||||
@@ -312,7 +312,7 @@ public class TestHoodieIndex extends HoodieClientTestHarness {
|
||||
|
||||
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
|
||||
JavaRDD<HoodieRecord> taggedRecordRDD = (JavaRDD<HoodieRecord>) index.tagLocation(recordRDD, context, hoodieTable);
|
||||
JavaRDD<HoodieRecord> taggedRecordRDD = tagLocation(index, recordRDD, hoodieTable);
|
||||
|
||||
// Should not find any files
|
||||
for (HoodieRecord record : taggedRecordRDD.collect()) {
|
||||
@@ -329,7 +329,7 @@ public class TestHoodieIndex extends HoodieClientTestHarness {
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
|
||||
taggedRecordRDD = (JavaRDD<HoodieRecord>) index.tagLocation(recordRDD, context, hoodieTable);
|
||||
taggedRecordRDD = tagLocation(index, recordRDD, hoodieTable);
|
||||
|
||||
// Check results
|
||||
for (HoodieRecord record : taggedRecordRDD.collect()) {
|
||||
@@ -419,7 +419,7 @@ public class TestHoodieIndex extends HoodieClientTestHarness {
|
||||
|
||||
// test against incoming record with a different partition
|
||||
JavaRDD<HoodieRecord> recordRDD = jsc.parallelize(Collections.singletonList(incomingRecord));
|
||||
JavaRDD<HoodieRecord> taggedRecordRDD = (JavaRDD<HoodieRecord>) index.tagLocation(recordRDD, context, hoodieTable);
|
||||
JavaRDD<HoodieRecord> taggedRecordRDD = tagLocation(index, recordRDD, hoodieTable);
|
||||
|
||||
assertEquals(2, taggedRecordRDD.count());
|
||||
for (HoodieRecord record : taggedRecordRDD.collect()) {
|
||||
@@ -440,7 +440,7 @@ public class TestHoodieIndex extends HoodieClientTestHarness {
|
||||
// test against incoming record with the same partition
|
||||
JavaRDD<HoodieRecord> recordRDDSamePartition = jsc
|
||||
.parallelize(Collections.singletonList(incomingRecordSamePartition));
|
||||
JavaRDD<HoodieRecord> taggedRecordRDDSamePartition = (JavaRDD<HoodieRecord>) index.tagLocation(recordRDDSamePartition, context, hoodieTable);
|
||||
JavaRDD<HoodieRecord> taggedRecordRDDSamePartition = tagLocation(index, recordRDDSamePartition, hoodieTable);
|
||||
|
||||
assertEquals(1, taggedRecordRDDSamePartition.count());
|
||||
HoodieRecord record = taggedRecordRDDSamePartition.first();
|
||||
@@ -463,8 +463,8 @@ public class TestHoodieIndex extends HoodieClientTestHarness {
|
||||
}
|
||||
|
||||
private JavaPairRDD<HoodieKey, Option<Pair<String, String>>> getRecordLocations(JavaRDD<HoodieKey> keyRDD, HoodieTable hoodieTable) {
|
||||
JavaRDD<HoodieRecord> recordRDD = (JavaRDD<HoodieRecord>) index.tagLocation(
|
||||
keyRDD.map(k -> new HoodieRecord(k, new EmptyHoodieRecordPayload())), context, hoodieTable);
|
||||
JavaRDD<HoodieRecord> recordRDD = tagLocation(
|
||||
index, keyRDD.map(k -> new HoodieRecord(k, new EmptyHoodieRecordPayload())), hoodieTable);
|
||||
return recordRDD.mapToPair(hr -> new Tuple2<>(hr.getKey(), hr.isCurrentLocationKnown()
|
||||
? Option.of(Pair.of(hr.getPartitionPath(), hr.getCurrentLocation().getFileId()))
|
||||
: Option.empty())
|
||||
|
||||
@@ -30,10 +30,11 @@ import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.exception.HoodieIndexException;
|
||||
import org.apache.hudi.index.HoodieIndex.IndexType;
|
||||
import org.apache.hudi.index.bloom.SparkHoodieBloomIndex;
|
||||
import org.apache.hudi.index.bloom.SparkHoodieGlobalBloomIndex;
|
||||
import org.apache.hudi.index.bloom.HoodieBloomIndex;
|
||||
import org.apache.hudi.index.bloom.HoodieGlobalBloomIndex;
|
||||
import org.apache.hudi.index.hbase.SparkHoodieHBaseIndex;
|
||||
import org.apache.hudi.index.simple.SparkHoodieSimpleIndex;
|
||||
import org.apache.hudi.index.inmemory.HoodieInMemoryHashIndex;
|
||||
import org.apache.hudi.index.simple.HoodieSimpleIndex;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
@@ -68,29 +69,29 @@ public class TestHoodieIndexConfigs {
|
||||
case INMEMORY:
|
||||
config = clientConfigBuilder.withPath(basePath)
|
||||
.withIndexConfig(indexConfigBuilder.withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build();
|
||||
assertTrue(SparkHoodieIndex.createIndex(config) instanceof SparkInMemoryHashIndex);
|
||||
assertTrue(SparkHoodieIndexFactory.createIndex(config) instanceof HoodieInMemoryHashIndex);
|
||||
break;
|
||||
case BLOOM:
|
||||
config = clientConfigBuilder.withPath(basePath)
|
||||
.withIndexConfig(indexConfigBuilder.withIndexType(HoodieIndex.IndexType.BLOOM).build()).build();
|
||||
assertTrue(SparkHoodieIndex.createIndex(config) instanceof SparkHoodieBloomIndex);
|
||||
assertTrue(SparkHoodieIndexFactory.createIndex(config) instanceof HoodieBloomIndex);
|
||||
break;
|
||||
case GLOBAL_BLOOM:
|
||||
config = clientConfigBuilder.withPath(basePath)
|
||||
.withIndexConfig(indexConfigBuilder.withIndexType(IndexType.GLOBAL_BLOOM).build()).build();
|
||||
assertTrue(SparkHoodieIndex.createIndex(config) instanceof SparkHoodieGlobalBloomIndex);
|
||||
assertTrue(SparkHoodieIndexFactory.createIndex(config) instanceof HoodieGlobalBloomIndex);
|
||||
break;
|
||||
case SIMPLE:
|
||||
config = clientConfigBuilder.withPath(basePath)
|
||||
.withIndexConfig(indexConfigBuilder.withIndexType(IndexType.SIMPLE).build()).build();
|
||||
assertTrue(SparkHoodieIndex.createIndex(config) instanceof SparkHoodieSimpleIndex);
|
||||
assertTrue(SparkHoodieIndexFactory.createIndex(config) instanceof HoodieSimpleIndex);
|
||||
break;
|
||||
case HBASE:
|
||||
config = clientConfigBuilder.withPath(basePath)
|
||||
.withIndexConfig(indexConfigBuilder.withIndexType(HoodieIndex.IndexType.HBASE)
|
||||
.withHBaseIndexConfig(new HoodieHBaseIndexConfig.Builder().build()).build())
|
||||
.build();
|
||||
assertTrue(SparkHoodieIndex.createIndex(config) instanceof SparkHoodieHBaseIndex);
|
||||
assertTrue(SparkHoodieIndexFactory.createIndex(config) instanceof SparkHoodieHBaseIndex);
|
||||
break;
|
||||
default:
|
||||
// no -op. just for checkstyle errors
|
||||
@@ -103,7 +104,7 @@ public class TestHoodieIndexConfigs {
|
||||
HoodieIndexConfig.Builder indexConfigBuilder = HoodieIndexConfig.newBuilder();
|
||||
HoodieWriteConfig config = clientConfigBuilder.withPath(basePath)
|
||||
.withIndexConfig(indexConfigBuilder.withIndexClass(DummyHoodieIndex.class.getName()).build()).build();
|
||||
assertTrue(SparkHoodieIndex.createIndex(config) instanceof DummyHoodieIndex);
|
||||
assertTrue(SparkHoodieIndexFactory.createIndex(config) instanceof DummyHoodieIndex);
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -113,14 +114,14 @@ public class TestHoodieIndexConfigs {
|
||||
final HoodieWriteConfig config1 = clientConfigBuilder.withPath(basePath)
|
||||
.withIndexConfig(indexConfigBuilder.withIndexClass(IndexWithConstructor.class.getName()).build()).build();
|
||||
final Throwable thrown1 = assertThrows(HoodieException.class, () -> {
|
||||
SparkHoodieIndex.createIndex(config1);
|
||||
SparkHoodieIndexFactory.createIndex(config1);
|
||||
}, "exception is expected");
|
||||
assertTrue(thrown1.getMessage().contains("is not a subclass of HoodieIndex"));
|
||||
|
||||
final HoodieWriteConfig config2 = clientConfigBuilder.withPath(basePath)
|
||||
.withIndexConfig(indexConfigBuilder.withIndexClass(IndexWithoutConstructor.class.getName()).build()).build();
|
||||
final Throwable thrown2 = assertThrows(HoodieException.class, () -> {
|
||||
SparkHoodieIndex.createIndex(config2);
|
||||
SparkHoodieIndexFactory.createIndex(config2);
|
||||
}, "exception is expected");
|
||||
assertTrue(thrown2.getMessage().contains("Unable to instantiate class"));
|
||||
}
|
||||
|
||||
@@ -26,9 +26,12 @@ import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.testutils.RawTripTestPayload;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.collection.ImmutablePair;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieIndexConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.data.HoodieJavaPairRDD;
|
||||
import org.apache.hudi.data.HoodieJavaRDD;
|
||||
import org.apache.hudi.io.HoodieKeyLookupHandle;
|
||||
import org.apache.hudi.table.HoodieSparkTable;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
@@ -103,7 +106,7 @@ public class TestHoodieBloomIndex extends HoodieClientTestHarness {
|
||||
@MethodSource("configParams")
|
||||
public void testLoadInvolvedFiles(boolean rangePruning, boolean treeFiltering, boolean bucketizedChecking) throws Exception {
|
||||
HoodieWriteConfig config = makeConfig(rangePruning, treeFiltering, bucketizedChecking);
|
||||
SparkHoodieBloomIndex index = new SparkHoodieBloomIndex(config);
|
||||
HoodieBloomIndex index = new HoodieBloomIndex(config, SparkHoodieBloomIndexHelper.getInstance());
|
||||
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(hoodieTable, SCHEMA);
|
||||
|
||||
@@ -131,7 +134,7 @@ public class TestHoodieBloomIndex extends HoodieClientTestHarness {
|
||||
new HoodieRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4);
|
||||
|
||||
List<String> partitions = Arrays.asList("2016/01/21", "2016/04/01", "2015/03/12");
|
||||
List<Tuple2<String, BloomIndexFileInfo>> filesList = index.loadInvolvedFiles(partitions, context, hoodieTable);
|
||||
List<ImmutablePair<String, BloomIndexFileInfo>> filesList = index.loadInvolvedFiles(partitions, context, hoodieTable);
|
||||
// Still 0, as no valid commit
|
||||
assertEquals(0, filesList.size());
|
||||
|
||||
@@ -145,20 +148,20 @@ public class TestHoodieBloomIndex extends HoodieClientTestHarness {
|
||||
|
||||
if (rangePruning) {
|
||||
// these files will not have the key ranges
|
||||
assertNull(filesList.get(0)._2().getMaxRecordKey());
|
||||
assertNull(filesList.get(0)._2().getMinRecordKey());
|
||||
assertFalse(filesList.get(1)._2().hasKeyRanges());
|
||||
assertNotNull(filesList.get(2)._2().getMaxRecordKey());
|
||||
assertNotNull(filesList.get(2)._2().getMinRecordKey());
|
||||
assertTrue(filesList.get(3)._2().hasKeyRanges());
|
||||
assertNull(filesList.get(0).getRight().getMaxRecordKey());
|
||||
assertNull(filesList.get(0).getRight().getMinRecordKey());
|
||||
assertFalse(filesList.get(1).getRight().hasKeyRanges());
|
||||
assertNotNull(filesList.get(2).getRight().getMaxRecordKey());
|
||||
assertNotNull(filesList.get(2).getRight().getMinRecordKey());
|
||||
assertTrue(filesList.get(3).getRight().hasKeyRanges());
|
||||
|
||||
// no longer sorted, but should have same files.
|
||||
|
||||
List<Tuple2<String, BloomIndexFileInfo>> expected =
|
||||
Arrays.asList(new Tuple2<>("2016/04/01", new BloomIndexFileInfo("2")),
|
||||
new Tuple2<>("2015/03/12", new BloomIndexFileInfo("1")),
|
||||
new Tuple2<>("2015/03/12", new BloomIndexFileInfo("3", "000", "000")),
|
||||
new Tuple2<>("2015/03/12", new BloomIndexFileInfo("4", "001", "003")));
|
||||
List<ImmutablePair<String, BloomIndexFileInfo>> expected =
|
||||
Arrays.asList(new ImmutablePair<>("2016/04/01", new BloomIndexFileInfo("2")),
|
||||
new ImmutablePair<>("2015/03/12", new BloomIndexFileInfo("1")),
|
||||
new ImmutablePair<>("2015/03/12", new BloomIndexFileInfo("3", "000", "000")),
|
||||
new ImmutablePair<>("2015/03/12", new BloomIndexFileInfo("4", "001", "003")));
|
||||
assertEquals(expected, filesList);
|
||||
}
|
||||
}
|
||||
@@ -167,7 +170,7 @@ public class TestHoodieBloomIndex extends HoodieClientTestHarness {
|
||||
@MethodSource("configParams")
|
||||
public void testRangePruning(boolean rangePruning, boolean treeFiltering, boolean bucketizedChecking) {
|
||||
HoodieWriteConfig config = makeConfig(rangePruning, treeFiltering, bucketizedChecking);
|
||||
SparkHoodieBloomIndex index = new SparkHoodieBloomIndex(config);
|
||||
HoodieBloomIndex index = new HoodieBloomIndex(config, SparkHoodieBloomIndexHelper.getInstance());
|
||||
|
||||
final Map<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo = new HashMap<>();
|
||||
partitionToFileIndexInfo.put("2017/10/22",
|
||||
@@ -179,12 +182,12 @@ public class TestHoodieBloomIndex extends HoodieClientTestHarness {
|
||||
jsc.parallelize(Arrays.asList(new Tuple2<>("2017/10/22", "003"), new Tuple2<>("2017/10/22", "002"),
|
||||
new Tuple2<>("2017/10/22", "005"), new Tuple2<>("2017/10/22", "004"))).mapToPair(t -> t);
|
||||
|
||||
List<Tuple2<String, HoodieKey>> comparisonKeyList =
|
||||
index.explodeRecordRDDWithFileComparisons(partitionToFileIndexInfo, partitionRecordKeyPairRDD).collect();
|
||||
List<Pair<String, HoodieKey>> comparisonKeyList = HoodieJavaRDD.getJavaRDD(
|
||||
index.explodeRecordsWithFileComparisons(partitionToFileIndexInfo, HoodieJavaPairRDD.of(partitionRecordKeyPairRDD))).collect();
|
||||
|
||||
assertEquals(10, comparisonKeyList.size());
|
||||
Map<String, List<String>> recordKeyToFileComps = comparisonKeyList.stream()
|
||||
.collect(Collectors.groupingBy(t -> t._2.getRecordKey(), Collectors.mapping(t -> t._1, Collectors.toList())));
|
||||
.collect(Collectors.groupingBy(t -> t.getRight().getRecordKey(), Collectors.mapping(Pair::getLeft, Collectors.toList())));
|
||||
|
||||
assertEquals(4, recordKeyToFileComps.size());
|
||||
assertEquals(new HashSet<>(Arrays.asList("f1", "f3", "f4")), new HashSet<>(recordKeyToFileComps.get("002")));
|
||||
@@ -262,10 +265,10 @@ public class TestHoodieBloomIndex extends HoodieClientTestHarness {
|
||||
HoodieSparkTable table = HoodieSparkTable.create(config, context, metaClient);
|
||||
|
||||
// Let's tag
|
||||
SparkHoodieBloomIndex bloomIndex = new SparkHoodieBloomIndex(config);
|
||||
HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, SparkHoodieBloomIndexHelper.getInstance());
|
||||
|
||||
assertDoesNotThrow(() -> {
|
||||
bloomIndex.tagLocation(recordRDD, context, table);
|
||||
tagLocation(bloomIndex, recordRDD, table);
|
||||
}, "EmptyRDD should not result in IllegalArgumentException: Positive number of slices required");
|
||||
}
|
||||
|
||||
@@ -301,8 +304,8 @@ public class TestHoodieBloomIndex extends HoodieClientTestHarness {
|
||||
HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(hoodieTable, SCHEMA);
|
||||
|
||||
// Let's tag
|
||||
SparkHoodieBloomIndex bloomIndex = new SparkHoodieBloomIndex(config);
|
||||
JavaRDD<HoodieRecord> taggedRecordRDD = bloomIndex.tagLocation(recordRDD, context, hoodieTable);
|
||||
HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, SparkHoodieBloomIndexHelper.getInstance());
|
||||
JavaRDD<HoodieRecord> taggedRecordRDD = tagLocation(bloomIndex, recordRDD, hoodieTable);
|
||||
|
||||
// Should not find any files
|
||||
for (HoodieRecord record : taggedRecordRDD.collect()) {
|
||||
@@ -315,7 +318,7 @@ public class TestHoodieBloomIndex extends HoodieClientTestHarness {
|
||||
String fileId3 = testTable.addCommit("003").getFileIdWithInserts("2015/01/31", record4);
|
||||
|
||||
// We do the tag again
|
||||
taggedRecordRDD = bloomIndex.tagLocation(recordRDD, context, HoodieSparkTable.create(config, context, metaClient));
|
||||
taggedRecordRDD = tagLocation(bloomIndex, recordRDD, HoodieSparkTable.create(config, context, metaClient));
|
||||
|
||||
// Check results
|
||||
for (HoodieRecord record : taggedRecordRDD.collect()) {
|
||||
@@ -366,8 +369,9 @@ public class TestHoodieBloomIndex extends HoodieClientTestHarness {
|
||||
HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(hoodieTable, SCHEMA);
|
||||
|
||||
// Let's tag
|
||||
SparkHoodieBloomIndex bloomIndex = new SparkHoodieBloomIndex(config);
|
||||
JavaRDD<HoodieRecord> taggedRecords = bloomIndex.tagLocation(keysRDD.map(k -> new HoodieRecord(k, null)), context, hoodieTable);
|
||||
HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, SparkHoodieBloomIndexHelper.getInstance());
|
||||
JavaRDD<HoodieRecord> taggedRecords = tagLocation(
|
||||
bloomIndex, keysRDD.map(k -> new HoodieRecord(k, null)), hoodieTable);
|
||||
JavaPairRDD<HoodieKey, Option<Pair<String, String>>> recordLocationsRDD = taggedRecords
|
||||
.mapToPair(hr -> new Tuple2<>(hr.getKey(), hr.isCurrentLocationKnown()
|
||||
? Option.of(Pair.of(hr.getPartitionPath(), hr.getCurrentLocation().getFileId()))
|
||||
@@ -387,7 +391,7 @@ public class TestHoodieBloomIndex extends HoodieClientTestHarness {
|
||||
// We do the tag again
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
taggedRecords = bloomIndex.tagLocation(keysRDD.map(k -> new HoodieRecord(k, null)), context, hoodieTable);
|
||||
taggedRecords = tagLocation(bloomIndex, keysRDD.map(k -> new HoodieRecord(k, null)), hoodieTable);
|
||||
recordLocationsRDD = taggedRecords
|
||||
.mapToPair(hr -> new Tuple2<>(hr.getKey(), hr.isCurrentLocationKnown()
|
||||
? Option.of(Pair.of(hr.getPartitionPath(), hr.getCurrentLocation().getFileId()))
|
||||
@@ -443,8 +447,8 @@ public class TestHoodieBloomIndex extends HoodieClientTestHarness {
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTable table = HoodieSparkTable.create(config, context, metaClient);
|
||||
|
||||
SparkHoodieBloomIndex bloomIndex = new SparkHoodieBloomIndex(config);
|
||||
JavaRDD<HoodieRecord> taggedRecordRDD = bloomIndex.tagLocation(recordRDD, context, table);
|
||||
HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, SparkHoodieBloomIndexHelper.getInstance());
|
||||
JavaRDD<HoodieRecord> taggedRecordRDD = tagLocation(bloomIndex, recordRDD, table);
|
||||
|
||||
// Check results
|
||||
for (HoodieRecord record : taggedRecordRDD.collect()) {
|
||||
|
||||
@@ -22,8 +22,12 @@ import org.apache.hudi.common.model.EmptyHoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.testutils.RawTripTestPayload;
|
||||
import org.apache.hudi.common.util.collection.ImmutablePair;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieIndexConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.data.HoodieJavaPairRDD;
|
||||
import org.apache.hudi.data.HoodieJavaRDD;
|
||||
import org.apache.hudi.table.HoodieSparkTable;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.testutils.HoodieClientTestHarness;
|
||||
@@ -74,7 +78,8 @@ public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness {
|
||||
@Test
|
||||
public void testLoadInvolvedFiles() throws Exception {
|
||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
|
||||
SparkHoodieGlobalBloomIndex index = new SparkHoodieGlobalBloomIndex(config);
|
||||
HoodieGlobalBloomIndex index =
|
||||
new HoodieGlobalBloomIndex(config, SparkHoodieBloomIndexHelper.getInstance());
|
||||
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(hoodieTable, SCHEMA);
|
||||
|
||||
@@ -104,7 +109,7 @@ public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness {
|
||||
// intentionally missed the partition "2015/03/12" to see if the GlobalBloomIndex can pick it up
|
||||
List<String> partitions = Arrays.asList("2016/01/21", "2016/04/01");
|
||||
// partitions will NOT be respected by this loadInvolvedFiles(...) call
|
||||
List<Tuple2<String, BloomIndexFileInfo>> filesList = index.loadInvolvedFiles(partitions, context, hoodieTable);
|
||||
List<Pair<String, BloomIndexFileInfo>> filesList = index.loadInvolvedFiles(partitions, context, hoodieTable);
|
||||
// Still 0, as no valid commit
|
||||
assertEquals(0, filesList.size());
|
||||
|
||||
@@ -138,7 +143,8 @@ public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness {
|
||||
public void testExplodeRecordRDDWithFileComparisons() {
|
||||
|
||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
|
||||
SparkHoodieGlobalBloomIndex index = new SparkHoodieGlobalBloomIndex(config);
|
||||
HoodieGlobalBloomIndex index =
|
||||
new HoodieGlobalBloomIndex(config, SparkHoodieBloomIndexHelper.getInstance());
|
||||
|
||||
final Map<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo = new HashMap<>();
|
||||
partitionToFileIndexInfo.put("2017/10/22", Arrays.asList(new BloomIndexFileInfo("f1"),
|
||||
@@ -152,8 +158,9 @@ public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness {
|
||||
jsc.parallelize(Arrays.asList(new Tuple2<>("2017/10/21", "003"), new Tuple2<>("2017/10/22", "002"),
|
||||
new Tuple2<>("2017/10/22", "005"), new Tuple2<>("2017/10/23", "004"))).mapToPair(t -> t);
|
||||
|
||||
List<Tuple2<String, HoodieKey>> comparisonKeyList =
|
||||
index.explodeRecordRDDWithFileComparisons(partitionToFileIndexInfo, partitionRecordKeyPairRDD).collect();
|
||||
List<ImmutablePair<String, HoodieKey>> comparisonKeyList = HoodieJavaRDD.getJavaRDD(
|
||||
index.explodeRecordsWithFileComparisons(partitionToFileIndexInfo,
|
||||
HoodieJavaPairRDD.of(partitionRecordKeyPairRDD))).collect();
|
||||
|
||||
/*
|
||||
* expecting: f4, HoodieKey { recordKey=003 partitionPath=2017/10/23} f1, HoodieKey { recordKey=003
|
||||
@@ -166,7 +173,7 @@ public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness {
|
||||
assertEquals(10, comparisonKeyList.size());
|
||||
|
||||
Map<String, List<String>> recordKeyToFileComps = comparisonKeyList.stream()
|
||||
.collect(Collectors.groupingBy(t -> t._2.getRecordKey(), Collectors.mapping(Tuple2::_1, Collectors.toList())));
|
||||
.collect(Collectors.groupingBy(t -> t.getRight().getRecordKey(), Collectors.mapping(Pair::getKey, Collectors.toList())));
|
||||
|
||||
assertEquals(4, recordKeyToFileComps.size());
|
||||
assertEquals(new HashSet<>(Arrays.asList("f4", "f1", "f3")), new HashSet<>(recordKeyToFileComps.get("002")));
|
||||
@@ -179,7 +186,8 @@ public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness {
|
||||
public void testTagLocation() throws Exception {
|
||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath)
|
||||
.withIndexConfig(HoodieIndexConfig.newBuilder().withBloomIndexUpdatePartitionPath(false).build()).build();
|
||||
SparkHoodieGlobalBloomIndex index = new SparkHoodieGlobalBloomIndex(config);
|
||||
HoodieGlobalBloomIndex index =
|
||||
new HoodieGlobalBloomIndex(config, SparkHoodieBloomIndexHelper.getInstance());
|
||||
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(hoodieTable, SCHEMA);
|
||||
|
||||
@@ -224,7 +232,7 @@ public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness {
|
||||
String fileId4 = testTable.addCommit("4000").getFileIdWithInserts("2015/03/12", record4);
|
||||
|
||||
// partitions will NOT be respected by this loadInvolvedFiles(...) call
|
||||
JavaRDD<HoodieRecord> taggedRecordRDD = index.tagLocation(recordRDD, context, hoodieTable);
|
||||
JavaRDD<HoodieRecord> taggedRecordRDD = tagLocation(index, recordRDD, hoodieTable);
|
||||
|
||||
for (HoodieRecord record : taggedRecordRDD.collect()) {
|
||||
switch (record.getRecordKey()) {
|
||||
@@ -260,7 +268,8 @@ public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness {
|
||||
.withPath(basePath)
|
||||
.withIndexConfig(HoodieIndexConfig.newBuilder().withBloomIndexUpdatePartitionPath(true).build())
|
||||
.build();
|
||||
SparkHoodieGlobalBloomIndex index = new SparkHoodieGlobalBloomIndex(config);
|
||||
HoodieGlobalBloomIndex index =
|
||||
new HoodieGlobalBloomIndex(config, SparkHoodieBloomIndexHelper.getInstance());
|
||||
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(hoodieTable, SCHEMA);
|
||||
final String p1 = "2016/01/31";
|
||||
@@ -304,7 +313,7 @@ public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness {
|
||||
|
||||
// test against incoming record with a different partition
|
||||
JavaRDD<HoodieRecord> recordRDD = jsc.parallelize(Collections.singletonList(incomingRecord));
|
||||
JavaRDD<HoodieRecord> taggedRecordRDD = index.tagLocation(recordRDD, context, hoodieTable);
|
||||
JavaRDD<HoodieRecord> taggedRecordRDD = tagLocation(index, recordRDD, hoodieTable);
|
||||
|
||||
assertEquals(2, taggedRecordRDD.count());
|
||||
for (HoodieRecord record : taggedRecordRDD.collect()) {
|
||||
@@ -325,7 +334,7 @@ public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness {
|
||||
// test against incoming record with the same partition
|
||||
JavaRDD<HoodieRecord> recordRDDSamePartition = jsc
|
||||
.parallelize(Collections.singletonList(incomingRecordSamePartition));
|
||||
JavaRDD<HoodieRecord> taggedRecordRDDSamePartition = index.tagLocation(recordRDDSamePartition, context, hoodieTable);
|
||||
JavaRDD<HoodieRecord> taggedRecordRDDSamePartition = tagLocation(index, recordRDDSamePartition, hoodieTable);
|
||||
|
||||
assertEquals(1, taggedRecordRDDSamePartition.count());
|
||||
HoodieRecord record = taggedRecordRDDSamePartition.first();
|
||||
@@ -335,10 +344,10 @@ public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness {
|
||||
}
|
||||
|
||||
// convert list to map to avoid sorting order dependencies
|
||||
private static Map<String, BloomIndexFileInfo> toFileMap(List<Tuple2<String, BloomIndexFileInfo>> filesList) {
|
||||
private static Map<String, BloomIndexFileInfo> toFileMap(List<Pair<String, BloomIndexFileInfo>> filesList) {
|
||||
Map<String, BloomIndexFileInfo> filesMap = new HashMap<>();
|
||||
for (Tuple2<String, BloomIndexFileInfo> t : filesList) {
|
||||
filesMap.put(t._1() + "/" + t._2().getFileId(), t._2());
|
||||
for (Pair<String, BloomIndexFileInfo> t : filesList) {
|
||||
filesMap.put(t.getKey() + "/" + t.getValue().getFileId(), t.getValue());
|
||||
}
|
||||
return filesMap;
|
||||
}
|
||||
|
||||
@@ -76,7 +76,7 @@ import org.apache.hudi.config.HoodieCompactionConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.index.SparkHoodieIndex;
|
||||
import org.apache.hudi.index.SparkHoodieIndexFactory;
|
||||
import org.apache.hudi.metadata.HoodieTableMetadataWriter;
|
||||
import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter;
|
||||
import org.apache.hudi.table.action.clean.CleanPlanner;
|
||||
@@ -169,8 +169,8 @@ public class TestCleaner extends HoodieClientTestBase {
|
||||
assertTrue(table.getCompletedCleanTimeline().empty());
|
||||
|
||||
if (client.getConfig().shouldAutoCommit()) {
|
||||
HoodieIndex index = SparkHoodieIndex.createIndex(cfg);
|
||||
List<HoodieRecord> taggedRecords = ((JavaRDD<HoodieRecord>) index.tagLocation(jsc.parallelize(records, 1), context, table)).collect();
|
||||
HoodieIndex index = SparkHoodieIndexFactory.createIndex(cfg);
|
||||
List<HoodieRecord> taggedRecords = tagLocation(index, jsc.parallelize(records, 1), table).collect();
|
||||
checkTaggedRecords(taggedRecords, newCommitTime);
|
||||
}
|
||||
return Pair.of(newCommitTime, statuses);
|
||||
|
||||
@@ -23,7 +23,8 @@ import org.apache.hudi.common.model.EmptyHoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.index.bloom.SparkHoodieBloomIndex;
|
||||
import org.apache.hudi.data.HoodieJavaRDD;
|
||||
import org.apache.hudi.index.bloom.HoodieBloomIndex;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
||||
|
||||
@@ -64,13 +65,20 @@ public class TestDeleteHelper {
|
||||
private static final boolean WITHOUT_COMBINE = false;
|
||||
private static final int DELETE_PARALLELISM = 200;
|
||||
|
||||
@Mock private SparkHoodieBloomIndex index;
|
||||
@Mock private HoodieTable<EmptyHoodieRecordPayload,JavaRDD<HoodieRecord>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table;
|
||||
@Mock private BaseSparkCommitActionExecutor<EmptyHoodieRecordPayload> executor;
|
||||
@Mock private HoodieWriteMetadata metadata;
|
||||
@Mock private JavaPairRDD keyPairs;
|
||||
@Mock private JavaSparkContext jsc;
|
||||
@Mock private HoodieSparkEngineContext context;
|
||||
@Mock
|
||||
private HoodieBloomIndex index;
|
||||
@Mock
|
||||
private HoodieTable<EmptyHoodieRecordPayload, JavaRDD<HoodieRecord>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table;
|
||||
@Mock
|
||||
private BaseSparkCommitActionExecutor<EmptyHoodieRecordPayload> executor;
|
||||
@Mock
|
||||
private HoodieWriteMetadata metadata;
|
||||
@Mock
|
||||
private JavaPairRDD keyPairs;
|
||||
@Mock
|
||||
private JavaSparkContext jsc;
|
||||
@Mock
|
||||
private HoodieSparkEngineContext context;
|
||||
|
||||
private JavaRDD<HoodieKey> rddToDelete;
|
||||
private HoodieWriteConfig config;
|
||||
@@ -149,7 +157,7 @@ public class TestDeleteHelper {
|
||||
JavaRDD recordsRdd = mock(JavaRDD.class);
|
||||
when(recordsRdd.filter(any())).thenReturn(recordsRdd);
|
||||
when(recordsRdd.isEmpty()).thenReturn(howMany <= 0);
|
||||
when(index.tagLocation(any(), any(), any())).thenReturn(recordsRdd);
|
||||
when(index.tagLocation(any(), any(), any())).thenReturn(HoodieJavaRDD.of(recordsRdd));
|
||||
|
||||
if (combineMode == CombineTestMode.GlobalIndex) {
|
||||
when(keyPairs.reduceByKey(any(), anyInt())).thenReturn(keyPairs);
|
||||
@@ -175,7 +183,7 @@ public class TestDeleteHelper {
|
||||
doReturn(Collections.emptyList()).when(emptyRdd).partitions();
|
||||
doReturn(emptyRdd).when(emptyRdd).map(any());
|
||||
|
||||
doReturn(emptyRdd).when(index).tagLocation(any(), any(), any());
|
||||
doReturn(HoodieJavaRDD.of(emptyRdd)).when(index).tagLocation(any(), any(), any());
|
||||
doReturn(emptyRdd).when(emptyRdd).filter(any());
|
||||
|
||||
doNothing().when(executor).saveWorkloadProfileMetadataToInflight(any(), anyString());
|
||||
|
||||
@@ -41,7 +41,8 @@ import org.apache.hudi.config.HoodieStorageConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieNotSupportedException;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.index.bloom.SparkHoodieBloomIndex;
|
||||
import org.apache.hudi.index.bloom.HoodieBloomIndex;
|
||||
import org.apache.hudi.index.bloom.SparkHoodieBloomIndexHelper;
|
||||
import org.apache.hudi.table.HoodieSparkTable;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.testutils.HoodieClientTestHarness;
|
||||
@@ -179,8 +180,8 @@ public class TestHoodieCompactor extends HoodieClientTestHarness {
|
||||
|
||||
List<HoodieRecord> updatedRecords = dataGen.generateUpdates(newCommitTime, records);
|
||||
JavaRDD<HoodieRecord> updatedRecordsRDD = jsc.parallelize(updatedRecords, 1);
|
||||
HoodieIndex index = new SparkHoodieBloomIndex<>(config);
|
||||
updatedRecords = ((JavaRDD<HoodieRecord>)index.tagLocation(updatedRecordsRDD, context, table)).collect();
|
||||
HoodieIndex index = new HoodieBloomIndex<>(config, SparkHoodieBloomIndexHelper.getInstance());
|
||||
updatedRecords = tagLocation(index, updatedRecordsRDD, table).collect();
|
||||
|
||||
// Write them to corresponding avro logfiles. Also, set the state transition properly.
|
||||
HoodieSparkWriteableTestTable.of(table, HoodieTestDataGenerator.AVRO_SCHEMA_WITH_METADATA_FIELDS)
|
||||
|
||||
@@ -41,8 +41,9 @@ import org.apache.hudi.config.HoodieCompactionConfig;
|
||||
import org.apache.hudi.config.HoodieIndexConfig;
|
||||
import org.apache.hudi.config.HoodieStorageConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.index.HoodieIndex.IndexType;
|
||||
import org.apache.hudi.index.SparkHoodieIndex;
|
||||
import org.apache.hudi.index.SparkHoodieIndexFactory;
|
||||
import org.apache.hudi.table.HoodieSparkTable;
|
||||
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
@@ -234,11 +235,11 @@ public class HoodieClientTestBase extends HoodieClientTestHarness {
|
||||
private Function2<List<HoodieRecord>, String, Integer> wrapRecordsGenFunctionForPreppedCalls(
|
||||
final HoodieWriteConfig writeConfig, final Function2<List<HoodieRecord>, String, Integer> recordGenFunction) {
|
||||
return (commit, numRecords) -> {
|
||||
final SparkHoodieIndex index = SparkHoodieIndex.createIndex(writeConfig);
|
||||
final HoodieIndex index = SparkHoodieIndexFactory.createIndex(writeConfig);
|
||||
List<HoodieRecord> records = recordGenFunction.apply(commit, numRecords);
|
||||
final HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(basePath).setLoadActiveTimelineOnLoad(true).build();
|
||||
HoodieSparkTable table = HoodieSparkTable.create(writeConfig, context, metaClient);
|
||||
JavaRDD<HoodieRecord> taggedRecords = index.tagLocation(jsc.parallelize(records, 1), context, table);
|
||||
JavaRDD<HoodieRecord> taggedRecords = tagLocation(index, jsc.parallelize(records, 1), table);
|
||||
return taggedRecords.collect();
|
||||
};
|
||||
}
|
||||
@@ -255,13 +256,13 @@ public class HoodieClientTestBase extends HoodieClientTestHarness {
|
||||
private Function<Integer, List<HoodieKey>> wrapDeleteKeysGenFunctionForPreppedCalls(
|
||||
final HoodieWriteConfig writeConfig, final Function<Integer, List<HoodieKey>> keyGenFunction) {
|
||||
return (numRecords) -> {
|
||||
final SparkHoodieIndex index = SparkHoodieIndex.createIndex(writeConfig);
|
||||
final HoodieIndex index = SparkHoodieIndexFactory.createIndex(writeConfig);
|
||||
List<HoodieKey> records = keyGenFunction.apply(numRecords);
|
||||
final HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(basePath).setLoadActiveTimelineOnLoad(true).build();
|
||||
HoodieSparkTable table = HoodieSparkTable.create(writeConfig, context, metaClient);
|
||||
JavaRDD<HoodieRecord> recordsToDelete = jsc.parallelize(records, 1)
|
||||
.map(key -> new HoodieRecord(key, new EmptyHoodieRecordPayload()));
|
||||
JavaRDD<HoodieRecord> taggedRecords = index.tagLocation(recordsToDelete, context, table);
|
||||
JavaRDD<HoodieRecord> taggedRecords = tagLocation(index, recordsToDelete, table);
|
||||
return taggedRecords.map(record -> record.getKey()).collect();
|
||||
};
|
||||
}
|
||||
|
||||
@@ -43,6 +43,7 @@ import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieIndexConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.data.HoodieJavaRDD;
|
||||
import org.apache.hudi.exception.HoodieMetadataException;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.keygen.SimpleKeyGenerator;
|
||||
@@ -413,6 +414,12 @@ public abstract class HoodieClientTestHarness extends HoodieCommonTestHarness im
|
||||
return tableView;
|
||||
}
|
||||
|
||||
public JavaRDD<HoodieRecord> tagLocation(
|
||||
HoodieIndex index, JavaRDD<HoodieRecord> records, HoodieTable table) {
|
||||
return HoodieJavaRDD.getJavaRDD(
|
||||
index.tagLocation(HoodieJavaRDD.of(records), context, table));
|
||||
}
|
||||
|
||||
public static Pair<HashMap<String, WorkloadStat>, WorkloadStat> buildProfile(JavaRDD<HoodieRecord> inputRecordsRDD) {
|
||||
HashMap<String, WorkloadStat> partitionPathStatMap = new HashMap<>();
|
||||
WorkloadStat globalStat = new WorkloadStat();
|
||||
|
||||
@@ -41,6 +41,7 @@ import org.apache.hudi.config.HoodieCompactionConfig;
|
||||
import org.apache.hudi.config.HoodieIndexConfig;
|
||||
import org.apache.hudi.config.HoodieStorageConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.data.HoodieJavaRDD;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.keygen.SimpleKeyGenerator;
|
||||
import org.apache.hudi.table.HoodieSparkTable;
|
||||
@@ -190,6 +191,18 @@ public class SparkClientFunctionalTestHarness implements SparkProvider, HoodieMe
|
||||
}
|
||||
}
|
||||
|
||||
protected JavaRDD<HoodieRecord> tagLocation(
|
||||
HoodieIndex index, JavaRDD<HoodieRecord> records, HoodieTable table) {
|
||||
return HoodieJavaRDD.getJavaRDD(
|
||||
index.tagLocation(HoodieJavaRDD.of(records), context, table));
|
||||
}
|
||||
|
||||
protected JavaRDD<WriteStatus> updateLocation(
|
||||
HoodieIndex index, JavaRDD<WriteStatus> writeStatus, HoodieTable table) {
|
||||
return HoodieJavaRDD.getJavaRDD(
|
||||
index.updateLocation(HoodieJavaRDD.of(writeStatus), context, table));
|
||||
}
|
||||
|
||||
protected void insertRecords(HoodieTableMetaClient metaClient, List<HoodieRecord> records, SparkRDDWriteClient client, HoodieWriteConfig cfg, String commitTime) throws IOException {
|
||||
HoodieTableMetaClient reloadedMetaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
|
||||
|
||||
Reference in New Issue
Block a user