[HUDI-652] Decouple HoodieReadClient and AbstractHoodieClient to break the inheritance chain (#1372)
* Removed timeline server support * Removed try-with-resource
This commit is contained in:
@@ -96,8 +96,8 @@ public class TestHoodieReadClient extends TestHoodieClientBase {
|
||||
*/
|
||||
private void testReadFilterExist(HoodieWriteConfig config,
|
||||
Function3<JavaRDD<WriteStatus>, HoodieWriteClient, JavaRDD<HoodieRecord>, String> writeFn) throws Exception {
|
||||
try (HoodieWriteClient writeClient = getHoodieWriteClient(config);
|
||||
HoodieReadClient readClient = getHoodieReadClient(config.getBasePath());) {
|
||||
try (HoodieWriteClient writeClient = getHoodieWriteClient(config);) {
|
||||
HoodieReadClient readClient = getHoodieReadClient(config.getBasePath());
|
||||
String newCommitTime = writeClient.startCommit();
|
||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 100);
|
||||
JavaRDD<HoodieRecord> recordsRDD = jsc.parallelize(records, 1);
|
||||
@@ -113,37 +113,36 @@ public class TestHoodieReadClient extends TestHoodieClientBase {
|
||||
// Verify there are no errors
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
try (HoodieReadClient anotherReadClient = getHoodieReadClient(config.getBasePath());) {
|
||||
filteredRDD = anotherReadClient.filterExists(recordsRDD);
|
||||
List<HoodieRecord> result = filteredRDD.collect();
|
||||
// Check results
|
||||
assertEquals(25, result.size());
|
||||
HoodieReadClient anotherReadClient = getHoodieReadClient(config.getBasePath());
|
||||
filteredRDD = anotherReadClient.filterExists(recordsRDD);
|
||||
List<HoodieRecord> result = filteredRDD.collect();
|
||||
// Check results
|
||||
assertEquals(25, result.size());
|
||||
|
||||
// check path exists for written keys
|
||||
JavaPairRDD<HoodieKey, Option<String>> keyToPathPair =
|
||||
anotherReadClient.checkExists(recordsRDD.map(r -> r.getKey()));
|
||||
JavaRDD<HoodieKey> keysWithPaths = keyToPathPair.filter(keyPath -> keyPath._2.isPresent())
|
||||
.map(keyPath -> keyPath._1);
|
||||
assertEquals(75, keysWithPaths.count());
|
||||
// check path exists for written keys
|
||||
JavaPairRDD<HoodieKey, Option<String>> keyToPathPair =
|
||||
anotherReadClient.checkExists(recordsRDD.map(r -> r.getKey()));
|
||||
JavaRDD<HoodieKey> keysWithPaths = keyToPathPair.filter(keyPath -> keyPath._2.isPresent())
|
||||
.map(keyPath -> keyPath._1);
|
||||
assertEquals(75, keysWithPaths.count());
|
||||
|
||||
// verify rows match inserted records
|
||||
Dataset<Row> rows = anotherReadClient.readROView(keysWithPaths, 1);
|
||||
assertEquals(75, rows.count());
|
||||
// verify rows match inserted records
|
||||
Dataset<Row> rows = anotherReadClient.readROView(keysWithPaths, 1);
|
||||
assertEquals(75, rows.count());
|
||||
|
||||
JavaRDD<HoodieKey> keysWithoutPaths = keyToPathPair.filter(keyPath -> !keyPath._2.isPresent())
|
||||
.map(keyPath -> keyPath._1);
|
||||
JavaRDD<HoodieKey> keysWithoutPaths = keyToPathPair.filter(keyPath -> !keyPath._2.isPresent())
|
||||
.map(keyPath -> keyPath._1);
|
||||
|
||||
try {
|
||||
anotherReadClient.readROView(keysWithoutPaths, 1);
|
||||
} catch (Exception e) {
|
||||
// data frame reader throws exception for empty records. ignore the error.
|
||||
assertEquals(e.getClass(), AnalysisException.class);
|
||||
}
|
||||
|
||||
// Actual tests of getPendingCompactions method are in TestAsyncCompaction
|
||||
// This is just testing empty list
|
||||
assertEquals(0, anotherReadClient.getPendingCompactions().size());
|
||||
try {
|
||||
anotherReadClient.readROView(keysWithoutPaths, 1);
|
||||
} catch (Exception e) {
|
||||
// data frame reader throws exception for empty records. ignore the error.
|
||||
assertEquals(e.getClass(), AnalysisException.class);
|
||||
}
|
||||
|
||||
// Actual tests of getPendingCompactions method are in TestAsyncCompaction
|
||||
// This is just testing empty list
|
||||
assertEquals(0, anotherReadClient.getPendingCompactions().size());
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -759,54 +759,54 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness {
|
||||
|
||||
List<HoodieRecord> updatedRecords = dataGen.generateUpdates(newCommitTime, records);
|
||||
JavaRDD<HoodieRecord> updatedRecordsRDD = jsc.parallelize(updatedRecords, 1);
|
||||
try (HoodieReadClient readClient = new HoodieReadClient(jsc, config);) {
|
||||
updatedRecords = readClient.tagLocation(updatedRecordsRDD).collect();
|
||||
|
||||
// Write them to corresponding avro logfiles
|
||||
HoodieTestUtils.writeRecordsToLogFiles(metaClient.getFs(), metaClient.getBasePath(),
|
||||
HoodieTestDataGenerator.AVRO_SCHEMA_WITH_METADATA_FIELDS, updatedRecords);
|
||||
HoodieReadClient readClient = new HoodieReadClient(jsc, config);
|
||||
updatedRecords = readClient.tagLocation(updatedRecordsRDD).collect();
|
||||
|
||||
// Verify that all data file has one log file
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
// In writeRecordsToLogFiles, no commit files are getting added, so resetting file-system view state
|
||||
((SyncableFileSystemView) (table.getSliceView())).reset();
|
||||
// Write them to corresponding avro logfiles
|
||||
HoodieTestUtils.writeRecordsToLogFiles(metaClient.getFs(), metaClient.getBasePath(),
|
||||
HoodieTestDataGenerator.AVRO_SCHEMA_WITH_METADATA_FIELDS, updatedRecords);
|
||||
|
||||
for (String partitionPath : dataGen.getPartitionPaths()) {
|
||||
List<FileSlice> groupedLogFiles =
|
||||
table.getSliceView().getLatestFileSlices(partitionPath).collect(Collectors.toList());
|
||||
for (FileSlice fileSlice : groupedLogFiles) {
|
||||
assertEquals("There should be 1 log file written for every data file", 1, fileSlice.getLogFiles().count());
|
||||
}
|
||||
// Verify that all data file has one log file
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
// In writeRecordsToLogFiles, no commit files are getting added, so resetting file-system view state
|
||||
((SyncableFileSystemView) (table.getSliceView())).reset();
|
||||
|
||||
for (String partitionPath : dataGen.getPartitionPaths()) {
|
||||
List<FileSlice> groupedLogFiles =
|
||||
table.getSliceView().getLatestFileSlices(partitionPath).collect(Collectors.toList());
|
||||
for (FileSlice fileSlice : groupedLogFiles) {
|
||||
assertEquals("There should be 1 log file written for every data file", 1, fileSlice.getLogFiles().count());
|
||||
}
|
||||
}
|
||||
|
||||
// Mark 2nd delta-instant as completed
|
||||
metaClient.getActiveTimeline().createNewInstant(new HoodieInstant(State.INFLIGHT,
|
||||
HoodieTimeline.DELTA_COMMIT_ACTION, newCommitTime));
|
||||
metaClient.getActiveTimeline().saveAsComplete(
|
||||
new HoodieInstant(State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, newCommitTime), Option.empty());
|
||||
// Mark 2nd delta-instant as completed
|
||||
metaClient.getActiveTimeline().createNewInstant(new HoodieInstant(State.INFLIGHT,
|
||||
HoodieTimeline.DELTA_COMMIT_ACTION, newCommitTime));
|
||||
metaClient.getActiveTimeline().saveAsComplete(
|
||||
new HoodieInstant(State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, newCommitTime), Option.empty());
|
||||
|
||||
// Do a compaction
|
||||
String compactionInstantTime = writeClient.scheduleCompaction(Option.empty()).get().toString();
|
||||
JavaRDD<WriteStatus> result = writeClient.compact(compactionInstantTime);
|
||||
// Do a compaction
|
||||
String compactionInstantTime = writeClient.scheduleCompaction(Option.empty()).get().toString();
|
||||
JavaRDD<WriteStatus> result = writeClient.compact(compactionInstantTime);
|
||||
|
||||
// Verify that recently written compacted data file has no log file
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
HoodieActiveTimeline timeline = metaClient.getActiveTimeline();
|
||||
// Verify that recently written compacted data file has no log file
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
HoodieActiveTimeline timeline = metaClient.getActiveTimeline();
|
||||
|
||||
assertTrue("Compaction commit should be > than last insert", HoodieTimeline
|
||||
.compareTimestamps(timeline.lastInstant().get().getTimestamp(), newCommitTime, HoodieTimeline.GREATER));
|
||||
assertTrue("Compaction commit should be > than last insert", HoodieTimeline
|
||||
.compareTimestamps(timeline.lastInstant().get().getTimestamp(), newCommitTime, HoodieTimeline.GREATER));
|
||||
|
||||
for (String partitionPath : dataGen.getPartitionPaths()) {
|
||||
List<FileSlice> groupedLogFiles =
|
||||
table.getSliceView().getLatestFileSlices(partitionPath).collect(Collectors.toList());
|
||||
for (FileSlice slice : groupedLogFiles) {
|
||||
assertEquals("After compaction there should be no log files visible on a full view", 0, slice.getLogFiles().count());
|
||||
}
|
||||
List<WriteStatus> writeStatuses = result.collect();
|
||||
assertTrue(writeStatuses.stream().anyMatch(writeStatus -> writeStatus.getStat().getPartitionPath().contentEquals(partitionPath)));
|
||||
for (String partitionPath : dataGen.getPartitionPaths()) {
|
||||
List<FileSlice> groupedLogFiles =
|
||||
table.getSliceView().getLatestFileSlices(partitionPath).collect(Collectors.toList());
|
||||
for (FileSlice slice : groupedLogFiles) {
|
||||
assertEquals("After compaction there should be no log files visible on a full view", 0, slice.getLogFiles().count());
|
||||
}
|
||||
List<WriteStatus> writeStatuses = result.collect();
|
||||
assertTrue(writeStatuses.stream().anyMatch(writeStatus -> writeStatus.getStat().getPartitionPath().contentEquals(partitionPath)));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -92,9 +92,8 @@ public class TestAsyncCompaction extends TestHoodieClientBase {
|
||||
public void testRollbackForInflightCompaction() throws Exception {
|
||||
// Rollback inflight compaction
|
||||
HoodieWriteConfig cfg = getConfig(false);
|
||||
try (HoodieWriteClient client = getHoodieWriteClient(cfg, true);
|
||||
HoodieReadClient readClient = getHoodieReadClient(cfg.getBasePath());) {
|
||||
|
||||
try (HoodieWriteClient client = getHoodieWriteClient(cfg, true);) {
|
||||
HoodieReadClient readClient = getHoodieReadClient(cfg.getBasePath());
|
||||
String firstInstantTime = "001";
|
||||
String secondInstantTime = "004";
|
||||
String compactionInstantTime = "005";
|
||||
@@ -155,9 +154,8 @@ public class TestAsyncCompaction extends TestHoodieClientBase {
|
||||
|
||||
int numRecs = 2000;
|
||||
|
||||
try (HoodieWriteClient client = getHoodieWriteClient(cfg, true);
|
||||
HoodieReadClient readClient = getHoodieReadClient(cfg.getBasePath());) {
|
||||
|
||||
try (HoodieWriteClient client = getHoodieWriteClient(cfg, true);) {
|
||||
HoodieReadClient readClient = getHoodieReadClient(cfg.getBasePath());
|
||||
List<HoodieRecord> records = dataGen.generateInserts(firstInstantTime, numRecs);
|
||||
records = runNextDeltaCommits(client, readClient, Arrays.asList(firstInstantTime, secondInstantTime), records, cfg, true,
|
||||
new ArrayList<>());
|
||||
@@ -197,9 +195,8 @@ public class TestAsyncCompaction extends TestHoodieClientBase {
|
||||
public void testInflightCompaction() throws Exception {
|
||||
// There is inflight compaction. Subsequent compaction run must work correctly
|
||||
HoodieWriteConfig cfg = getConfig(true);
|
||||
try (HoodieWriteClient client = getHoodieWriteClient(cfg, true);
|
||||
HoodieReadClient readClient = getHoodieReadClient(cfg.getBasePath());) {
|
||||
|
||||
try (HoodieWriteClient client = getHoodieWriteClient(cfg, true);) {
|
||||
HoodieReadClient readClient = getHoodieReadClient(cfg.getBasePath());
|
||||
String firstInstantTime = "001";
|
||||
String secondInstantTime = "004";
|
||||
String compactionInstantTime = "005";
|
||||
@@ -351,9 +348,8 @@ public class TestAsyncCompaction extends TestHoodieClientBase {
|
||||
public void testCompactionAfterTwoDeltaCommits() throws Exception {
|
||||
// No Delta Commits after compaction request
|
||||
HoodieWriteConfig cfg = getConfig(true);
|
||||
try (HoodieWriteClient client = getHoodieWriteClient(cfg, true);
|
||||
HoodieReadClient readClient = getHoodieReadClient(cfg.getBasePath());) {
|
||||
|
||||
try (HoodieWriteClient client = getHoodieWriteClient(cfg, true);) {
|
||||
HoodieReadClient readClient = getHoodieReadClient(cfg.getBasePath());
|
||||
String firstInstantTime = "001";
|
||||
String secondInstantTime = "004";
|
||||
String compactionInstantTime = "005";
|
||||
@@ -373,9 +369,8 @@ public class TestAsyncCompaction extends TestHoodieClientBase {
|
||||
public void testInterleavedCompaction() throws Exception {
|
||||
// Case: Two delta commits before and after compaction schedule
|
||||
HoodieWriteConfig cfg = getConfig(true);
|
||||
try (HoodieWriteClient client = getHoodieWriteClient(cfg, true);
|
||||
HoodieReadClient readClient = getHoodieReadClient(cfg.getBasePath());) {
|
||||
|
||||
try (HoodieWriteClient client = getHoodieWriteClient(cfg, true);) {
|
||||
HoodieReadClient readClient = getHoodieReadClient(cfg.getBasePath());
|
||||
String firstInstantTime = "001";
|
||||
String secondInstantTime = "004";
|
||||
String compactionInstantTime = "005";
|
||||
|
||||
Reference in New Issue
Block a user