[HUDI-2036] Move the compaction plan scheduling out of flink writer coordinator (#3101)
Since HUDI-1955 was fixed, we can move the scheduling out if the coordinator to make the coordinator more lightweight.
This commit is contained in:
@@ -36,7 +36,6 @@ import org.apache.hudi.sink.compact.CompactionPlanEvent;
|
||||
import org.apache.hudi.sink.compact.CompactionPlanOperator;
|
||||
import org.apache.hudi.sink.compact.CompactionPlanSourceFunction;
|
||||
import org.apache.hudi.sink.compact.FlinkCompactionConfig;
|
||||
import org.apache.hudi.sink.compact.NonKeyedCompactFunction;
|
||||
import org.apache.hudi.sink.partitioner.BucketAssignFunction;
|
||||
import org.apache.hudi.sink.partitioner.BucketAssignOperator;
|
||||
import org.apache.hudi.sink.transform.RowDataToHoodieFunction;
|
||||
@@ -62,11 +61,9 @@ import org.apache.flink.streaming.api.CheckpointingMode;
|
||||
import org.apache.flink.streaming.api.datastream.DataStream;
|
||||
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
|
||||
import org.apache.flink.streaming.api.functions.source.FileProcessingMode;
|
||||
import org.apache.flink.streaming.api.operators.KeyedProcessOperator;
|
||||
import org.apache.flink.streaming.api.operators.ProcessOperator;
|
||||
import org.apache.flink.table.api.EnvironmentSettings;
|
||||
import org.apache.flink.table.api.TableEnvironment;
|
||||
import org.apache.flink.table.api.TableResult;
|
||||
import org.apache.flink.table.api.config.ExecutionConfigOptions;
|
||||
import org.apache.flink.table.api.internal.TableEnvironmentImpl;
|
||||
import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
|
||||
@@ -143,8 +140,8 @@ public class StreamWriteITCase extends TestLogger {
|
||||
|
||||
if (conf.getBoolean(FlinkOptions.INDEX_BOOTSTRAP_ENABLED)) {
|
||||
hoodieDataStream = hoodieDataStream.transform("index_bootstrap",
|
||||
TypeInformation.of(HoodieRecord.class),
|
||||
new ProcessOperator<>(new BootstrapFunction<>(conf)));
|
||||
TypeInformation.of(HoodieRecord.class),
|
||||
new ProcessOperator<>(new BootstrapFunction<>(conf)));
|
||||
}
|
||||
|
||||
DataStream<Object> pipeline = hoodieDataStream
|
||||
@@ -174,24 +171,26 @@ public class StreamWriteITCase extends TestLogger {
|
||||
EnvironmentSettings settings = EnvironmentSettings.newInstance().inBatchMode().build();
|
||||
TableEnvironment tableEnv = TableEnvironmentImpl.create(settings);
|
||||
tableEnv.getConfig().getConfiguration()
|
||||
.setInteger(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 1);
|
||||
.setInteger(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 1);
|
||||
Map<String, String> options = new HashMap<>();
|
||||
options.put(FlinkOptions.COMPACTION_ASYNC_ENABLED.key(), "false");
|
||||
options.put(FlinkOptions.PATH.key(), tempFile.getAbsolutePath());
|
||||
options.put(FlinkOptions.TABLE_TYPE.key(), "MERGE_ON_READ");
|
||||
String hoodieTableDDL = TestConfigurations.getCreateHoodieTableDDL("t1", options);
|
||||
tableEnv.executeSql(hoodieTableDDL);
|
||||
String insertInto = "insert into t1 values\n"
|
||||
+ "('id1','Danny',23,TIMESTAMP '1970-01-01 00:00:01','par1'),\n"
|
||||
+ "('id2','Stephen',33,TIMESTAMP '1970-01-01 00:00:02','par1'),\n"
|
||||
+ "('id3','Julian',53,TIMESTAMP '1970-01-01 00:00:03','par2'),\n"
|
||||
+ "('id4','Fabian',31,TIMESTAMP '1970-01-01 00:00:04','par2'),\n"
|
||||
+ "('id5','Sophia',18,TIMESTAMP '1970-01-01 00:00:05','par3'),\n"
|
||||
+ "('id6','Emma',20,TIMESTAMP '1970-01-01 00:00:06','par3'),\n"
|
||||
+ "('id7','Bob',44,TIMESTAMP '1970-01-01 00:00:07','par4'),\n"
|
||||
+ "('id8','Han',56,TIMESTAMP '1970-01-01 00:00:08','par4')";
|
||||
TableResult tableResult = tableEnv.executeSql(insertInto);
|
||||
TimeUnit.SECONDS.sleep(5);
|
||||
tableResult.await();
|
||||
+ "('id1','Danny',23,TIMESTAMP '1970-01-01 00:00:01','par1'),\n"
|
||||
+ "('id2','Stephen',33,TIMESTAMP '1970-01-01 00:00:02','par1'),\n"
|
||||
+ "('id3','Julian',53,TIMESTAMP '1970-01-01 00:00:03','par2'),\n"
|
||||
+ "('id4','Fabian',31,TIMESTAMP '1970-01-01 00:00:04','par2'),\n"
|
||||
+ "('id5','Sophia',18,TIMESTAMP '1970-01-01 00:00:05','par3'),\n"
|
||||
+ "('id6','Emma',20,TIMESTAMP '1970-01-01 00:00:06','par3'),\n"
|
||||
+ "('id7','Bob',44,TIMESTAMP '1970-01-01 00:00:07','par4'),\n"
|
||||
+ "('id8','Han',56,TIMESTAMP '1970-01-01 00:00:08','par4')";
|
||||
tableEnv.executeSql(insertInto).await();
|
||||
|
||||
// wait for the asynchronous commit to finish
|
||||
TimeUnit.SECONDS.sleep(3);
|
||||
|
||||
// Make configuration and setAvroSchema.
|
||||
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
|
||||
@@ -199,7 +198,6 @@ public class StreamWriteITCase extends TestLogger {
|
||||
cfg.path = tempFile.getAbsolutePath();
|
||||
Configuration conf = FlinkCompactionConfig.toFlinkConfig(cfg);
|
||||
conf.setString(FlinkOptions.TABLE_TYPE.key(), "MERGE_ON_READ");
|
||||
conf.setString(FlinkOptions.PARTITION_PATH_FIELD.key(), "partition");
|
||||
|
||||
// create metaClient
|
||||
HoodieTableMetaClient metaClient = CompactionUtil.createMetaClient(conf);
|
||||
@@ -212,34 +210,30 @@ public class StreamWriteITCase extends TestLogger {
|
||||
|
||||
// judge whether have operation
|
||||
// To compute the compaction instant time and do compaction.
|
||||
String instantTime = CompactionUtil.getCompactionInstantTime(metaClient);
|
||||
String compactionInstantTime = CompactionUtil.getCompactionInstantTime(metaClient);
|
||||
HoodieFlinkWriteClient writeClient = StreamerUtil.createWriteClient(conf, null);
|
||||
writeClient.scheduleCompactionAtInstant(instantTime, Option.empty());
|
||||
writeClient.scheduleCompactionAtInstant(compactionInstantTime, Option.empty());
|
||||
|
||||
HoodieFlinkTable<?> table = writeClient.getHoodieTable();
|
||||
// the last instant takes the highest priority.
|
||||
Option<HoodieInstant> compactionInstant = table.getActiveTimeline().filterPendingCompactionTimeline().lastInstant();
|
||||
String compactionInstantTime = compactionInstant.get().getTimestamp();
|
||||
|
||||
// generate compaction plan
|
||||
// should support configurable commit metadata
|
||||
HoodieCompactionPlan compactionPlan = CompactionUtils.getCompactionPlan(
|
||||
table.getMetaClient(), compactionInstantTime);
|
||||
table.getMetaClient(), compactionInstantTime);
|
||||
|
||||
HoodieInstant instant = HoodieTimeline.getCompactionRequestedInstant(compactionInstantTime);
|
||||
|
||||
env.addSource(new CompactionPlanSourceFunction(table, instant, compactionPlan, compactionInstantTime))
|
||||
.name("compaction_source")
|
||||
.uid("uid_compaction_source")
|
||||
.rebalance()
|
||||
.transform("compact_task",
|
||||
TypeInformation.of(CompactionCommitEvent.class),
|
||||
new ProcessOperator<>(new NonKeyedCompactFunction(conf)))
|
||||
.setParallelism(compactionPlan.getOperations().size())
|
||||
.addSink(new CompactionCommitSink(conf))
|
||||
.name("clean_commits")
|
||||
.uid("uid_clean_commits")
|
||||
.setParallelism(1);
|
||||
.name("compaction_source")
|
||||
.uid("uid_compaction_source")
|
||||
.rebalance()
|
||||
.transform("compact_task",
|
||||
TypeInformation.of(CompactionCommitEvent.class),
|
||||
new ProcessOperator<>(new CompactFunction(conf)))
|
||||
.setParallelism(compactionPlan.getOperations().size())
|
||||
.addSink(new CompactionCommitSink(conf))
|
||||
.name("clean_commits")
|
||||
.uid("uid_clean_commits")
|
||||
.setParallelism(1);
|
||||
|
||||
env.execute("flink_hudi_compaction");
|
||||
TestData.checkWrittenFullData(tempFile, EXPECTED);
|
||||
@@ -284,12 +278,12 @@ public class StreamWriteITCase extends TestLogger {
|
||||
.readFile(format, sourcePath, FileProcessingMode.PROCESS_CONTINUOUSLY, 1000, typeInfo)
|
||||
.map(record -> deserializationSchema.deserialize(record.getBytes(StandardCharsets.UTF_8)))
|
||||
.setParallelism(4)
|
||||
.map(new RowDataToHoodieFunction<>(rowType, conf), TypeInformation.of(HoodieRecord.class));
|
||||
.map(new RowDataToHoodieFunction<>(rowType, conf), TypeInformation.of(HoodieRecord.class));
|
||||
|
||||
if (conf.getBoolean(FlinkOptions.INDEX_BOOTSTRAP_ENABLED)) {
|
||||
hoodieDataStream = hoodieDataStream.transform("index_bootstrap",
|
||||
TypeInformation.of(HoodieRecord.class),
|
||||
new ProcessOperator<>(new BootstrapFunction<>(conf)));
|
||||
TypeInformation.of(HoodieRecord.class),
|
||||
new ProcessOperator<>(new BootstrapFunction<>(conf)));
|
||||
}
|
||||
|
||||
DataStream<Object> pipeline = hoodieDataStream
|
||||
@@ -314,10 +308,10 @@ public class StreamWriteITCase extends TestLogger {
|
||||
new CompactionPlanOperator(conf))
|
||||
.uid("uid_compact_plan_generate")
|
||||
.setParallelism(1) // plan generate must be singleton
|
||||
.keyBy(event -> event.getOperation().hashCode())
|
||||
.rebalance()
|
||||
.transform("compact_task",
|
||||
TypeInformation.of(CompactionCommitEvent.class),
|
||||
new KeyedProcessOperator<>(new CompactFunction(conf)))
|
||||
new ProcessOperator<>(new CompactFunction(conf)))
|
||||
.addSink(new CompactionCommitSink(conf))
|
||||
.name("compact_commit")
|
||||
.setParallelism(1);
|
||||
|
||||
@@ -28,11 +28,13 @@ import org.apache.hudi.common.table.TableSchemaResolver;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
import org.apache.hudi.table.HoodieFlinkTable;
|
||||
import org.apache.hudi.util.StreamerUtil;
|
||||
import org.apache.hudi.utils.TestData;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Disabled;
|
||||
@@ -62,6 +64,11 @@ public class TestWriteMergeOnRead extends TestWriteCopyOnWrite {
|
||||
new FlinkTaskContextSupplier(null));
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void setUp(Configuration conf) {
|
||||
conf.setBoolean(FlinkOptions.COMPACTION_ASYNC_ENABLED, false);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void checkWrittenData(File baseFile, Map<String, String> expected, int partitions) throws Exception {
|
||||
HoodieTableMetaClient metaClient = HoodieFlinkTable.create(writeConfig, context).getMetaClient();
|
||||
|
||||
@@ -31,6 +31,7 @@ import org.apache.hudi.sink.event.BatchWriteSuccessEvent;
|
||||
import org.apache.hudi.sink.partitioner.BucketAssignFunction;
|
||||
import org.apache.hudi.sink.partitioner.BucketAssignOperator;
|
||||
import org.apache.hudi.sink.transform.RowDataToHoodieFunction;
|
||||
import org.apache.hudi.util.StreamerUtil;
|
||||
import org.apache.hudi.utils.TestConfigurations;
|
||||
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
@@ -53,6 +54,7 @@ import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
/**
|
||||
* A wrapper class to manipulate the {@link StreamWriteFunction} instance for testing.
|
||||
@@ -82,6 +84,8 @@ public class StreamWriteFunctionWrapper<I> {
|
||||
|
||||
private CompactFunctionWrapper compactFunctionWrapper;
|
||||
|
||||
private final boolean asyncCompaction;
|
||||
|
||||
public StreamWriteFunctionWrapper(String tablePath) throws Exception {
|
||||
this(tablePath, TestConfigurations.getDefaultConf(tablePath));
|
||||
}
|
||||
@@ -103,6 +107,7 @@ public class StreamWriteFunctionWrapper<I> {
|
||||
this.bucketAssignOperatorContext = new MockBucketAssignOperatorContext();
|
||||
this.functionInitializationContext = new MockFunctionInitializationContext();
|
||||
this.compactFunctionWrapper = new CompactFunctionWrapper(this.conf);
|
||||
this.asyncCompaction = StreamerUtil.needsAsyncCompaction(conf);
|
||||
this.bucketAssignOperatorContext = new MockBucketAssignOperatorContext();
|
||||
}
|
||||
|
||||
@@ -131,7 +136,7 @@ public class StreamWriteFunctionWrapper<I> {
|
||||
writeFunction.setOperatorEventGateway(gateway);
|
||||
writeFunction.open(conf);
|
||||
|
||||
if (conf.getBoolean(FlinkOptions.COMPACTION_ASYNC_ENABLED)) {
|
||||
if (asyncCompaction) {
|
||||
compactFunctionWrapper.openFunction();
|
||||
}
|
||||
}
|
||||
@@ -208,10 +213,19 @@ public class StreamWriteFunctionWrapper<I> {
|
||||
|
||||
public void checkpointComplete(long checkpointId) {
|
||||
functionInitializationContext.getOperatorStateStore().checkpointSuccess(checkpointId);
|
||||
if (asyncCompaction) {
|
||||
// sleep for a while to give a change for scheduling compaction,
|
||||
// see HoodieActiveTimeline#createNewInstantTime for details.
|
||||
try {
|
||||
TimeUnit.SECONDS.sleep(2);
|
||||
} catch (InterruptedException e) {
|
||||
throw new HoodieException("Waiting for checkpoint success exception", e);
|
||||
}
|
||||
}
|
||||
coordinator.notifyCheckpointComplete(checkpointId);
|
||||
this.bucketAssignerFunction.notifyCheckpointComplete(checkpointId);
|
||||
this.writeFunction.notifyCheckpointComplete(checkpointId);
|
||||
if (conf.getBoolean(FlinkOptions.COMPACTION_ASYNC_ENABLED)) {
|
||||
if (asyncCompaction) {
|
||||
try {
|
||||
compactFunctionWrapper.compact(checkpointId);
|
||||
} catch (Exception e) {
|
||||
|
||||
Reference in New Issue
Block a user