[HUDI-2475] [HUDI-2862] Metadata table creation and avoid bootstrapping race for write client & add locking for upgrade (#4114)
Co-authored-by: Sivabalan Narayanan <n.siva.b@gmail.com>
This commit is contained in:
committed by
GitHub
parent
3a8d64e584
commit
2c7656c35f
@@ -23,23 +23,30 @@ import org.apache.hudi.common.model.HoodieBaseFile;
|
||||
import org.apache.hudi.common.model.HoodieCleaningPolicy;
|
||||
import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.WriteOperationType;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.view.TableFileSystemView.BaseFileOnlyView;
|
||||
import org.apache.hudi.common.testutils.FileCreateUtils;
|
||||
import org.apache.hudi.common.testutils.HoodieMetadataTestTable;
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.common.testutils.HoodieTestTable;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieCompactionConfig;
|
||||
import org.apache.hudi.config.HoodieIndexConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieRollbackException;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.metadata.HoodieTableMetadataWriter;
|
||||
import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter;
|
||||
import org.apache.hudi.table.HoodieSparkTable;
|
||||
import org.apache.hudi.testutils.HoodieClientTestBase;
|
||||
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
@@ -192,14 +199,6 @@ public class TestClientRollback extends HoodieClientTestBase {
|
||||
put(p3, "id33");
|
||||
}
|
||||
};
|
||||
HoodieTestTable testTable = HoodieTestTable.of(metaClient)
|
||||
.withPartitionMetaFiles(p1, p2, p3)
|
||||
.addCommit(commitTime1)
|
||||
.withBaseFilesInPartitions(partitionAndFileId1)
|
||||
.addCommit(commitTime2)
|
||||
.withBaseFilesInPartitions(partitionAndFileId2)
|
||||
.addInflightCommit(commitTime3)
|
||||
.withBaseFilesInPartitions(partitionAndFileId3);
|
||||
|
||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath)
|
||||
.withRollbackUsingMarkers(false)
|
||||
@@ -207,6 +206,24 @@ public class TestClientRollback extends HoodieClientTestBase {
|
||||
.withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY).build())
|
||||
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build();
|
||||
|
||||
HoodieTableMetadataWriter metadataWriter = SparkHoodieBackedTableMetadataWriter.create(hadoopConf, config, context);
|
||||
HoodieTestTable testTable = HoodieMetadataTestTable.of(metaClient, metadataWriter);
|
||||
|
||||
Map<String, List<Pair<String, Integer>>> partitionToFilesNameLengthMap1 = new HashMap<>();
|
||||
partitionAndFileId1.forEach((k, v) -> partitionToFilesNameLengthMap1.put(k, Collections.singletonList(Pair.of(v, 100))));
|
||||
testTable.doWriteOperation(commitTime1, WriteOperationType.INSERT, Arrays.asList(p1, p2, p3), partitionToFilesNameLengthMap1,
|
||||
false, false);
|
||||
|
||||
Map<String, List<Pair<String, Integer>>> partitionToFilesNameLengthMap2 = new HashMap<>();
|
||||
partitionAndFileId2.forEach((k, v) -> partitionToFilesNameLengthMap2.put(k, Collections.singletonList(Pair.of(v, 200))));
|
||||
testTable.doWriteOperation(commitTime2, WriteOperationType.INSERT, Collections.emptyList(), partitionToFilesNameLengthMap2,
|
||||
false, false);
|
||||
|
||||
Map<String, List<Pair<String, Integer>>> partitionToFilesNameLengthMap3 = new HashMap<>();
|
||||
partitionAndFileId3.forEach((k, v) -> partitionToFilesNameLengthMap3.put(k, Collections.singletonList(Pair.of(v, 300))));
|
||||
testTable.doWriteOperation(commitTime3, WriteOperationType.INSERT, Collections.emptyList(), partitionToFilesNameLengthMap3,
|
||||
false, true);
|
||||
|
||||
try (SparkRDDWriteClient client = getHoodieWriteClient(config)) {
|
||||
|
||||
// Rollback commit3
|
||||
@@ -359,14 +376,6 @@ public class TestClientRollback extends HoodieClientTestBase {
|
||||
put(p3, "id33");
|
||||
}
|
||||
};
|
||||
HoodieTestTable testTable = HoodieTestTable.of(metaClient)
|
||||
.withPartitionMetaFiles(p1, p2, p3)
|
||||
.addCommit(commitTime1)
|
||||
.withBaseFilesInPartitions(partitionAndFileId1)
|
||||
.addInflightCommit(commitTime2)
|
||||
.withBaseFilesInPartitions(partitionAndFileId2)
|
||||
.addInflightCommit(commitTime3)
|
||||
.withBaseFilesInPartitions(partitionAndFileId3);
|
||||
|
||||
// Set Failed Writes rollback to LAZY
|
||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath)
|
||||
@@ -374,6 +383,24 @@ public class TestClientRollback extends HoodieClientTestBase {
|
||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder()
|
||||
.withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY).build()).build();
|
||||
|
||||
HoodieTableMetadataWriter metadataWriter = SparkHoodieBackedTableMetadataWriter.create(hadoopConf, config, context);
|
||||
HoodieTestTable testTable = HoodieMetadataTestTable.of(metaClient, metadataWriter);
|
||||
|
||||
Map<String, List<Pair<String, Integer>>> partitionToFilesNameLengthMap1 = new HashMap<>();
|
||||
partitionAndFileId1.forEach((k, v) -> partitionToFilesNameLengthMap1.put(k, Collections.singletonList(Pair.of(v, 100))));
|
||||
testTable.doWriteOperation(commitTime1, WriteOperationType.INSERT, Arrays.asList(p1, p2, p3), partitionToFilesNameLengthMap1,
|
||||
false, false);
|
||||
|
||||
Map<String, List<Pair<String, Integer>>> partitionToFilesNameLengthMap2 = new HashMap<>();
|
||||
partitionAndFileId2.forEach((k, v) -> partitionToFilesNameLengthMap2.put(k, Collections.singletonList(Pair.of(v, 200))));
|
||||
testTable.doWriteOperation(commitTime2, WriteOperationType.INSERT, Collections.emptyList(), partitionToFilesNameLengthMap2,
|
||||
false, true);
|
||||
|
||||
Map<String, List<Pair<String, Integer>>> partitionToFilesNameLengthMap3 = new HashMap<>();
|
||||
partitionAndFileId3.forEach((k, v) -> partitionToFilesNameLengthMap3.put(k, Collections.singletonList(Pair.of(v, 300))));
|
||||
testTable.doWriteOperation(commitTime3, WriteOperationType.INSERT, Collections.emptyList(), partitionToFilesNameLengthMap3,
|
||||
false, true);
|
||||
|
||||
final String commitTime4 = "20160506030621";
|
||||
try (SparkRDDWriteClient client = getHoodieWriteClient(config)) {
|
||||
client.startCommitWithTime(commitTime4);
|
||||
|
||||
@@ -151,8 +151,8 @@ public class TestHoodieClientMultiWriter extends HoodieClientTestBase {
|
||||
|
||||
Properties properties = new Properties();
|
||||
properties.setProperty(FILESYSTEM_LOCK_PATH_PROP_KEY, basePath + "/.hoodie/.locks");
|
||||
properties.setProperty(LockConfiguration.LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP_KEY, "3");
|
||||
properties.setProperty(LockConfiguration.LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY, "5000");
|
||||
properties.setProperty(LockConfiguration.LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP_KEY, "10");
|
||||
properties.setProperty(LockConfiguration.LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY, "3000");
|
||||
|
||||
HoodieWriteConfig cfg = getConfigBuilder()
|
||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder()
|
||||
@@ -168,6 +168,8 @@ public class TestHoodieClientMultiWriter extends HoodieClientTestBase {
|
||||
// Timeline-server-based markers are not used for multi-writer tests
|
||||
.withMarkersType(MarkerType.DIRECT.name())
|
||||
.withProperties(properties)
|
||||
.withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder().withStorageType(FileSystemViewStorageType.REMOTE_FIRST)
|
||||
.withSecondaryStorageType(FileSystemViewStorageType.MEMORY).build())
|
||||
.build();
|
||||
|
||||
// Create the first commit
|
||||
@@ -338,9 +340,11 @@ public class TestHoodieClientMultiWriter extends HoodieClientTestBase {
|
||||
.withLockConfig(HoodieLockConfig.newBuilder().withLockProvider(FileSystemBasedLockProviderTestClass.class)
|
||||
.build()).withAutoCommit(false).withProperties(properties);
|
||||
HoodieWriteConfig cfg = writeConfigBuilder.build();
|
||||
HoodieWriteConfig cfg2 = writeConfigBuilder
|
||||
HoodieWriteConfig cfg2 = writeConfigBuilder.build();
|
||||
HoodieWriteConfig cfg3 = writeConfigBuilder
|
||||
.withClusteringConfig(HoodieClusteringConfig.newBuilder().withInlineClustering(true).withInlineClusteringNumCommits(1).build())
|
||||
.build();
|
||||
|
||||
// Create the first commit
|
||||
createCommitWithInserts(cfg, getHoodieWriteClient(cfg), "000", "001", 200);
|
||||
// Start another inflight commit
|
||||
@@ -359,7 +363,7 @@ public class TestHoodieClientMultiWriter extends HoodieClientTestBase {
|
||||
numRecords, 200, 2);
|
||||
client2.commit(newCommitTime, result2);
|
||||
// Schedule and run clustering while previous writer for commit 003 is running
|
||||
SparkRDDWriteClient client3 = getHoodieWriteClient(cfg2);
|
||||
SparkRDDWriteClient client3 = getHoodieWriteClient(cfg3);
|
||||
// schedule clustering
|
||||
Option<String> clusterInstant = client3.scheduleTableService(Option.empty(), TableServiceType.CLUSTER);
|
||||
assertTrue(clusterInstant.isPresent());
|
||||
|
||||
@@ -82,6 +82,7 @@ import org.apache.hadoop.util.Time;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.junit.jupiter.api.Disabled;
|
||||
import org.junit.jupiter.api.Tag;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.params.ParameterizedTest;
|
||||
@@ -903,7 +904,7 @@ public class TestHoodieBackedMetadata extends TestHoodieMetadataBase {
|
||||
* <p>
|
||||
* Metadata Table should be automatically compacted as per config.
|
||||
*/
|
||||
@Test
|
||||
@Disabled
|
||||
public void testCleaningArchivingAndCompaction() throws Exception {
|
||||
init(HoodieTableType.COPY_ON_WRITE, false);
|
||||
HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc);
|
||||
|
||||
@@ -2148,14 +2148,20 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
||||
@MethodSource("rollbackFailedCommitsParams")
|
||||
public void testRollbackFailedCommits(HoodieFailedWritesCleaningPolicy cleaningPolicy, boolean populateMetaFields) throws Exception {
|
||||
HoodieTestUtils.init(hadoopConf, basePath);
|
||||
// Perform 2 failed writes to table
|
||||
SparkRDDWriteClient client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, populateMetaFields));
|
||||
|
||||
// perform 1 successfull commit
|
||||
writeBatch(client, "100", "100", Option.of(Arrays.asList("100")), "100",
|
||||
100, dataGen::generateInserts, SparkRDDWriteClient::bulkInsert, false, 100, 300,
|
||||
0, true);
|
||||
|
||||
// Perform 2 failed writes to table
|
||||
writeBatch(client, "200", "100", Option.of(Arrays.asList("200")), "100",
|
||||
100, dataGen::generateInserts, SparkRDDWriteClient::bulkInsert, false, 100, 300,
|
||||
0, false);
|
||||
client.close();
|
||||
client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, populateMetaFields));
|
||||
writeBatch(client, "200", "100", Option.of(Arrays.asList("200")), "200",
|
||||
writeBatch(client, "300", "200", Option.of(Arrays.asList("300")), "300",
|
||||
100, dataGen::generateInserts, SparkRDDWriteClient::bulkInsert, false, 100, 300,
|
||||
0, false);
|
||||
client.close();
|
||||
@@ -2163,7 +2169,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
||||
dataGen = new HoodieTestDataGenerator();
|
||||
// Perform 1 successful write
|
||||
client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, populateMetaFields));
|
||||
writeBatch(client, "300", "200", Option.of(Arrays.asList("300")), "300",
|
||||
writeBatch(client, "400", "300", Option.of(Arrays.asList("400")), "400",
|
||||
100, dataGen::generateInserts, SparkRDDWriteClient::bulkInsert, false, 100, 300,
|
||||
0, true);
|
||||
HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(basePath).build();
|
||||
@@ -2171,16 +2177,16 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
||||
assertTrue(metaClient.getActiveTimeline().getTimelineOfActions(
|
||||
CollectionUtils.createSet(ROLLBACK_ACTION)).countInstants() == 0);
|
||||
assertTrue(metaClient.getActiveTimeline().filterInflights().countInstants() == 2);
|
||||
assertTrue(metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants().countInstants() == 1);
|
||||
assertTrue(metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants().countInstants() == 2);
|
||||
// Await till enough time passes such that the first 2 failed commits heartbeats are expired
|
||||
boolean conditionMet = false;
|
||||
while (!conditionMet) {
|
||||
conditionMet = client.getHeartbeatClient().isHeartbeatExpired("200");
|
||||
conditionMet = client.getHeartbeatClient().isHeartbeatExpired("300");
|
||||
Thread.sleep(2000);
|
||||
}
|
||||
client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, populateMetaFields));
|
||||
// Perform 1 successful write
|
||||
writeBatch(client, "400", "300", Option.of(Arrays.asList("400")), "400",
|
||||
writeBatch(client, "500", "400", Option.of(Arrays.asList("500")), "500",
|
||||
100, dataGen::generateInserts, SparkRDDWriteClient::bulkInsert, false, 100, 300,
|
||||
0, true);
|
||||
client.clean();
|
||||
@@ -2197,7 +2203,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
||||
.getTimelineOfActions(CollectionUtils.createSet(CLEAN_ACTION))
|
||||
.countInstants()
|
||||
== 0);
|
||||
assertTrue(timeline.getCommitsTimeline().filterCompletedInstants().countInstants() == 2);
|
||||
assertTrue(timeline.getCommitsTimeline().filterCompletedInstants().countInstants() == 3);
|
||||
} else if (cleaningPolicy.isNever()) {
|
||||
assertTrue(
|
||||
timeline
|
||||
@@ -2210,7 +2216,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
||||
.getTimelineOfActions(CollectionUtils.createSet(CLEAN_ACTION))
|
||||
.countInstants()
|
||||
== 0);
|
||||
assertTrue(timeline.getCommitsTimeline().filterCompletedInstants().countInstants() == 2);
|
||||
assertTrue(timeline.getCommitsTimeline().filterCompletedInstants().countInstants() == 3);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -2220,8 +2226,13 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
||||
HoodieTestUtils.init(hadoopConf, basePath);
|
||||
HoodieFailedWritesCleaningPolicy cleaningPolicy = EAGER;
|
||||
SparkRDDWriteClient client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, populateMetaFields));
|
||||
// Perform 1 failed writes to table
|
||||
// Perform 1 successful writes to table
|
||||
writeBatch(client, "100", "100", Option.of(Arrays.asList("100")), "100",
|
||||
100, dataGen::generateInserts, SparkRDDWriteClient::bulkInsert, false, 100, 300,
|
||||
0, true);
|
||||
|
||||
// Perform 1 failed writes to table
|
||||
writeBatch(client, "200", "100", Option.of(Arrays.asList("200")), "200",
|
||||
100, dataGen::generateInserts, SparkRDDWriteClient::bulkInsert, false, 100, 300,
|
||||
0, false);
|
||||
client.close();
|
||||
@@ -2229,19 +2240,19 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
||||
cleaningPolicy = HoodieFailedWritesCleaningPolicy.LAZY;
|
||||
// Perform 2 failed writes to table
|
||||
client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, populateMetaFields));
|
||||
writeBatch(client, "200", "100", Option.of(Arrays.asList("200")), "200",
|
||||
100, dataGen::generateInserts, SparkRDDWriteClient::bulkInsert, false, 100, 300,
|
||||
0, false);
|
||||
client.close();
|
||||
client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, populateMetaFields));
|
||||
writeBatch(client, "300", "200", Option.of(Arrays.asList("300")), "300",
|
||||
100, dataGen::generateInserts, SparkRDDWriteClient::bulkInsert, false, 100, 300,
|
||||
0, false);
|
||||
client.close();
|
||||
// Await till enough time passes such that the first 2 failed commits heartbeats are expired
|
||||
client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, populateMetaFields));
|
||||
writeBatch(client, "400", "300", Option.of(Arrays.asList("400")), "400",
|
||||
100, dataGen::generateInserts, SparkRDDWriteClient::bulkInsert, false, 100, 300,
|
||||
0, false);
|
||||
client.close();
|
||||
// Await till enough time passes such that the 2 failed commits heartbeats are expired
|
||||
boolean conditionMet = false;
|
||||
while (!conditionMet) {
|
||||
conditionMet = client.getHeartbeatClient().isHeartbeatExpired("300");
|
||||
conditionMet = client.getHeartbeatClient().isHeartbeatExpired("400");
|
||||
Thread.sleep(2000);
|
||||
}
|
||||
client.clean();
|
||||
@@ -2250,12 +2261,12 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
||||
CollectionUtils.createSet(ROLLBACK_ACTION)).countInstants() == 3);
|
||||
// Perform 2 failed commits
|
||||
client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, populateMetaFields));
|
||||
writeBatch(client, "400", "300", Option.of(Arrays.asList("400")), "400",
|
||||
writeBatch(client, "500", "400", Option.of(Arrays.asList("300")), "300",
|
||||
100, dataGen::generateInserts, SparkRDDWriteClient::bulkInsert, false, 100, 300,
|
||||
0, false);
|
||||
client.close();
|
||||
client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, populateMetaFields));
|
||||
writeBatch(client, "500", "400", Option.of(Arrays.asList("500")), "500",
|
||||
writeBatch(client, "600", "500", Option.of(Arrays.asList("400")), "400",
|
||||
100, dataGen::generateInserts, SparkRDDWriteClient::bulkInsert, false, 100, 300,
|
||||
0, false);
|
||||
client.close();
|
||||
@@ -2266,7 +2277,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
||||
timeline = metaClient.getActiveTimeline().reload();
|
||||
assertTrue(timeline.getTimelineOfActions(
|
||||
CollectionUtils.createSet(ROLLBACK_ACTION)).countInstants() == 5);
|
||||
assertTrue(timeline.getCommitsTimeline().filterCompletedInstants().countInstants() == 0);
|
||||
assertTrue(timeline.getCommitsTimeline().filterCompletedInstants().countInstants() == 1);
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -2274,14 +2285,19 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
||||
HoodieFailedWritesCleaningPolicy cleaningPolicy = HoodieFailedWritesCleaningPolicy.LAZY;
|
||||
ExecutorService service = Executors.newFixedThreadPool(2);
|
||||
HoodieTestUtils.init(hadoopConf, basePath);
|
||||
// Perform 2 failed writes to table
|
||||
SparkRDDWriteClient client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, true));
|
||||
// perform 1 successfull write
|
||||
writeBatch(client, "100", "100", Option.of(Arrays.asList("100")), "100",
|
||||
100, dataGen::generateInserts, SparkRDDWriteClient::bulkInsert, false, 100, 100,
|
||||
0, true);
|
||||
|
||||
// Perform 2 failed writes to table
|
||||
writeBatch(client, "200", "100", Option.of(Arrays.asList("200")), "200",
|
||||
100, dataGen::generateInserts, SparkRDDWriteClient::bulkInsert, false, 100, 100,
|
||||
0, false);
|
||||
client.close();
|
||||
client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, true));
|
||||
writeBatch(client, "200", "200", Option.of(Arrays.asList("200")), "200",
|
||||
writeBatch(client, "300", "200", Option.of(Arrays.asList("300")), "300",
|
||||
100, dataGen::generateInserts, SparkRDDWriteClient::bulkInsert, false, 100, 100,
|
||||
0, false);
|
||||
client.close();
|
||||
@@ -2289,7 +2305,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
||||
dataGen = new HoodieTestDataGenerator();
|
||||
// Create a succesful commit
|
||||
Future<JavaRDD<WriteStatus>> commit3 = service.submit(() -> writeBatch(new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, true)),
|
||||
"300", "200", Option.of(Arrays.asList("300")), "200", 100, dataGen::generateInserts,
|
||||
"400", "300", Option.of(Arrays.asList("400")), "300", 100, dataGen::generateInserts,
|
||||
SparkRDDWriteClient::bulkInsert, false, 100, 100, 0, true));
|
||||
commit3.get();
|
||||
HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(basePath).build();
|
||||
@@ -2297,16 +2313,16 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
||||
assertTrue(metaClient.getActiveTimeline().getTimelineOfActions(
|
||||
CollectionUtils.createSet(ROLLBACK_ACTION)).countInstants() == 0);
|
||||
assertTrue(metaClient.getActiveTimeline().filterInflights().countInstants() == 2);
|
||||
assertTrue(metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants().countInstants() == 1);
|
||||
assertTrue(metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants().countInstants() == 2);
|
||||
client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, true));
|
||||
// Await till enough time passes such that the first 2 failed commits heartbeats are expired
|
||||
boolean conditionMet = false;
|
||||
while (!conditionMet) {
|
||||
conditionMet = client.getHeartbeatClient().isHeartbeatExpired("200");
|
||||
conditionMet = client.getHeartbeatClient().isHeartbeatExpired("300");
|
||||
Thread.sleep(2000);
|
||||
}
|
||||
Future<JavaRDD<WriteStatus>> commit4 = service.submit(() -> writeBatch(new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, true)),
|
||||
"400", "300", Option.of(Arrays.asList("400")), "400", 100, dataGen::generateInserts,
|
||||
"500", "400", Option.of(Arrays.asList("500")), "500", 100, dataGen::generateInserts,
|
||||
SparkRDDWriteClient::bulkInsert, false, 100, 100, 0, true));
|
||||
Future<HoodieCleanMetadata> clean1 = service.submit(() -> new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, true)).clean());
|
||||
commit4.get();
|
||||
@@ -2317,7 +2333,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
||||
// Since we write rollbacks not clean, there should be no clean action on the timeline
|
||||
assertTrue(timeline.getTimelineOfActions(
|
||||
CollectionUtils.createSet(CLEAN_ACTION)).countInstants() == 0);
|
||||
assertTrue(timeline.getCommitsTimeline().filterCompletedInstants().countInstants() == 2);
|
||||
assertTrue(timeline.getCommitsTimeline().filterCompletedInstants().countInstants() == 3);
|
||||
}
|
||||
|
||||
private Pair<Path, JavaRDD<WriteStatus>> testConsistencyCheck(HoodieTableMetaClient metaClient, String instantTime, boolean enableOptimisticConsistencyGuard)
|
||||
|
||||
@@ -1298,7 +1298,7 @@ public class TestCleaner extends HoodieClientTestBase {
|
||||
@Test
|
||||
public void testCleanMarkerDataFilesOnRollback() throws Exception {
|
||||
HoodieTestTable testTable = HoodieTestTable.of(metaClient)
|
||||
.addRequestedCommit("000")
|
||||
.addRequestedCommit("001")
|
||||
.withMarkerFiles("default", 10, IOType.MERGE);
|
||||
final int numTempFilesBefore = testTable.listAllFilesInTempFolder().length;
|
||||
assertEquals(10, numTempFilesBefore, "Some marker files are created.");
|
||||
@@ -1310,11 +1310,11 @@ public class TestCleaner extends HoodieClientTestBase {
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTable table = HoodieSparkTable.create(config, context, metaClient);
|
||||
table.getActiveTimeline().transitionRequestedToInflight(
|
||||
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "000"), Option.empty());
|
||||
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "001"), Option.empty());
|
||||
metaClient.reloadActiveTimeline();
|
||||
HoodieInstant rollbackInstant = new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, "000");
|
||||
table.scheduleRollback(context, "001", rollbackInstant, false, config.shouldRollbackUsingMarkers());
|
||||
table.rollback(context, "001", rollbackInstant, true, false);
|
||||
HoodieInstant rollbackInstant = new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, "001");
|
||||
table.scheduleRollback(context, "002", rollbackInstant, false, config.shouldRollbackUsingMarkers());
|
||||
table.rollback(context, "002", rollbackInstant, true, false);
|
||||
final int numTempFilesAfter = testTable.listAllFilesInTempFolder().length;
|
||||
assertEquals(0, numTempFilesAfter, "All temp files are deleted.");
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user