[HUDI-308] Avoid Renames for tracking state transitions of all actions on dataset
This commit is contained in:
committed by
Balaji Varadarajan
parent
8963a68e6a
commit
9a1f698eef
@@ -116,6 +116,8 @@ public class TestAsyncCompaction extends TestHoodieClientBase {
|
||||
// Reload and rollback inflight compaction
|
||||
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
|
||||
// hoodieTable.rollback(jsc,
|
||||
// new HoodieInstant(true, HoodieTimeline.COMPACTION_ACTION, compactionInstantTime), false);
|
||||
|
||||
client.rollbackInflightCompaction(
|
||||
new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, compactionInstantTime), hoodieTable);
|
||||
@@ -166,7 +168,7 @@ public class TestAsyncCompaction extends TestHoodieClientBase {
|
||||
assertEquals("Pending Compaction instant has expected instant time", pendingCompactionInstant.getTimestamp(),
|
||||
compactionInstantTime);
|
||||
HoodieInstant inflightInstant =
|
||||
metaClient.getActiveTimeline().filterInflightsExcludingCompaction().firstInstant().get();
|
||||
metaClient.getActiveTimeline().filterPendingExcludingCompaction().firstInstant().get();
|
||||
assertEquals("inflight instant has expected instant time", inflightInstant.getTimestamp(), inflightInstantTime);
|
||||
|
||||
// This should rollback
|
||||
@@ -174,10 +176,10 @@ public class TestAsyncCompaction extends TestHoodieClientBase {
|
||||
|
||||
// Validate
|
||||
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
inflightInstant = metaClient.getActiveTimeline().filterInflightsExcludingCompaction().firstInstant().get();
|
||||
inflightInstant = metaClient.getActiveTimeline().filterPendingExcludingCompaction().firstInstant().get();
|
||||
assertEquals("inflight instant has expected instant time", inflightInstant.getTimestamp(), nextInflightInstantTime);
|
||||
assertEquals("Expect only one inflight instant", 1, metaClient.getActiveTimeline()
|
||||
.filterInflightsExcludingCompaction().getInstants().count());
|
||||
.filterPendingExcludingCompaction().getInstants().count());
|
||||
// Expect pending Compaction to be present
|
||||
pendingCompactionInstant = metaClient.getActiveTimeline().filterPendingCompactionTimeline().firstInstant().get();
|
||||
assertEquals("Pending Compaction instant has expected instant time", pendingCompactionInstant.getTimestamp(),
|
||||
@@ -274,7 +276,7 @@ public class TestAsyncCompaction extends TestHoodieClientBase {
|
||||
|
||||
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
HoodieInstant inflightInstant =
|
||||
metaClient.getActiveTimeline().filterInflightsExcludingCompaction().firstInstant().get();
|
||||
metaClient.getActiveTimeline().filterPendingExcludingCompaction().firstInstant().get();
|
||||
assertEquals("inflight instant has expected instant time", inflightInstant.getTimestamp(), inflightInstantTime);
|
||||
|
||||
boolean gotException = false;
|
||||
|
||||
@@ -434,7 +434,8 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
metaClient.reloadActiveTimeline()
|
||||
.revertToInflight(new HoodieInstant(State.COMPLETED, HoodieTimeline.CLEAN_ACTION, cleanInstantTs));
|
||||
final HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
HoodieCleanMetadata cleanMetadata2 = writeClient.runClean(table, cleanInstantTs);
|
||||
HoodieCleanMetadata cleanMetadata2 = writeClient.runClean(table,
|
||||
HoodieTimeline.getCleanInflightInstant(cleanInstantTs));
|
||||
Assert.assertTrue(
|
||||
Objects.equals(cleanMetadata1.getEarliestCommitToRetain(), cleanMetadata2.getEarliestCommitToRetain()));
|
||||
Assert.assertEquals(new Integer(0), cleanMetadata2.getTotalFilesDeleted());
|
||||
@@ -646,8 +647,11 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
|
||||
HoodieCleanMetadata metadata =
|
||||
CleanerUtils.convertCleanMetadata(metaClient, commitTime, Option.of(0L), Arrays.asList(cleanStat1, cleanStat2));
|
||||
metadata.setVersion(CleanerUtils.CLEAN_METADATA_VERSION_1);
|
||||
|
||||
Assert.assertEquals(CleanerUtils.LATEST_CLEAN_METADATA_VERSION, metadata.getVersion());
|
||||
// NOw upgrade and check
|
||||
CleanMetadataMigrator metadataMigrator = new CleanMetadataMigrator(metaClient);
|
||||
metadata = metadataMigrator.upgradeToLatest(metadata, metadata.getVersion());
|
||||
testCleanMetadataPathEquality(metadata, newExpected);
|
||||
|
||||
CleanMetadataMigrator migrator = new CleanMetadataMigrator(metaClient);
|
||||
@@ -736,7 +740,7 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
.build();
|
||||
|
||||
// make 1 commit, with 1 file per partition
|
||||
HoodieTestUtils.createCommitFiles(basePath, "000");
|
||||
HoodieTestUtils.createInflightCommitFiles(basePath, "000");
|
||||
|
||||
String file1P0C0 =
|
||||
HoodieTestUtils.createNewDataFile(basePath, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, "000");
|
||||
@@ -769,7 +773,7 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
file1P1C0));
|
||||
|
||||
// make next commit, with 1 insert & 1 update per partition
|
||||
HoodieTestUtils.createCommitFiles(basePath, "001");
|
||||
HoodieTestUtils.createInflightCommitFiles(basePath, "001");
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
|
||||
@@ -809,7 +813,7 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
file1P1C0));
|
||||
|
||||
// make next commit, with 2 updates to existing files, and 1 insert
|
||||
HoodieTestUtils.createCommitFiles(basePath, "002");
|
||||
HoodieTestUtils.createInflightCommitFiles(basePath, "002");
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
|
||||
@@ -837,7 +841,7 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
file1P0C0));
|
||||
|
||||
// make next commit, with 2 updates to existing files, and 1 insert
|
||||
HoodieTestUtils.createCommitFiles(basePath, "003");
|
||||
HoodieTestUtils.createInflightCommitFiles(basePath, "003");
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
|
||||
@@ -882,8 +886,10 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
.put(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH,
|
||||
new ImmutableList.Builder<>().add(file3P0C2).build())
|
||||
.build());
|
||||
metaClient.getActiveTimeline().saveToInflight(
|
||||
new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, "004"),
|
||||
metaClient.getActiveTimeline().createNewInstant(
|
||||
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "004"));
|
||||
metaClient.getActiveTimeline().transitionRequestedToInflight(
|
||||
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "004"),
|
||||
Option.of(commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
|
||||
List<HoodieCleanStat> hoodieCleanStatsFive = runCleaner(config, simulateFailureRetry);
|
||||
HoodieCleanStat cleanStat = getCleanStat(hoodieCleanStatsFive, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH);
|
||||
@@ -900,7 +906,6 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
*/
|
||||
@Test
|
||||
public void testCleanMarkerDataFilesOnRollback() throws IOException {
|
||||
HoodieTestUtils.createCommitFiles(basePath, "000");
|
||||
List<String> markerFiles = createMarkerFiles("000", 10);
|
||||
assertEquals("Some marker files are created.", 10, markerFiles.size());
|
||||
assertEquals("Some marker files are created.", markerFiles.size(), getTotalTempFiles());
|
||||
@@ -908,8 +913,12 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
|
||||
table.rollback(jsc, "000", true);
|
||||
table.getActiveTimeline().createNewInstant(new HoodieInstant(State.REQUESTED,
|
||||
HoodieTimeline.COMMIT_ACTION, "000"));
|
||||
table.getActiveTimeline().transitionRequestedToInflight(
|
||||
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "000"), Option.empty());
|
||||
metaClient.reloadActiveTimeline();
|
||||
table.rollback(jsc, new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, "000"), true);
|
||||
assertEquals("All temp files are deleted.", 0, getTotalTempFiles());
|
||||
}
|
||||
|
||||
|
||||
@@ -263,6 +263,8 @@ public class TestClientRollback extends TestHoodieClientBase {
|
||||
String commitTime1 = "20160501010101";
|
||||
String commitTime2 = "20160502020601";
|
||||
String commitTime3 = "20160506030611";
|
||||
String commitTime4 = "20160506030621";
|
||||
String commitTime5 = "20160506030631";
|
||||
new File(basePath + "/.hoodie").mkdirs();
|
||||
HoodieTestDataGenerator.writePartitionMetadata(fs, new String[] {"2016/05/01", "2016/05/02", "2016/05/06"},
|
||||
basePath);
|
||||
@@ -292,7 +294,7 @@ public class TestClientRollback extends TestHoodieClientBase {
|
||||
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build();
|
||||
|
||||
try (HoodieWriteClient client = getHoodieWriteClient(config, false);) {
|
||||
|
||||
client.startCommitWithTime(commitTime4);
|
||||
// Check results, nothing changed
|
||||
assertTrue(HoodieTestUtils.doesCommitExist(basePath, commitTime1));
|
||||
assertTrue(HoodieTestUtils.doesInflightExist(basePath, commitTime2));
|
||||
@@ -310,7 +312,7 @@ public class TestClientRollback extends TestHoodieClientBase {
|
||||
|
||||
// Turn auto rollback on
|
||||
try (HoodieWriteClient client = getHoodieWriteClient(config, true)) {
|
||||
client.startCommit();
|
||||
client.startCommitWithTime(commitTime5);
|
||||
assertTrue(HoodieTestUtils.doesCommitExist(basePath, commitTime1));
|
||||
assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime2));
|
||||
assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime3));
|
||||
|
||||
@@ -27,9 +27,11 @@ import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRollingStat;
|
||||
import org.apache.hudi.common.model.HoodieRollingStatMetadata;
|
||||
import org.apache.hudi.common.model.HoodieTestUtils;
|
||||
import org.apache.hudi.common.model.TimelineLayoutVersion;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.TableFileSystemView.ReadOptimizedView;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.util.ConsistencyGuardConfig;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
@@ -259,13 +261,16 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
|
||||
/**
|
||||
* Test one of HoodieWriteClient upsert(Prepped) APIs.
|
||||
*
|
||||
* @param hoodieWriteConfig Write Config
|
||||
* @param config Write Config
|
||||
* @param writeFn One of Hoodie Write Function API
|
||||
* @throws Exception in case of error
|
||||
*/
|
||||
private void testUpsertsInternal(HoodieWriteConfig hoodieWriteConfig,
|
||||
private void testUpsertsInternal(HoodieWriteConfig config,
|
||||
Function3<JavaRDD<WriteStatus>, HoodieWriteClient, JavaRDD<HoodieRecord>, String> writeFn, boolean isPrepped)
|
||||
throws Exception {
|
||||
// Force using older timeline layout
|
||||
HoodieWriteConfig hoodieWriteConfig = getConfigBuilder().withProps(config.getProps()).withTimelineLayoutVersion(
|
||||
TimelineLayoutVersion.VERSION_0).build();
|
||||
HoodieWriteClient client = getHoodieWriteClient(hoodieWriteConfig, false);
|
||||
|
||||
// Write 1 (only inserts)
|
||||
@@ -292,6 +297,44 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
|
||||
deleteBatch(hoodieWriteConfig, client, newCommitTime, prevCommitTime,
|
||||
initCommitTime, numRecords, HoodieWriteClient::delete, isPrepped, true,
|
||||
0, 150);
|
||||
|
||||
// Now simulate an upgrade and perform a restore operation
|
||||
HoodieWriteConfig newConfig = getConfigBuilder().withProps(config.getProps()).withTimelineLayoutVersion(
|
||||
TimelineLayoutVersion.CURR_VERSION).build();
|
||||
client = getHoodieWriteClient(newConfig, false);
|
||||
client.restoreToInstant("004");
|
||||
|
||||
// Check the entire dataset has all records still
|
||||
String[] fullPartitionPaths = new String[dataGen.getPartitionPaths().length];
|
||||
for (int i = 0; i < fullPartitionPaths.length; i++) {
|
||||
fullPartitionPaths[i] = String.format("%s/%s/*", basePath, dataGen.getPartitionPaths()[i]);
|
||||
}
|
||||
assertEquals("Must contain " + 200 + " records", 200,
|
||||
HoodieClientTestUtils.read(jsc, basePath, sqlContext, fs, fullPartitionPaths).count());
|
||||
|
||||
// Perform Delete again on upgraded dataset.
|
||||
prevCommitTime = newCommitTime;
|
||||
newCommitTime = "006";
|
||||
numRecords = 50;
|
||||
|
||||
deleteBatch(newConfig, client, newCommitTime, prevCommitTime,
|
||||
initCommitTime, numRecords, HoodieWriteClient::delete, isPrepped, true,
|
||||
0, 150);
|
||||
|
||||
HoodieActiveTimeline activeTimeline = new HoodieActiveTimeline(metaClient, false);
|
||||
List<HoodieInstant> instants = activeTimeline.getCommitTimeline().getInstants().collect(Collectors.toList());
|
||||
Assert.assertEquals(5, instants.size());
|
||||
Assert.assertEquals(new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "001"),
|
||||
instants.get(0));
|
||||
Assert.assertEquals(new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "004"),
|
||||
instants.get(1));
|
||||
// New Format should have all states of instants
|
||||
Assert.assertEquals(new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "006"),
|
||||
instants.get(2));
|
||||
Assert.assertEquals(new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, "006"),
|
||||
instants.get(3));
|
||||
Assert.assertEquals(new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "006"),
|
||||
instants.get(4));
|
||||
}
|
||||
|
||||
/**
|
||||
|
||||
@@ -145,19 +145,30 @@ public class HoodieTestDataGenerator {
|
||||
createCommitFile(basePath, commitTime, HoodieTestUtils.getDefaultHadoopConf());
|
||||
}
|
||||
|
||||
public static void createCommitFile(String basePath, String commitTime, Configuration configuration)
|
||||
throws IOException {
|
||||
Path commitFile = new Path(
|
||||
basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline.makeCommitFileName(commitTime));
|
||||
FileSystem fs = FSUtils.getFs(basePath, configuration);
|
||||
FSDataOutputStream os = fs.create(commitFile, true);
|
||||
HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata();
|
||||
try {
|
||||
// Write empty commit metadata
|
||||
os.writeBytes(new String(commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
|
||||
} finally {
|
||||
os.close();
|
||||
}
|
||||
public static void createCommitFile(String basePath, String commitTime, Configuration configuration) {
|
||||
Arrays.asList(HoodieTimeline.makeCommitFileName(commitTime), HoodieTimeline.makeInflightCommitFileName(commitTime),
|
||||
HoodieTimeline.makeRequestedCommitFileName(commitTime)).forEach(f -> {
|
||||
Path commitFile = new Path(
|
||||
basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + f);
|
||||
FSDataOutputStream os = null;
|
||||
try {
|
||||
FileSystem fs = FSUtils.getFs(basePath, configuration);
|
||||
os = fs.create(commitFile, true);
|
||||
HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata();
|
||||
// Write empty commit metadata
|
||||
os.writeBytes(new String(commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
|
||||
} catch (IOException ioe) {
|
||||
throw new HoodieIOException(ioe.getMessage(), ioe);
|
||||
} finally {
|
||||
if (null != os) {
|
||||
try {
|
||||
os.close();
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException(e.getMessage(), e);
|
||||
}
|
||||
}
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
public static void createCompactionRequestedFile(String basePath, String commitTime, Configuration configuration)
|
||||
|
||||
@@ -43,7 +43,7 @@ import static org.mockito.Mockito.when;
|
||||
|
||||
public class TestBoundedInMemoryExecutor extends HoodieClientTestHarness {
|
||||
|
||||
private final String commitTime = HoodieActiveTimeline.createNewCommitTime();
|
||||
private final String commitTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
|
||||
@Before
|
||||
public void setUp() throws Exception {
|
||||
|
||||
@@ -59,7 +59,7 @@ import static org.mockito.Mockito.when;
|
||||
|
||||
public class TestBoundedInMemoryQueue extends HoodieClientTestHarness {
|
||||
|
||||
private final String commitTime = HoodieActiveTimeline.createNewCommitTime();
|
||||
private final String commitTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
|
||||
@Before
|
||||
public void setUp() throws Exception {
|
||||
|
||||
@@ -37,6 +37,7 @@ import org.apache.hudi.index.hbase.HBaseIndexQPSResourceAllocator;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
@@ -131,7 +132,6 @@ public class TestHbaseIndex extends HoodieClientTestHarness {
|
||||
HoodieWriteConfig config = getConfig();
|
||||
HBaseIndex index = new HBaseIndex(config);
|
||||
try (HoodieWriteClient writeClient = getWriteClient(config);) {
|
||||
writeClient.startCommit();
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
|
||||
@@ -140,6 +140,7 @@ public class TestHbaseIndex extends HoodieClientTestHarness {
|
||||
assert (javaRDD.filter(record -> record.isCurrentLocationKnown()).collect().size() == 0);
|
||||
|
||||
// Insert 200 records
|
||||
writeClient.startCommitWithTime(newCommitTime);
|
||||
JavaRDD<WriteStatus> writeStatues = writeClient.upsert(writeRecords, newCommitTime);
|
||||
assertNoWriteErrors(writeStatues.collect());
|
||||
|
||||
@@ -171,13 +172,19 @@ public class TestHbaseIndex extends HoodieClientTestHarness {
|
||||
HoodieWriteConfig config = getConfig();
|
||||
HBaseIndex index = new HBaseIndex(config);
|
||||
HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config);
|
||||
writeClient.startCommit();
|
||||
writeClient.startCommitWithTime(newCommitTime);
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
|
||||
JavaRDD<WriteStatus> writeStatues = writeClient.upsert(writeRecords, newCommitTime);
|
||||
JavaRDD<HoodieRecord> javaRDD1 = index.tagLocation(writeRecords, jsc, 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
|
||||
// recomputed. This includes the state transitions. We need to delete the inflight instance so that subsequent
|
||||
// upsert will not run into conflicts.
|
||||
metaClient.getFs().delete(new Path(metaClient.getMetaPath(), "001.inflight"));
|
||||
|
||||
writeClient.upsert(writeRecords, newCommitTime);
|
||||
assertNoWriteErrors(writeStatues.collect());
|
||||
|
||||
|
||||
@@ -45,8 +45,8 @@ import org.junit.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
@@ -142,13 +142,12 @@ public class TestHoodieCommitArchiveLog extends HoodieClientTestHarness {
|
||||
assertEquals("Loaded 6 commits and the count should match", 6, timeline.countInstants());
|
||||
|
||||
HoodieTestUtils.createCleanFiles(metaClient, basePath, "100", dfs.getConf());
|
||||
HoodieTestUtils.createInflightCleanFiles(basePath, dfs.getConf(), "101");
|
||||
HoodieTestUtils.createCleanFiles(metaClient, basePath, "101", dfs.getConf());
|
||||
HoodieTestUtils.createCleanFiles(metaClient, basePath, "102", dfs.getConf());
|
||||
HoodieTestUtils.createCleanFiles(metaClient, basePath, "103", dfs.getConf());
|
||||
HoodieTestUtils.createCleanFiles(metaClient, basePath, "104", dfs.getConf());
|
||||
HoodieTestUtils.createCleanFiles(metaClient, basePath, "105", dfs.getConf());
|
||||
HoodieTestUtils.createInflightCleanFiles(basePath, dfs.getConf(), "106", "107");
|
||||
HoodieTestUtils.createPendingCleanFiles(metaClient, dfs.getConf(), "106", "107");
|
||||
|
||||
// reload the timeline and get all the commmits before archive
|
||||
timeline = metaClient.getActiveTimeline().reload().getAllCommitsTimeline().filterCompletedInstants();
|
||||
@@ -157,7 +156,7 @@ public class TestHoodieCommitArchiveLog extends HoodieClientTestHarness {
|
||||
assertEquals("Loaded 6 commits and the count should match", 12, timeline.countInstants());
|
||||
|
||||
// verify in-flight instants before archive
|
||||
verifyInflightInstants(metaClient, 3);
|
||||
verifyInflightInstants(metaClient, 2);
|
||||
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, metaClient);
|
||||
@@ -169,8 +168,8 @@ public class TestHoodieCommitArchiveLog extends HoodieClientTestHarness {
|
||||
originalCommits.removeAll(timeline.getInstants().collect(Collectors.toList()));
|
||||
|
||||
// Check compaction instants
|
||||
List<HoodieInstant> instants = HoodieTableMetaClient.scanHoodieInstantsFromFileSystem(metaClient.getFs(),
|
||||
new Path(metaClient.getMetaAuxiliaryPath()), HoodieActiveTimeline.VALID_EXTENSIONS_IN_ACTIVE_TIMELINE);
|
||||
List<HoodieInstant> instants = metaClient.scanHoodieInstantsFromFileSystem(
|
||||
new Path(metaClient.getMetaAuxiliaryPath()), HoodieActiveTimeline.VALID_EXTENSIONS_IN_ACTIVE_TIMELINE, false);
|
||||
assertEquals("Should delete all compaction instants < 104", 4, instants.size());
|
||||
assertFalse("Requested Compaction must be absent for 100",
|
||||
instants.contains(new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "100")));
|
||||
@@ -201,30 +200,31 @@ public class TestHoodieCommitArchiveLog extends HoodieClientTestHarness {
|
||||
Reader reader =
|
||||
HoodieLogFormat.newReader(dfs, new HoodieLogFile(new Path(basePath + "/.hoodie/.commits_.archive.1_1-0-1")),
|
||||
HoodieArchivedMetaEntry.getClassSchema());
|
||||
|
||||
int archivedRecordsCount = 0;
|
||||
List<IndexedRecord> readRecords = new ArrayList<>();
|
||||
// read the avro blocks and validate the number of records written in each avro block
|
||||
int numBlocks = 0;
|
||||
while (reader.hasNext()) {
|
||||
HoodieAvroDataBlock blk = (HoodieAvroDataBlock) reader.next();
|
||||
List<IndexedRecord> records = blk.getRecords();
|
||||
readRecords.addAll(records);
|
||||
assertEquals("Archived and read records for each block are same", 8, records.size());
|
||||
archivedRecordsCount += records.size();
|
||||
numBlocks++;
|
||||
}
|
||||
assertEquals("Total archived records and total read records are the same count", 8, archivedRecordsCount);
|
||||
|
||||
System.out.println("Read Records :" + readRecords.stream().map(r -> (GenericRecord) r)
|
||||
.map(r -> r.get("actionType") + "_" + r.get("actionState") + "_" + r.get("commitTime")).collect(Collectors.toList()));
|
||||
assertEquals("Total archived records and total read records are the same count", 24, archivedRecordsCount);
|
||||
assertTrue("Average Archived records per block is greater than 1", archivedRecordsCount / numBlocks > 1);
|
||||
// make sure the archived commits are the same as the (originalcommits - commitsleft)
|
||||
List<String> readCommits = readRecords.stream().map(r -> (GenericRecord) r).map(r -> {
|
||||
Set<String> readCommits = readRecords.stream().map(r -> (GenericRecord) r).map(r -> {
|
||||
return r.get("commitTime").toString();
|
||||
}).collect(Collectors.toList());
|
||||
Collections.sort(readCommits);
|
||||
}).collect(Collectors.toSet());
|
||||
|
||||
assertEquals("Read commits map should match the originalCommits - commitsLoadedFromArchival",
|
||||
originalCommits.stream().map(HoodieInstant::getTimestamp).collect(Collectors.toList()), readCommits);
|
||||
originalCommits.stream().map(HoodieInstant::getTimestamp).collect(Collectors.toSet()), readCommits);
|
||||
|
||||
// verify in-flight instants after archive
|
||||
verifyInflightInstants(metaClient, 3);
|
||||
verifyInflightInstants(metaClient, 2);
|
||||
reader.close();
|
||||
}
|
||||
|
||||
@@ -272,8 +272,8 @@ public class TestHoodieCommitArchiveLog extends HoodieClientTestHarness {
|
||||
timeline = metaClient.getActiveTimeline().reload().getCommitsTimeline().filterCompletedInstants();
|
||||
assertEquals("Should not archive commits when maxCommitsToKeep is 5", 4, timeline.countInstants());
|
||||
|
||||
List<HoodieInstant> instants = HoodieTableMetaClient.scanHoodieInstantsFromFileSystem(metaClient.getFs(),
|
||||
new Path(metaClient.getMetaAuxiliaryPath()), HoodieActiveTimeline.VALID_EXTENSIONS_IN_ACTIVE_TIMELINE);
|
||||
List<HoodieInstant> instants = metaClient.scanHoodieInstantsFromFileSystem(
|
||||
new Path(metaClient.getMetaAuxiliaryPath()), HoodieActiveTimeline.VALID_EXTENSIONS_IN_ACTIVE_TIMELINE, false);
|
||||
assertEquals("Should not delete any aux compaction files when maxCommitsToKeep is 5", 8, instants.size());
|
||||
assertTrue("Requested Compaction must be present for 100",
|
||||
instants.contains(new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "100")));
|
||||
|
||||
@@ -101,7 +101,7 @@ public class TestHoodieCompactor extends HoodieClientTestHarness {
|
||||
public void testCompactionOnCopyOnWriteFail() throws Exception {
|
||||
metaClient = HoodieTestUtils.init(hadoopConf, basePath, HoodieTableType.COPY_ON_WRITE);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig(), jsc);
|
||||
String compactionInstantTime = HoodieActiveTimeline.createNewCommitTime();
|
||||
String compactionInstantTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
table.compact(jsc, compactionInstantTime, table.scheduleCompaction(jsc, compactionInstantTime));
|
||||
}
|
||||
|
||||
@@ -117,7 +117,7 @@ public class TestHoodieCompactor extends HoodieClientTestHarness {
|
||||
JavaRDD<HoodieRecord> recordsRDD = jsc.parallelize(records, 1);
|
||||
writeClient.insert(recordsRDD, newCommitTime).collect();
|
||||
|
||||
String compactionInstantTime = HoodieActiveTimeline.createNewCommitTime();
|
||||
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());
|
||||
@@ -167,7 +167,7 @@ public class TestHoodieCompactor extends HoodieClientTestHarness {
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
|
||||
String compactionInstantTime = HoodieActiveTimeline.createNewCommitTime();
|
||||
String compactionInstantTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
JavaRDD<WriteStatus> result =
|
||||
table.compact(jsc, compactionInstantTime, table.scheduleCompaction(jsc, compactionInstantTime));
|
||||
|
||||
|
||||
@@ -196,7 +196,7 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness {
|
||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 200);
|
||||
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
|
||||
|
||||
client.startCommit();
|
||||
client.startCommitWithTime(newCommitTime);
|
||||
|
||||
List<WriteStatus> statuses = client.upsert(writeRecords, newCommitTime).collect();
|
||||
assertNoWriteErrors(statuses);
|
||||
@@ -821,6 +821,8 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness {
|
||||
}
|
||||
|
||||
// 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());
|
||||
|
||||
@@ -1009,6 +1011,8 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness {
|
||||
table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath), config, jsc);
|
||||
tableRTFileSystemView = table.getRTFileSystemView();
|
||||
((SyncableFileSystemView) tableRTFileSystemView).reset();
|
||||
Option<HoodieInstant> lastInstant = ((SyncableFileSystemView) tableRTFileSystemView).getLastInstant();
|
||||
System.out.println("Last Instant =" + lastInstant);
|
||||
for (String partitionPath : dataGen.getPartitionPaths()) {
|
||||
Assert.assertTrue(tableRTFileSystemView.getLatestFileSlices(partitionPath)
|
||||
.filter(fileSlice -> fileSlice.getDataFile().isPresent()).count() == 0);
|
||||
@@ -1032,8 +1036,10 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness {
|
||||
// Create a commit without rolling stats in metadata to test backwards compatibility
|
||||
HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
|
||||
String commitActionType = table.getMetaClient().getCommitActionType();
|
||||
HoodieInstant instant = new HoodieInstant(true, commitActionType, "000");
|
||||
activeTimeline.createInflight(instant);
|
||||
HoodieInstant instant = new HoodieInstant(State.REQUESTED, commitActionType, "000");
|
||||
activeTimeline.createNewInstant(instant);
|
||||
activeTimeline.transitionRequestedToInflight(instant, Option.empty());
|
||||
instant = new HoodieInstant(State.INFLIGHT, commitActionType, "000");
|
||||
activeTimeline.saveAsComplete(instant, Option.empty());
|
||||
|
||||
String commitTime = "001";
|
||||
|
||||
Reference in New Issue
Block a user