[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:
@@ -156,6 +156,7 @@ public class HoodieJavaApp {
|
||||
nonPartitionedTable ? NonpartitionedKeyGenerator.class.getCanonicalName()
|
||||
: SimpleKeyGenerator.class.getCanonicalName())
|
||||
.option(DataSourceWriteOptions.ASYNC_COMPACT_ENABLE_OPT_KEY().key(), "false")
|
||||
.option(DataSourceWriteOptions.ASYNC_CLUSTERING_ENABLE_OPT_KEY().key(), "true")
|
||||
// This will remove any existing data at path below, and create a
|
||||
.mode(SaveMode.Overwrite);
|
||||
|
||||
@@ -183,6 +184,7 @@ public class HoodieJavaApp {
|
||||
: SimpleKeyGenerator.class.getCanonicalName()) // Add Key Extractor
|
||||
.option(HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS_PROP.key(), "1")
|
||||
.option(DataSourceWriteOptions.ASYNC_COMPACT_ENABLE_OPT_KEY().key(), "false")
|
||||
.option(DataSourceWriteOptions.ASYNC_CLUSTERING_ENABLE_OPT_KEY().key(), "true")
|
||||
.option(HoodieWriteConfig.TABLE_NAME.key(), tableName).mode(SaveMode.Append);
|
||||
|
||||
updateHiveSyncConfig(writer);
|
||||
@@ -210,6 +212,7 @@ public class HoodieJavaApp {
|
||||
: SimpleKeyGenerator.class.getCanonicalName()) // Add Key Extractor
|
||||
.option(HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS_PROP.key(), "1")
|
||||
.option(DataSourceWriteOptions.ASYNC_COMPACT_ENABLE_OPT_KEY().key(), "false")
|
||||
.option(DataSourceWriteOptions.ASYNC_CLUSTERING_ENABLE_OPT_KEY().key(), "true")
|
||||
.option(HoodieWriteConfig.TABLE_NAME.key(), tableName).mode(SaveMode.Append);
|
||||
|
||||
updateHiveSyncConfig(writer);
|
||||
|
||||
@@ -362,6 +362,7 @@ public class HoodieJavaStreamingApp {
|
||||
.option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY().key(), "timestamp")
|
||||
.option(HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS_PROP.key(), "1")
|
||||
.option(DataSourceWriteOptions.ASYNC_COMPACT_ENABLE_OPT_KEY().key(), "true")
|
||||
.option(DataSourceWriteOptions.ASYNC_CLUSTERING_ENABLE_OPT_KEY().key(), "true")
|
||||
.option(HoodieWriteConfig.TABLE_NAME.key(), tableName).option("checkpointLocation", checkpointLocation)
|
||||
.outputMode(OutputMode.Append());
|
||||
|
||||
|
||||
@@ -190,9 +190,13 @@ class TestStructuredStreaming extends HoodieClientTestBase {
|
||||
numInstants
|
||||
}
|
||||
|
||||
def getInlineClusteringOpts( isInlineClustering: String, clusteringNumCommit: String, fileMaxRecordNum: Int):Map[String, String] = {
|
||||
def getClusteringOpts(isInlineClustering: String, isAsyncClustering: String, isAsyncCompaction: String,
|
||||
clusteringNumCommit: String, fileMaxRecordNum: Int):Map[String, String] = {
|
||||
commonOpts + (HoodieClusteringConfig.INLINE_CLUSTERING_PROP.key -> isInlineClustering,
|
||||
HoodieClusteringConfig.INLINE_CLUSTERING_MAX_COMMIT_PROP.key -> clusteringNumCommit,
|
||||
DataSourceWriteOptions.ASYNC_CLUSTERING_ENABLE_OPT_KEY.key -> isAsyncClustering,
|
||||
DataSourceWriteOptions.ASYNC_COMPACT_ENABLE_OPT_KEY.key -> isAsyncCompaction,
|
||||
HoodieClusteringConfig.ASYNC_CLUSTERING_MAX_COMMIT_PROP.key -> clusteringNumCommit,
|
||||
HoodieStorageConfig.PARQUET_FILE_MAX_BYTES.key -> dataGen.getEstimatedFileSizeInBytes(fileMaxRecordNum).toString
|
||||
)
|
||||
}
|
||||
@@ -207,12 +211,40 @@ class TestStructuredStreaming extends HoodieClientTestBase {
|
||||
metaClient.reloadActiveTimeline()
|
||||
assertEquals(1, getLatestFileGroupsFileId(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH).size)
|
||||
}
|
||||
structuredStreamingForTestClusteringRunner(sourcePath, destPath, true,
|
||||
structuredStreamingForTestClusteringRunner(sourcePath, destPath, true, false, false,
|
||||
HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, checkClusteringResult)
|
||||
}
|
||||
|
||||
@Test
|
||||
def testStructuredStreamingWithoutInlineClustering(): Unit = {
|
||||
def testStructuredStreamingWithAsyncClustering(): Unit = {
|
||||
val (sourcePath, destPath) = initStreamingSourceAndDestPath("source", "dest")
|
||||
|
||||
def checkClusteringResult(destPath: String):Unit = {
|
||||
// check have schedule clustering and clustering file group to one
|
||||
waitTillHasCompletedReplaceInstant(destPath, 120, 5)
|
||||
metaClient.reloadActiveTimeline()
|
||||
assertEquals(1, getLatestFileGroupsFileId(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH).size)
|
||||
}
|
||||
structuredStreamingForTestClusteringRunner(sourcePath, destPath, false, true, false,
|
||||
HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, checkClusteringResult)
|
||||
}
|
||||
|
||||
@Test
|
||||
def testStructuredStreamingWithAsyncClusteringAndCompaction(): Unit = {
|
||||
val (sourcePath, destPath) = initStreamingSourceAndDestPath("source", "dest")
|
||||
|
||||
def checkClusteringResult(destPath: String):Unit = {
|
||||
// check have schedule clustering and clustering file group to one
|
||||
waitTillHasCompletedReplaceInstant(destPath, 120, 5)
|
||||
metaClient.reloadActiveTimeline()
|
||||
assertEquals(1, getLatestFileGroupsFileId(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH).size)
|
||||
}
|
||||
structuredStreamingForTestClusteringRunner(sourcePath, destPath, false, true, true,
|
||||
HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, checkClusteringResult)
|
||||
}
|
||||
|
||||
@Test
|
||||
def testStructuredStreamingWithoutClustering(): Unit = {
|
||||
val (sourcePath, destPath) = initStreamingSourceAndDestPath("source", "dest")
|
||||
|
||||
def checkClusteringResult(destPath: String):Unit = {
|
||||
@@ -224,12 +256,13 @@ class TestStructuredStreaming extends HoodieClientTestBase {
|
||||
}, msg)
|
||||
println(msg)
|
||||
}
|
||||
structuredStreamingForTestClusteringRunner(sourcePath, destPath, false,
|
||||
structuredStreamingForTestClusteringRunner(sourcePath, destPath, false, false, false,
|
||||
HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, checkClusteringResult)
|
||||
}
|
||||
|
||||
def structuredStreamingForTestClusteringRunner(sourcePath: String, destPath: String,
|
||||
isInlineClustering: Boolean, partitionOfRecords: String, checkClusteringResult: String => Unit): Unit = {
|
||||
def structuredStreamingForTestClusteringRunner(sourcePath: String, destPath: String, isInlineClustering: Boolean,
|
||||
isAsyncClustering: Boolean, isAsyncCompaction: Boolean,
|
||||
partitionOfRecords: String, checkClusteringResult: String => Unit): Unit = {
|
||||
// First insert of data
|
||||
val records1 = recordsToStrings(dataGen.generateInsertsForPartition("000", 100, partitionOfRecords)).toList
|
||||
val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2))
|
||||
@@ -238,7 +271,8 @@ class TestStructuredStreaming extends HoodieClientTestBase {
|
||||
val records2 = recordsToStrings(dataGen.generateInsertsForPartition("001", 100, partitionOfRecords)).toList
|
||||
val inputDF2 = spark.read.json(spark.sparkContext.parallelize(records2, 2))
|
||||
|
||||
val hudiOptions = getInlineClusteringOpts(isInlineClustering.toString, "2", 100)
|
||||
val hudiOptions = getClusteringOpts(isInlineClustering.toString, isAsyncClustering.toString,
|
||||
isAsyncCompaction.toString, "2", 100)
|
||||
val f1 = initStreamingWriteFuture(inputDF1.schema, sourcePath, destPath, hudiOptions)
|
||||
|
||||
val f2 = Future {
|
||||
|
||||
Reference in New Issue
Block a user