[HUDI-785] Refactor compaction/savepoint execution based on ActionExector abstraction (#1548)
- Savepoint and compaction classes moved to table.action.* packages - HoodieWriteClient#savepoint(...) returns void - Renamed HoodieCommitArchiveLog -> HoodieTimelineArchiveLog - Fixed tests to take into account the additional validation done - Moved helper code into CompactHelpers and SavepointHelpers
This commit is contained in:
@@ -58,7 +58,7 @@ public class TestClientRollback extends TestHoodieClientBase {
|
||||
public void testSavepointAndRollback() throws Exception {
|
||||
HoodieWriteConfig cfg = getConfigBuilder().withCompactionConfig(HoodieCompactionConfig.newBuilder()
|
||||
.withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(1).build()).build();
|
||||
try (HoodieWriteClient client = getHoodieWriteClient(cfg);) {
|
||||
try (HoodieWriteClient client = getHoodieWriteClient(cfg)) {
|
||||
HoodieTestDataGenerator.writePartitionMetadata(fs, HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS, basePath);
|
||||
|
||||
/**
|
||||
|
||||
@@ -32,7 +32,7 @@ import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.table.compact.OperationResult;
|
||||
import org.apache.hudi.table.action.compact.OperationResult;
|
||||
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
@@ -31,7 +31,7 @@ import org.apache.hudi.common.table.timeline.HoodieInstant.State;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.config.HoodieCompactionConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieCommitArchiveLog;
|
||||
import org.apache.hudi.table.HoodieTimelineArchiveLog;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
@@ -79,7 +79,7 @@ public class TestHoodieCommitArchiveLog extends HoodieClientTestHarness {
|
||||
HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA)
|
||||
.withParallelism(2, 2).forTable("test-trip-table").build();
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, metaClient);
|
||||
HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, metaClient);
|
||||
boolean result = archiveLog.archiveIfRequired(jsc);
|
||||
assertTrue(result);
|
||||
}
|
||||
@@ -157,7 +157,7 @@ public class TestHoodieCommitArchiveLog extends HoodieClientTestHarness {
|
||||
verifyInflightInstants(metaClient, 2);
|
||||
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, metaClient);
|
||||
HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, metaClient);
|
||||
|
||||
assertTrue(archiveLog.archiveIfRequired(jsc));
|
||||
|
||||
@@ -216,7 +216,7 @@ public class TestHoodieCommitArchiveLog extends HoodieClientTestHarness {
|
||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 5).build())
|
||||
.build();
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, metaClient);
|
||||
HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, metaClient);
|
||||
// Requested Compaction
|
||||
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
|
||||
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "100"), dfs.getConf());
|
||||
@@ -281,7 +281,7 @@ public class TestHoodieCommitArchiveLog extends HoodieClientTestHarness {
|
||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 5).build())
|
||||
.build();
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, metaClient);
|
||||
HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, metaClient);
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "100", dfs.getConf());
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "101", dfs.getConf());
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "102", dfs.getConf());
|
||||
@@ -307,7 +307,7 @@ public class TestHoodieCommitArchiveLog extends HoodieClientTestHarness {
|
||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 5).build())
|
||||
.build();
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, metaClient);
|
||||
HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, metaClient);
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "100", dfs.getConf());
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "101", dfs.getConf());
|
||||
HoodieTestDataGenerator.createSavepointFile(basePath, "101", dfs.getConf());
|
||||
@@ -339,7 +339,7 @@ public class TestHoodieCommitArchiveLog extends HoodieClientTestHarness {
|
||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 5).build())
|
||||
.build();
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, metaClient);
|
||||
HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, metaClient);
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "100", dfs.getConf());
|
||||
HoodieTestDataGenerator.createCompactionRequestedFile(basePath, "101", dfs.getConf());
|
||||
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
|
||||
@@ -386,7 +386,7 @@ public class TestHoodieCommitArchiveLog extends HoodieClientTestHarness {
|
||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 3).build())
|
||||
.build();
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, metaClient);
|
||||
HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, metaClient);
|
||||
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "1", dfs.getConf());
|
||||
HoodieInstant instant1 = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "1");
|
||||
@@ -426,7 +426,7 @@ public class TestHoodieCommitArchiveLog extends HoodieClientTestHarness {
|
||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 5).build())
|
||||
.build();
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, metaClient);
|
||||
HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, metaClient);
|
||||
|
||||
org.apache.hudi.avro.model.HoodieCommitMetadata expectedCommitMetadata = archiveLog.convertCommitMetadata(hoodieCommitMetadata);
|
||||
assertEquals(expectedCommitMetadata.getOperationType(), WriteOperationType.INSERT.toString());
|
||||
|
||||
@@ -16,7 +16,7 @@
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.table.compact;
|
||||
package org.apache.hudi.table.action.compact;
|
||||
|
||||
import org.apache.hudi.avro.model.HoodieCompactionOperation;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
@@ -16,8 +16,9 @@
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.table.compact;
|
||||
package org.apache.hudi.table.action.compact;
|
||||
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.client.HoodieWriteClient;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.common.HoodieClientTestHarness;
|
||||
@@ -29,6 +30,7 @@ import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.model.HoodieTestUtils;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieCompactionConfig;
|
||||
import org.apache.hudi.config.HoodieIndexConfig;
|
||||
import org.apache.hudi.config.HoodieMemoryConfig;
|
||||
@@ -50,6 +52,7 @@ import java.util.stream.Collectors;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
|
||||
public class TestHoodieCompactor extends HoodieClientTestHarness {
|
||||
|
||||
@@ -100,9 +103,10 @@ public class TestHoodieCompactor extends HoodieClientTestHarness {
|
||||
@Test(expected = HoodieNotSupportedException.class)
|
||||
public void testCompactionOnCopyOnWriteFail() throws Exception {
|
||||
metaClient = HoodieTestUtils.init(hadoopConf, basePath, HoodieTableType.COPY_ON_WRITE);
|
||||
HoodieTable table = HoodieTable.create(metaClient, getConfig(), jsc);
|
||||
HoodieTable<?> table = HoodieTable.create(metaClient, getConfig(), jsc);
|
||||
String compactionInstantTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
table.compact(jsc, compactionInstantTime, table.scheduleCompaction(jsc, compactionInstantTime));
|
||||
table.scheduleCompaction(jsc, compactionInstantTime, Option.empty());
|
||||
table.compact(jsc, compactionInstantTime);
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -118,9 +122,8 @@ public class TestHoodieCompactor extends HoodieClientTestHarness {
|
||||
writeClient.insert(recordsRDD, newCommitTime).collect();
|
||||
|
||||
String compactionInstantTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
JavaRDD<WriteStatus> result =
|
||||
table.compact(jsc, compactionInstantTime, table.scheduleCompaction(jsc, compactionInstantTime));
|
||||
assertTrue("If there is nothing to compact, result will be empty", result.isEmpty());
|
||||
Option<HoodieCompactionPlan> plan = table.scheduleCompaction(jsc, compactionInstantTime, Option.empty());
|
||||
assertFalse("If there is nothing to compact, result will be empty", plan.isPresent());
|
||||
}
|
||||
}
|
||||
|
||||
@@ -128,18 +131,16 @@ public class TestHoodieCompactor extends HoodieClientTestHarness {
|
||||
public void testWriteStatusContentsAfterCompaction() throws Exception {
|
||||
// insert 100 records
|
||||
HoodieWriteConfig config = getConfig();
|
||||
try (HoodieWriteClient writeClient = getWriteClient(config);) {
|
||||
try (HoodieWriteClient writeClient = getWriteClient(config)) {
|
||||
String newCommitTime = "100";
|
||||
writeClient.startCommitWithTime(newCommitTime);
|
||||
|
||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 100);
|
||||
JavaRDD<HoodieRecord> recordsRDD = jsc.parallelize(records, 1);
|
||||
List<WriteStatus> statuses = writeClient.insert(recordsRDD, newCommitTime).collect();
|
||||
writeClient.insert(recordsRDD, newCommitTime).collect();
|
||||
|
||||
// Update all the 100 records
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTable table = HoodieTable.create(metaClient, config, jsc);
|
||||
|
||||
HoodieTable table = HoodieTable.create(config, jsc);
|
||||
newCommitTime = "101";
|
||||
writeClient.startCommitWithTime(newCommitTime);
|
||||
|
||||
@@ -153,8 +154,7 @@ public class TestHoodieCompactor extends HoodieClientTestHarness {
|
||||
HoodieTestDataGenerator.AVRO_SCHEMA_WITH_METADATA_FIELDS, updatedRecords);
|
||||
|
||||
// Verify that all data file has one log file
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
table = HoodieTable.create(metaClient, config, jsc);
|
||||
table = HoodieTable.create(config, jsc);
|
||||
for (String partitionPath : dataGen.getPartitionPaths()) {
|
||||
List<FileSlice> groupedLogFiles =
|
||||
table.getSliceView().getLatestFileSlices(partitionPath).collect(Collectors.toList());
|
||||
@@ -162,14 +162,14 @@ public class TestHoodieCompactor extends HoodieClientTestHarness {
|
||||
assertEquals("There should be 1 log file written for every data file", 1, fileSlice.getLogFiles().count());
|
||||
}
|
||||
}
|
||||
HoodieTestUtils.createDeltaCommitFiles(basePath, newCommitTime);
|
||||
|
||||
// Do a compaction
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
table = HoodieTable.create(metaClient, config, jsc);
|
||||
|
||||
String compactionInstantTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
JavaRDD<WriteStatus> result =
|
||||
table.compact(jsc, compactionInstantTime, table.scheduleCompaction(jsc, compactionInstantTime));
|
||||
table = HoodieTable.create(config, jsc);
|
||||
String compactionInstantTime = "102";
|
||||
table.scheduleCompaction(jsc, compactionInstantTime, Option.empty());
|
||||
table.getMetaClient().reloadActiveTimeline();
|
||||
JavaRDD<WriteStatus> result = table.compact(jsc, compactionInstantTime).getWriteStatuses();
|
||||
|
||||
// Verify that all partition paths are present in the WriteStatus result
|
||||
for (String partitionPath : dataGen.getPartitionPaths()) {
|
||||
@@ -184,8 +184,4 @@ public class TestHoodieCompactor extends HoodieClientTestHarness {
|
||||
protected HoodieTableType getTableType() {
|
||||
return HoodieTableType.MERGE_ON_READ;
|
||||
}
|
||||
|
||||
// TODO - after modifying HoodieReadClient to support mor tables - add more tests to make
|
||||
// sure the data read is the updated data (compaction correctness)
|
||||
// TODO - add more test cases for compactions after a failed commit/compaction
|
||||
}
|
||||
@@ -16,7 +16,7 @@
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.table.compact.strategy;
|
||||
package org.apache.hudi.table.action.compact.strategy;
|
||||
|
||||
import org.apache.hudi.avro.model.HoodieCompactionOperation;
|
||||
import org.apache.hudi.common.model.HoodieBaseFile;
|
||||
Reference in New Issue
Block a user