1
0

[HUDI-1481] add structured streaming and delta streamer clustering unit test (#2360)

This commit is contained in:
lw0090
2020-12-28 12:27:09 +08:00
committed by GitHub
parent 8cf6a7223f
commit 9e6889a8ce
3 changed files with 188 additions and 40 deletions

View File

@@ -33,6 +33,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.config.HoodieClusteringConfig;
import org.apache.hudi.config.HoodieCompactionConfig;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.TableNotFoundException;
@@ -622,23 +623,14 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
String tableBasePath = dfsBasePath + "/" + tempDir;
// Keep it higher than batch-size to test continuous mode
int totalRecords = 3000;
// Initial bulk insert
HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(tableBasePath, WriteOperationType.UPSERT);
cfg.continuousMode = true;
cfg.tableType = tableType.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));
HoodieDeltaStreamer ds = new HoodieDeltaStreamer(cfg, jsc);
Future dsFuture = Executors.newSingleThreadExecutor().submit(() -> {
try {
ds.sync();
} catch (Exception ex) {
throw new RuntimeException(ex.getMessage(), ex);
}
});
TestHelpers.waitTillCondition((r) -> {
deltaStreamerTestRunner(cfg, (r) -> {
if (tableType.equals(HoodieTableType.MERGE_ON_READ)) {
TestHelpers.assertAtleastNDeltaCommits(5, tableBasePath, dfs);
TestHelpers.assertAtleastNCompactionCommits(2, tableBasePath, dfs);
@@ -648,11 +640,48 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
TestHelpers.assertRecordCount(totalRecords, tableBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertDistanceCount(totalRecords, tableBasePath + "/*/*.parquet", sqlContext);
return true;
}, 180);
});
}
private void deltaStreamerTestRunner(HoodieDeltaStreamer.Config cfg, Function<Boolean, Boolean> condition) throws Exception {
HoodieDeltaStreamer ds = new HoodieDeltaStreamer(cfg, jsc);
Future dsFuture = Executors.newSingleThreadExecutor().submit(() -> {
try {
ds.sync();
} catch (Exception ex) {
throw new RuntimeException(ex.getMessage(), ex);
}
});
TestHelpers.waitTillCondition(condition, 180);
ds.shutdownGracefully();
dsFuture.get();
}
@Test
public void testInlineClustering() throws Exception {
String tableBasePath = dfsBasePath + "/inlineClustering";
// Keep it higher than batch-size to test continuous mode
int totalRecords = 3000;
// Initial bulk insert
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));
cfg.configs.add(String.format("%s=%s", HoodieClusteringConfig.INLINE_CLUSTERING_PROP, "true"));
cfg.configs.add(String.format("%s=%s", HoodieClusteringConfig.INLINE_CLUSTERING_MAX_COMMIT_PROP, "2"));
deltaStreamerTestRunner(cfg, (r) -> {
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(this.dfs.getConf(), tableBasePath, true);
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;
});
}
/**
* Test Bulk Insert and upserts with hive syncing. Tests Hudi incremental processing using a 2 step pipeline The first
* step involves using a SQL template to transform a source TEST-DATA-SOURCE ============================> HUDI TABLE