[HUDI-1483] Support async clustering for deltastreamer and Spark streaming (#3142)
- Integrate async clustering service with HoodieDeltaStreamer and HoodieStreamingSink - Added methods in HoodieAsyncService to reuse code
This commit is contained in:
@@ -40,6 +40,7 @@ import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.ReflectionUtils;
|
||||
import org.apache.hudi.common.util.ValidationUtils;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieClusteringConfig;
|
||||
import org.apache.hudi.config.HoodieCompactionConfig;
|
||||
import org.apache.hudi.config.HoodiePayloadConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
@@ -91,8 +92,10 @@ import java.util.stream.Collectors;
|
||||
import scala.collection.JavaConversions;
|
||||
|
||||
import static org.apache.hudi.common.table.HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP;
|
||||
import static org.apache.hudi.config.HoodieClusteringConfig.ASYNC_CLUSTERING_ENABLE_OPT_KEY;
|
||||
import static org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.CHECKPOINT_KEY;
|
||||
import static org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.CHECKPOINT_RESET_KEY;
|
||||
import static org.apache.hudi.config.HoodieClusteringConfig.INLINE_CLUSTERING_PROP;
|
||||
import static org.apache.hudi.config.HoodieCompactionConfig.INLINE_COMPACT_PROP;
|
||||
import static org.apache.hudi.config.HoodieWriteConfig.COMBINE_BEFORE_INSERT_PROP;
|
||||
import static org.apache.hudi.config.HoodieWriteConfig.COMBINE_BEFORE_UPSERT_PROP;
|
||||
@@ -645,6 +648,9 @@ public class DeltaSync implements Serializable {
|
||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder().withPayloadClass(cfg.payloadClassName)
|
||||
// Inline compaction is disabled for continuous mode. otherwise enabled for MOR
|
||||
.withInlineCompaction(cfg.isInlineCompactionEnabled()).build())
|
||||
.withClusteringConfig(HoodieClusteringConfig.newBuilder()
|
||||
.withInlineClustering(cfg.isInlineClusteringEnabled())
|
||||
.withAsyncClustering(cfg.isAsyncClusteringEnabled()).build())
|
||||
.withPayloadConfig(HoodiePayloadConfig.newBuilder().withPayloadOrderingField(cfg.sourceOrderingField)
|
||||
.build())
|
||||
.forTable(cfg.targetTableName)
|
||||
@@ -663,6 +669,10 @@ public class DeltaSync implements Serializable {
|
||||
// Validate what deltastreamer assumes of write-config to be really safe
|
||||
ValidationUtils.checkArgument(config.inlineCompactionEnabled() == cfg.isInlineCompactionEnabled(),
|
||||
String.format("%s should be set to %s", INLINE_COMPACT_PROP.key(), cfg.isInlineCompactionEnabled()));
|
||||
ValidationUtils.checkArgument(config.inlineClusteringEnabled() == cfg.isInlineClusteringEnabled(),
|
||||
String.format("%s should be set to %s", INLINE_CLUSTERING_PROP.key(), cfg.isInlineClusteringEnabled()));
|
||||
ValidationUtils.checkArgument(config.isAsyncClusteringEnabled() == cfg.isAsyncClusteringEnabled(),
|
||||
String.format("%s should be set to %s", ASYNC_CLUSTERING_ENABLE_OPT_KEY.key(), cfg.isAsyncClusteringEnabled()));
|
||||
ValidationUtils.checkArgument(!config.shouldAutoCommit(),
|
||||
String.format("%s should be set to %s", HOODIE_AUTO_COMMIT_PROP.key(), autoCommit));
|
||||
ValidationUtils.checkArgument(config.shouldCombineBeforeInsert() == cfg.filterDupes,
|
||||
@@ -736,4 +746,14 @@ public class DeltaSync implements Serializable {
|
||||
public Option<HoodieTimeline> getCommitTimelineOpt() {
|
||||
return commitTimelineOpt;
|
||||
}
|
||||
|
||||
/**
|
||||
* Schedule clustering.
|
||||
* Called from {@link HoodieDeltaStreamer} when async clustering is enabled.
|
||||
*
|
||||
* @return Requested clustering instant.
|
||||
*/
|
||||
public Option<String> getClusteringInstantOpt() {
|
||||
return writeClient.scheduleClustering(Option.empty());
|
||||
}
|
||||
}
|
||||
|
||||
@@ -18,8 +18,10 @@
|
||||
|
||||
package org.apache.hudi.utilities.deltastreamer;
|
||||
|
||||
import org.apache.hudi.async.HoodieAsyncService;
|
||||
import org.apache.hudi.async.AsyncClusteringService;
|
||||
import org.apache.hudi.async.AsyncCompactService;
|
||||
import org.apache.hudi.async.HoodieAsyncService;
|
||||
import org.apache.hudi.async.SparkAsyncClusteringService;
|
||||
import org.apache.hudi.async.SparkAsyncCompactService;
|
||||
import org.apache.hudi.client.SparkRDDWriteClient;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
@@ -35,15 +37,17 @@ import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant.State;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.util.ClusteringUtils;
|
||||
import org.apache.hudi.common.util.CompactionUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.ValidationUtils;
|
||||
import org.apache.hudi.utilities.IdentitySplitter;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieClusteringConfig;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.hive.HiveSyncTool;
|
||||
import org.apache.hudi.utilities.HiveIncrementalPuller;
|
||||
import org.apache.hudi.utilities.IdentitySplitter;
|
||||
import org.apache.hudi.utilities.UtilHelpers;
|
||||
import org.apache.hudi.utilities.checkpointing.InitialCheckPointProvider;
|
||||
import org.apache.hudi.utilities.schema.SchemaProvider;
|
||||
@@ -282,6 +286,11 @@ public class HoodieDeltaStreamer implements Serializable {
|
||||
+ "outstanding compactions is less than this number")
|
||||
public Integer maxPendingCompactions = 5;
|
||||
|
||||
@Parameter(names = {"--max-pending-clustering"},
|
||||
description = "Maximum number of outstanding inflight/requested clustering. Delta Sync will not happen unless"
|
||||
+ "outstanding clustering is less than this number")
|
||||
public Integer maxPendingClustering = 5;
|
||||
|
||||
@Parameter(names = {"--continuous"}, description = "Delta Streamer runs in continuous mode running"
|
||||
+ " source-fetch -> Transform -> Hudi Write in loop")
|
||||
public Boolean continuousMode = false;
|
||||
@@ -351,6 +360,16 @@ public class HoodieDeltaStreamer implements Serializable {
|
||||
&& HoodieTableType.MERGE_ON_READ.equals(HoodieTableType.valueOf(tableType));
|
||||
}
|
||||
|
||||
public boolean isAsyncClusteringEnabled() {
|
||||
return Boolean.parseBoolean(String.valueOf(UtilHelpers.getConfig(this.configs).getConfig()
|
||||
.getOrDefault(HoodieClusteringConfig.ASYNC_CLUSTERING_ENABLE_OPT_KEY.key(), false)));
|
||||
}
|
||||
|
||||
public boolean isInlineClusteringEnabled() {
|
||||
return Boolean.parseBoolean(String.valueOf(UtilHelpers.getConfig(this.configs).getConfig()
|
||||
.getOrDefault(HoodieClusteringConfig.INLINE_CLUSTERING_PROP.key(), false)));
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o) {
|
||||
@@ -376,6 +395,7 @@ public class HoodieDeltaStreamer implements Serializable {
|
||||
&& Objects.equals(filterDupes, config.filterDupes)
|
||||
&& Objects.equals(enableHiveSync, config.enableHiveSync)
|
||||
&& Objects.equals(maxPendingCompactions, config.maxPendingCompactions)
|
||||
&& Objects.equals(maxPendingClustering, config.maxPendingClustering)
|
||||
&& Objects.equals(continuousMode, config.continuousMode)
|
||||
&& Objects.equals(minSyncIntervalSeconds, config.minSyncIntervalSeconds)
|
||||
&& Objects.equals(sparkMaster, config.sparkMaster)
|
||||
@@ -396,7 +416,7 @@ public class HoodieDeltaStreamer implements Serializable {
|
||||
baseFileFormat, propsFilePath, configs, sourceClassName,
|
||||
sourceOrderingField, payloadClassName, schemaProviderClassName,
|
||||
transformerClassNames, sourceLimit, operation, filterDupes,
|
||||
enableHiveSync, maxPendingCompactions, continuousMode,
|
||||
enableHiveSync, maxPendingCompactions, maxPendingClustering, continuousMode,
|
||||
minSyncIntervalSeconds, sparkMaster, commitOnErrors,
|
||||
deltaSyncSchedulingWeight, compactSchedulingWeight, deltaSyncSchedulingMinShare,
|
||||
compactSchedulingMinShare, forceDisableCompaction, checkpoint,
|
||||
@@ -422,6 +442,7 @@ public class HoodieDeltaStreamer implements Serializable {
|
||||
+ ", filterDupes=" + filterDupes
|
||||
+ ", enableHiveSync=" + enableHiveSync
|
||||
+ ", maxPendingCompactions=" + maxPendingCompactions
|
||||
+ ", maxPendingClustering=" + maxPendingClustering
|
||||
+ ", continuousMode=" + continuousMode
|
||||
+ ", minSyncIntervalSeconds=" + minSyncIntervalSeconds
|
||||
+ ", sparkMaster='" + sparkMaster + '\''
|
||||
@@ -519,6 +540,11 @@ public class HoodieDeltaStreamer implements Serializable {
|
||||
*/
|
||||
private Option<AsyncCompactService> asyncCompactService;
|
||||
|
||||
/**
|
||||
* Async clustering service.
|
||||
*/
|
||||
private Option<AsyncClusteringService> asyncClusteringService;
|
||||
|
||||
/**
|
||||
* Table Type.
|
||||
*/
|
||||
@@ -535,6 +561,7 @@ public class HoodieDeltaStreamer implements Serializable {
|
||||
this.jssc = jssc;
|
||||
this.sparkSession = SparkSession.builder().config(jssc.getConf()).getOrCreate();
|
||||
this.asyncCompactService = Option.empty();
|
||||
this.asyncClusteringService = Option.empty();
|
||||
|
||||
if (fs.exists(new Path(cfg.targetBasePath))) {
|
||||
HoodieTableMetaClient meta =
|
||||
@@ -598,9 +625,17 @@ public class HoodieDeltaStreamer implements Serializable {
|
||||
Option<Pair<Option<String>, JavaRDD<WriteStatus>>> scheduledCompactionInstantAndRDD = Option.ofNullable(deltaSync.syncOnce());
|
||||
if (scheduledCompactionInstantAndRDD.isPresent() && scheduledCompactionInstantAndRDD.get().getLeft().isPresent()) {
|
||||
LOG.info("Enqueuing new pending compaction instant (" + scheduledCompactionInstantAndRDD.get().getLeft() + ")");
|
||||
asyncCompactService.get().enqueuePendingCompaction(new HoodieInstant(State.REQUESTED,
|
||||
asyncCompactService.get().enqueuePendingAsyncServiceInstant(new HoodieInstant(State.REQUESTED,
|
||||
HoodieTimeline.COMPACTION_ACTION, scheduledCompactionInstantAndRDD.get().getLeft().get()));
|
||||
asyncCompactService.get().waitTillPendingCompactionsReducesTo(cfg.maxPendingCompactions);
|
||||
asyncCompactService.get().waitTillPendingAsyncServiceInstantsReducesTo(cfg.maxPendingCompactions);
|
||||
}
|
||||
if (cfg.isAsyncClusteringEnabled()) {
|
||||
Option<String> clusteringInstant = deltaSync.getClusteringInstantOpt();
|
||||
if (clusteringInstant.isPresent()) {
|
||||
LOG.info("Scheduled async clustering for instant: " + clusteringInstant.get());
|
||||
asyncClusteringService.get().enqueuePendingAsyncServiceInstant(new HoodieInstant(State.REQUESTED, HoodieTimeline.REPLACE_COMMIT_ACTION, clusteringInstant.get()));
|
||||
asyncClusteringService.get().waitTillPendingAsyncServiceInstantsReducesTo(cfg.maxPendingClustering);
|
||||
}
|
||||
}
|
||||
long toSleepMs = cfg.minSyncIntervalSeconds * 1000 - (System.currentTimeMillis() - start);
|
||||
if (toSleepMs > 0) {
|
||||
@@ -615,21 +650,25 @@ public class HoodieDeltaStreamer implements Serializable {
|
||||
}
|
||||
}
|
||||
} finally {
|
||||
shutdownCompactor(error);
|
||||
shutdownAsyncServices(error);
|
||||
}
|
||||
return true;
|
||||
}, executor), executor);
|
||||
}
|
||||
|
||||
/**
|
||||
* Shutdown compactor as DeltaSync is shutdown.
|
||||
* Shutdown async services like compaction/clustering as DeltaSync is shutdown.
|
||||
*/
|
||||
private void shutdownCompactor(boolean error) {
|
||||
private void shutdownAsyncServices(boolean error) {
|
||||
LOG.info("Delta Sync shutdown. Error ?" + error);
|
||||
if (asyncCompactService.isPresent()) {
|
||||
LOG.warn("Gracefully shutting down compactor");
|
||||
asyncCompactService.get().shutdown(false);
|
||||
}
|
||||
if (asyncClusteringService.isPresent()) {
|
||||
LOG.warn("Gracefully shutting down clustering service");
|
||||
asyncClusteringService.get().shutdown(false);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -649,19 +688,43 @@ public class HoodieDeltaStreamer implements Serializable {
|
||||
HoodieTableMetaClient meta =
|
||||
HoodieTableMetaClient.builder().setConf(new Configuration(jssc.hadoopConfiguration())).setBasePath(cfg.targetBasePath).setLoadActiveTimelineOnLoad(true).build();
|
||||
List<HoodieInstant> pending = CompactionUtils.getPendingCompactionInstantTimes(meta);
|
||||
pending.forEach(hoodieInstant -> asyncCompactService.get().enqueuePendingCompaction(hoodieInstant));
|
||||
pending.forEach(hoodieInstant -> asyncCompactService.get().enqueuePendingAsyncServiceInstant(hoodieInstant));
|
||||
asyncCompactService.get().start((error) -> {
|
||||
// Shutdown DeltaSync
|
||||
shutdown(false);
|
||||
return true;
|
||||
});
|
||||
try {
|
||||
asyncCompactService.get().waitTillPendingCompactionsReducesTo(cfg.maxPendingCompactions);
|
||||
asyncCompactService.get().waitTillPendingAsyncServiceInstantsReducesTo(cfg.maxPendingCompactions);
|
||||
} catch (InterruptedException ie) {
|
||||
throw new HoodieException(ie);
|
||||
}
|
||||
}
|
||||
}
|
||||
// start async clustering if required
|
||||
if (cfg.isAsyncClusteringEnabled()) {
|
||||
if (asyncClusteringService.isPresent()) {
|
||||
asyncClusteringService.get().updateWriteClient(writeClient);
|
||||
} else {
|
||||
asyncClusteringService = Option.ofNullable(new SparkAsyncClusteringService(writeClient));
|
||||
HoodieTableMetaClient meta = HoodieTableMetaClient.builder()
|
||||
.setConf(new Configuration(jssc.hadoopConfiguration()))
|
||||
.setBasePath(cfg.targetBasePath)
|
||||
.setLoadActiveTimelineOnLoad(true).build();
|
||||
List<HoodieInstant> pending = ClusteringUtils.getPendingClusteringInstantTimes(meta);
|
||||
LOG.info(String.format("Found %d pending clustering instants ", pending.size()));
|
||||
pending.forEach(hoodieInstant -> asyncClusteringService.get().enqueuePendingAsyncServiceInstant(hoodieInstant));
|
||||
asyncClusteringService.get().start((error) -> {
|
||||
shutdown(false);
|
||||
return true;
|
||||
});
|
||||
try {
|
||||
asyncClusteringService.get().waitTillPendingAsyncServiceInstantsReducesTo(cfg.maxPendingClustering);
|
||||
} catch (InterruptedException e) {
|
||||
throw new HoodieException(e);
|
||||
}
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
|
||||
@@ -193,6 +193,7 @@ public class HoodieMultiTableDeltaStreamer {
|
||||
tableConfig.payloadClassName = globalConfig.payloadClassName;
|
||||
tableConfig.forceDisableCompaction = globalConfig.forceDisableCompaction;
|
||||
tableConfig.maxPendingCompactions = globalConfig.maxPendingCompactions;
|
||||
tableConfig.maxPendingClustering = globalConfig.maxPendingClustering;
|
||||
tableConfig.minSyncIntervalSeconds = globalConfig.minSyncIntervalSeconds;
|
||||
tableConfig.transformerClassNames = globalConfig.transformerClassNames;
|
||||
tableConfig.commitOnErrors = globalConfig.commitOnErrors;
|
||||
@@ -296,6 +297,11 @@ public class HoodieMultiTableDeltaStreamer {
|
||||
+ "outstanding compactions is less than this number")
|
||||
public Integer maxPendingCompactions = 5;
|
||||
|
||||
@Parameter(names = {"--max-pending-clustering"},
|
||||
description = "Maximum number of outstanding inflight/requested clustering. Delta Sync will not happen unless"
|
||||
+ "outstanding clustering is less than this number")
|
||||
public Integer maxPendingClustering = 5;
|
||||
|
||||
@Parameter(names = {"--continuous"}, description = "Delta Streamer runs in continuous mode running"
|
||||
+ " source-fetch -> Transform -> Hudi Write in loop")
|
||||
public Boolean continuousMode = false;
|
||||
|
||||
@@ -38,6 +38,7 @@ import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.StringUtils;
|
||||
import org.apache.hudi.config.HoodieClusteringConfig;
|
||||
import org.apache.hudi.config.HoodieCompactionConfig;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
@@ -500,6 +501,14 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
|
||||
int numDeltaCommits = (int) timeline.getInstants().count();
|
||||
assertTrue(minExpected <= numDeltaCommits, "Got=" + numDeltaCommits + ", exp >=" + minExpected);
|
||||
}
|
||||
|
||||
static void assertAtLeastNReplaceCommits(int minExpected, String tablePath, FileSystem fs) {
|
||||
HoodieTableMetaClient meta = HoodieTableMetaClient.builder().setConf(fs.getConf()).setBasePath(tablePath).setLoadActiveTimelineOnLoad(true).build();
|
||||
HoodieTimeline timeline = meta.getActiveTimeline().getCompletedReplaceTimeline();
|
||||
LOG.info("Timeline Instants=" + meta.getActiveTimeline().getInstants().collect(Collectors.toList()));
|
||||
int numDeltaCommits = (int) timeline.getInstants().count();
|
||||
assertTrue(minExpected <= numDeltaCommits, "Got=" + numDeltaCommits + ", exp >=" + minExpected);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -987,20 +996,35 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
|
||||
HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(tableBasePath, WriteOperationType.UPSERT);
|
||||
cfg.continuousMode = true;
|
||||
cfg.tableType = HoodieTableType.MERGE_ON_READ.name();
|
||||
cfg.configs.add(String.format("%s=%d", SourceConfigs.MAX_UNIQUE_RECORDS_PROP, totalRecords));
|
||||
cfg.configs.add(String.format("%s=false", HoodieCompactionConfig.AUTO_CLEAN_PROP.key()));
|
||||
cfg.configs.add(String.format("%s=%s", HoodieClusteringConfig.INLINE_CLUSTERING_PROP.key(), "true"));
|
||||
cfg.configs.add(String.format("%s=%s", HoodieClusteringConfig.INLINE_CLUSTERING_MAX_COMMIT_PROP.key(), "2"));
|
||||
cfg.configs.addAll(getAsyncServicesConfigs(totalRecords, "false", "true", "2", "", ""));
|
||||
HoodieDeltaStreamer ds = new HoodieDeltaStreamer(cfg, jsc);
|
||||
deltaStreamerTestRunner(ds, cfg, (r) -> {
|
||||
HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(this.dfs.getConf()).setBasePath(tableBasePath).setLoadActiveTimelineOnLoad(true).build();
|
||||
int pendingReplaceSize = metaClient.getActiveTimeline().filterPendingReplaceTimeline().getInstants().toArray().length;
|
||||
int completeReplaceSize = metaClient.getActiveTimeline().getCompletedReplaceTimeline().getInstants().toArray().length;
|
||||
LOG.info("PendingReplaceSize=" + pendingReplaceSize + ",completeReplaceSize = " + completeReplaceSize);
|
||||
return completeReplaceSize > 0;
|
||||
TestHelpers.assertAtLeastNCommits(2, tableBasePath, dfs);
|
||||
TestHelpers.assertAtLeastNReplaceCommits(1, tableBasePath, dfs);
|
||||
return true;
|
||||
});
|
||||
HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(this.dfs.getConf()).setBasePath(tableBasePath).setLoadActiveTimelineOnLoad(true).build();
|
||||
assertEquals(1, metaClient.getActiveTimeline().getCompletedReplaceTimeline().getInstants().toArray().length);
|
||||
}
|
||||
|
||||
private List<String> getAsyncServicesConfigs(int totalRecords, String autoClean, String inlineCluster,
|
||||
String inlineClusterMaxCommit, String asyncCluster, String asyncClusterMaxCommit) {
|
||||
List<String> configs = new ArrayList<>();
|
||||
configs.add(String.format("%s=%d", SourceConfigs.MAX_UNIQUE_RECORDS_PROP, totalRecords));
|
||||
if (!StringUtils.isNullOrEmpty(autoClean)) {
|
||||
configs.add(String.format("%s=%s", HoodieCompactionConfig.AUTO_CLEAN_PROP.key(), autoClean));
|
||||
}
|
||||
if (!StringUtils.isNullOrEmpty(inlineCluster)) {
|
||||
configs.add(String.format("%s=%s", HoodieClusteringConfig.INLINE_CLUSTERING_PROP.key(), inlineCluster));
|
||||
}
|
||||
if (!StringUtils.isNullOrEmpty(inlineClusterMaxCommit)) {
|
||||
configs.add(String.format("%s=%s", HoodieClusteringConfig.INLINE_CLUSTERING_MAX_COMMIT_PROP.key(), inlineClusterMaxCommit));
|
||||
}
|
||||
if (!StringUtils.isNullOrEmpty(asyncCluster)) {
|
||||
configs.add(String.format("%s=%s", HoodieClusteringConfig.ASYNC_CLUSTERING_ENABLE_OPT_KEY.key(), asyncCluster));
|
||||
}
|
||||
if (!StringUtils.isNullOrEmpty(asyncClusterMaxCommit)) {
|
||||
configs.add(String.format("%s=%s", HoodieClusteringConfig.ASYNC_CLUSTERING_MAX_COMMIT_PROP.key(), asyncClusterMaxCommit));
|
||||
}
|
||||
return configs;
|
||||
}
|
||||
|
||||
private HoodieClusteringJob.Config buildHoodieClusteringUtilConfig(String basePath,
|
||||
@@ -1023,9 +1047,7 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
|
||||
HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(tableBasePath, WriteOperationType.INSERT);
|
||||
cfg.continuousMode = true;
|
||||
cfg.tableType = HoodieTableType.COPY_ON_WRITE.name();
|
||||
cfg.configs.add(String.format("%s=%d", SourceConfigs.MAX_UNIQUE_RECORDS_PROP, totalRecords));
|
||||
cfg.configs.add(String.format("%s=false", HoodieCompactionConfig.AUTO_CLEAN_PROP.key()));
|
||||
cfg.configs.add(String.format("%s=true", HoodieClusteringConfig.ASYNC_CLUSTERING_ENABLE_OPT_KEY.key()));
|
||||
cfg.configs.addAll(getAsyncServicesConfigs(totalRecords, "false", "", "", "true", ""));
|
||||
HoodieDeltaStreamer ds = new HoodieDeltaStreamer(cfg, jsc);
|
||||
deltaStreamerTestRunner(ds, cfg, (r) -> {
|
||||
TestHelpers.assertAtLeastNCommits(2, tableBasePath, dfs);
|
||||
@@ -1049,14 +1071,48 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
|
||||
} else {
|
||||
LOG.warn("Schedule clustering failed");
|
||||
}
|
||||
HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(this.dfs.getConf()).setBasePath(tableBasePath).setLoadActiveTimelineOnLoad(true).build();
|
||||
int pendingReplaceSize = metaClient.getActiveTimeline().filterPendingReplaceTimeline().getInstants().toArray().length;
|
||||
int completeReplaceSize = metaClient.getActiveTimeline().getCompletedReplaceTimeline().getInstants().toArray().length;
|
||||
System.out.println("PendingReplaceSize=" + pendingReplaceSize + ",completeReplaceSize = " + completeReplaceSize);
|
||||
return completeReplaceSize > 0;
|
||||
TestHelpers.assertAtLeastNReplaceCommits(2, tableBasePath, dfs);
|
||||
return true;
|
||||
});
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAsyncClusteringService() throws Exception {
|
||||
String tableBasePath = dfsBasePath + "/asyncClustering";
|
||||
// Keep it higher than batch-size to test continuous mode
|
||||
int totalRecords = 3000;
|
||||
|
||||
// Initial bulk insert
|
||||
HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(tableBasePath, WriteOperationType.INSERT);
|
||||
cfg.continuousMode = true;
|
||||
cfg.tableType = HoodieTableType.COPY_ON_WRITE.name();
|
||||
cfg.configs.addAll(getAsyncServicesConfigs(totalRecords, "false", "", "", "true", "2"));
|
||||
HoodieDeltaStreamer ds = new HoodieDeltaStreamer(cfg, jsc);
|
||||
deltaStreamerTestRunner(ds, cfg, (r) -> {
|
||||
TestHelpers.assertAtLeastNCommits(2, tableBasePath, dfs);
|
||||
TestHelpers.assertAtLeastNReplaceCommits(2, tableBasePath, dfs);
|
||||
return true;
|
||||
});
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAsyncClusteringServiceWithCompaction() throws Exception {
|
||||
String tableBasePath = dfsBasePath + "/asyncClusteringCompaction";
|
||||
// Keep it higher than batch-size to test continuous mode
|
||||
int totalRecords = 3000;
|
||||
|
||||
// Initial bulk insert
|
||||
HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(tableBasePath, WriteOperationType.INSERT);
|
||||
cfg.continuousMode = true;
|
||||
cfg.tableType = HoodieTableType.MERGE_ON_READ.name();
|
||||
cfg.configs.addAll(getAsyncServicesConfigs(totalRecords, "false", "", "", "true", "2"));
|
||||
HoodieDeltaStreamer ds = new HoodieDeltaStreamer(cfg, jsc);
|
||||
deltaStreamerTestRunner(ds, cfg, (r) -> {
|
||||
TestHelpers.assertAtLeastNCommits(2, tableBasePath, dfs);
|
||||
TestHelpers.assertAtleastNCompactionCommits(2, tableBasePath, dfs);
|
||||
TestHelpers.assertAtLeastNReplaceCommits(2, tableBasePath, dfs);
|
||||
return true;
|
||||
});
|
||||
HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(this.dfs.getConf()).setBasePath(tableBasePath).setLoadActiveTimelineOnLoad(true).build();
|
||||
assertEquals(1, metaClient.getActiveTimeline().getCompletedReplaceTimeline().getInstants().toArray().length);
|
||||
}
|
||||
|
||||
/**
|
||||
|
||||
Reference in New Issue
Block a user