Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[HUDI-5042]fix clustering schedule problem in flink #6976

Merged
merged 1 commit into from
Oct 25, 2022
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -171,7 +171,7 @@ public static HoodieWriteConfig getHoodieClientConfig(
.withMergeAllowDuplicateOnInserts(OptionsResolver.insertClustering(conf))
.withClusteringConfig(
HoodieClusteringConfig.newBuilder()
.withAsyncClustering(conf.getBoolean(FlinkOptions.CLUSTERING_ASYNC_ENABLED))
.withAsyncClustering(conf.getBoolean(FlinkOptions.CLUSTERING_SCHEDULE_ENABLED))
.withClusteringPlanStrategyClass(conf.getString(FlinkOptions.CLUSTERING_PLAN_STRATEGY_CLASS))
.withClusteringPlanPartitionFilterMode(
ClusteringPlanPartitionFilterMode.valueOf(conf.getString(FlinkOptions.CLUSTERING_PLAN_PARTITION_FILTER_MODE_NAME)))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -63,6 +63,7 @@
import java.util.Map;
import java.util.concurrent.TimeUnit;

import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertTrue;

/**
Expand Down Expand Up @@ -220,4 +221,70 @@ public void testHoodieFlinkClusteringService() throws Exception {

TestData.checkWrittenData(tempFile, EXPECTED, 4);
}

@Test
public void testHoodieFlinkClusteringSchedule() throws Exception {
// Create hoodie table and insert into data.
EnvironmentSettings settings = EnvironmentSettings.newInstance().inBatchMode().build();
TableEnvironment tableEnv = TableEnvironmentImpl.create(settings);
Map<String, String> options = new HashMap<>();
options.put(FlinkOptions.PATH.key(), tempFile.getAbsolutePath());

// use append mode
options.put(FlinkOptions.OPERATION.key(), WriteOperationType.INSERT.value());
options.put(FlinkOptions.INSERT_CLUSTER.key(), "false");

String hoodieTableDDL = TestConfigurations.getCreateHoodieTableDDL("t1", options);
tableEnv.executeSql(hoodieTableDDL);
tableEnv.executeSql(TestSQL.INSERT_T1).await();

// wait for the asynchronous commit to finish
TimeUnit.SECONDS.sleep(3);

// Make configuration and setAvroSchema.
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
FlinkClusteringConfig cfg = new FlinkClusteringConfig();
cfg.path = tempFile.getAbsolutePath();
Configuration conf = FlinkClusteringConfig.toFlinkConfig(cfg);

// create metaClient
HoodieTableMetaClient metaClient = StreamerUtil.createMetaClient(conf);

// set the table name
conf.setString(FlinkOptions.TABLE_NAME, metaClient.getTableConfig().getTableName());

// set record key field
conf.setString(FlinkOptions.RECORD_KEY_FIELD, metaClient.getTableConfig().getRecordKeyFieldProp());
// set partition field
conf.setString(FlinkOptions.PARTITION_PATH_FIELD, metaClient.getTableConfig().getPartitionFieldProp());

long ckpTimeout = env.getCheckpointConfig().getCheckpointTimeout();
conf.setLong(FlinkOptions.WRITE_COMMIT_ACK_TIMEOUT, ckpTimeout);
conf.setString(FlinkOptions.PARTITION_PATH_FIELD, "partition");
conf.setInteger(FlinkOptions.CLUSTERING_DELTA_COMMITS, 2);
conf.setBoolean(FlinkOptions.CLUSTERING_ASYNC_ENABLED, false);
conf.setBoolean(FlinkOptions.CLUSTERING_SCHEDULE_ENABLED, true);

// set table schema
CompactionUtil.setAvroSchema(conf, metaClient);

// To compute the clustering instant time.
String clusteringInstantTime = HoodieActiveTimeline.createNewInstantTime();

HoodieFlinkWriteClient writeClient = StreamerUtil.createWriteClient(conf);

boolean scheduled = writeClient.scheduleClusteringAtInstant(clusteringInstantTime, Option.empty());

assertFalse(scheduled, "1 delta commit, the clustering plan should not be scheduled");

tableEnv.executeSql(TestSQL.INSERT_T1).await();
// wait for the asynchronous commit to finish
TimeUnit.SECONDS.sleep(3);

clusteringInstantTime = HoodieActiveTimeline.createNewInstantTime();

scheduled = writeClient.scheduleClusteringAtInstant(clusteringInstantTime, Option.empty());

assertTrue(scheduled, "2 delta commits, the clustering plan should be scheduled");
}
}