Skip to content

Commit 89e7924

Browse files
author
hbg
committed
[HUDI-5042]fix clustering schedule problem in flink when enable schedule clustering and disable async clustering.
1 parent 779a965 commit 89e7924

File tree

6 files changed

+227
-21
lines changed

6 files changed

+227
-21
lines changed

hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/ClusteringPlanActionExecutor.java

Lines changed: 20 additions & 18 deletions
Original file line numberDiff line numberDiff line change
@@ -23,6 +23,7 @@
2323
import org.apache.hudi.common.engine.HoodieEngineContext;
2424
import org.apache.hudi.common.model.HoodieRecordPayload;
2525
import org.apache.hudi.common.model.WriteOperationType;
26+
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
2627
import org.apache.hudi.common.table.timeline.HoodieInstant;
2728
import org.apache.hudi.common.table.timeline.HoodieTimeline;
2829
import org.apache.hudi.common.table.timeline.TimelineMetadataUtils;
@@ -56,35 +57,36 @@ public ClusteringPlanActionExecutor(HoodieEngineContext context,
5657
this.extraMetadata = extraMetadata;
5758
}
5859

59-
protected Option<HoodieClusteringPlan> createClusteringPlan() {
60+
protected boolean isScheduleClustering(HoodieActiveTimeline activeTimeline) {
6061
LOG.info("Checking if clustering needs to be run on " + config.getBasePath());
61-
Option<HoodieInstant> lastClusteringInstant = table.getActiveTimeline()
62+
Option<HoodieInstant> lastClusteringInstant = activeTimeline
6263
.filter(s -> s.getAction().equalsIgnoreCase(HoodieTimeline.REPLACE_COMMIT_ACTION)).lastInstant();
6364

64-
int commitsSinceLastClustering = table.getActiveTimeline().getCommitsTimeline().filterCompletedInstants()
65+
int commitsSinceLastClustering = activeTimeline.getCommitsTimeline().filterCompletedInstants()
6566
.findInstantsAfter(lastClusteringInstant.map(HoodieInstant::getTimestamp).orElse("0"), Integer.MAX_VALUE)
6667
.countInstants();
6768

68-
if (config.inlineClusteringEnabled() && config.getInlineClusterMaxCommits() > commitsSinceLastClustering) {
69-
LOG.info("Not scheduling inline clustering as only " + commitsSinceLastClustering
69+
int clusterMaxCommit = config.inlineClusteringEnabled() ? config.getInlineClusterMaxCommits() : config.getAsyncClusterMaxCommits();
70+
if (clusterMaxCommit > commitsSinceLastClustering) {
71+
LOG.info("Not scheduling clustering as only " + commitsSinceLastClustering
7072
+ " commits was found since last clustering " + lastClusteringInstant + ". Waiting for "
71-
+ config.getInlineClusterMaxCommits());
72-
return Option.empty();
73+
+ clusterMaxCommit);
74+
return false;
75+
} else {
76+
LOG.info("Generating clustering plan for table " + config.getBasePath());
77+
return true;
7378
}
79+
}
7480

75-
if (config.isAsyncClusteringEnabled() && config.getAsyncClusterMaxCommits() > commitsSinceLastClustering) {
76-
LOG.info("Not scheduling async clustering as only " + commitsSinceLastClustering
77-
+ " commits was found since last clustering " + lastClusteringInstant + ". Waiting for "
78-
+ config.getAsyncClusterMaxCommits());
81+
protected Option<HoodieClusteringPlan> createClusteringPlan() {
82+
if (isScheduleClustering(table.getActiveTimeline())) {
83+
ClusteringPlanStrategy strategy = (ClusteringPlanStrategy) ReflectionUtils.loadClass(
84+
ClusteringPlanStrategy.checkAndGetClusteringPlanStrategy(config),
85+
new Class<?>[] {HoodieTable.class, HoodieEngineContext.class, HoodieWriteConfig.class}, table, context, config);
86+
return strategy.generateClusteringPlan();
87+
} else {
7988
return Option.empty();
8089
}
81-
82-
LOG.info("Generating clustering plan for table " + config.getBasePath());
83-
ClusteringPlanStrategy strategy = (ClusteringPlanStrategy) ReflectionUtils.loadClass(
84-
ClusteringPlanStrategy.checkAndGetClusteringPlanStrategy(config),
85-
new Class<?>[] {HoodieTable.class, HoodieEngineContext.class, HoodieWriteConfig.class}, table, context, config);
86-
87-
return strategy.generateClusteringPlan();
8890
}
8991

9092
@Override
Lines changed: 121 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,121 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one
3+
* or more contributor license agreements. See the NOTICE file
4+
* distributed with this work for additional information
5+
* regarding copyright ownership. The ASF licenses this file
6+
* to you under the Apache License, Version 2.0 (the
7+
* "License"); you may not use this file except in compliance
8+
* with the License. You may obtain a copy of the License at
9+
*
10+
* http://www.apache.org/licenses/LICENSE-2.0
11+
*
12+
* Unless required by applicable law or agreed to in writing,
13+
* software distributed under the License is distributed on an
14+
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15+
* KIND, either express or implied. See the License for the
16+
* specific language governing permissions and limitations
17+
* under the License.
18+
*/
19+
20+
package org.apache.hudi.table.action.cluster;
21+
22+
import org.apache.hudi.common.engine.HoodieEngineContext;
23+
import org.apache.hudi.common.engine.HoodieLocalEngineContext;
24+
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
25+
import org.apache.hudi.common.table.timeline.HoodieInstant;
26+
import org.apache.hudi.common.testutils.MockHoodieTimeline;
27+
import org.apache.hudi.common.util.Option;
28+
import org.apache.hudi.config.HoodieClusteringConfig;
29+
import org.apache.hudi.config.HoodieWriteConfig;
30+
31+
import org.apache.hadoop.conf.Configuration;
32+
import org.junit.jupiter.api.Assertions;
33+
import org.junit.jupiter.api.Test;
34+
35+
import java.util.Arrays;
36+
import java.util.List;
37+
38+
import static org.apache.hudi.common.model.ActionType.commit;
39+
import static org.apache.hudi.common.model.ActionType.replacecommit;
40+
41+
class TestClusteringPlanActionExecutor {
42+
@Test
43+
public void testDeltaCommitsClusteringPlanScheduling() {
44+
ClusteringPlanActionExecutor inlineExecutor = getClusteringPlanActionExecutor(true);
45+
ClusteringPlanActionExecutor notInlineExecutor = getClusteringPlanActionExecutor(false);
46+
47+
List<HoodieInstant> instants =
48+
Arrays.asList(
49+
new HoodieInstant(HoodieInstant.State.COMPLETED, commit.name(), "1"),
50+
new HoodieInstant(HoodieInstant.State.COMPLETED, commit.name(), "2")
51+
);
52+
HoodieActiveTimeline activeTimeline = new MockHoodieTimeline(instants);
53+
Assertions.assertFalse(inlineExecutor.isScheduleClustering(activeTimeline),
54+
"Enable inline clustering, 2 delta commits, should not schedule clustering");
55+
Assertions.assertFalse(notInlineExecutor.isScheduleClustering(activeTimeline),
56+
"Disable inline clustering, 2 delta commits, should not schedule clustering");
57+
58+
instants =
59+
Arrays.asList(
60+
new HoodieInstant(HoodieInstant.State.COMPLETED, commit.name(), "1"),
61+
new HoodieInstant(HoodieInstant.State.COMPLETED, commit.name(), "2"),
62+
new HoodieInstant(HoodieInstant.State.COMPLETED, commit.name(), "3")
63+
);
64+
activeTimeline = new MockHoodieTimeline(instants);
65+
Assertions.assertFalse(inlineExecutor.isScheduleClustering(activeTimeline),
66+
"Enable inline clustering, 3 delta commits, should not schedule clustering");
67+
Assertions.assertTrue(notInlineExecutor.isScheduleClustering(activeTimeline),
68+
"Disable inline clustering, 3 delta commits, should schedule clustering");
69+
70+
instants =
71+
Arrays.asList(
72+
new HoodieInstant(HoodieInstant.State.COMPLETED, commit.name(), "1"),
73+
new HoodieInstant(HoodieInstant.State.COMPLETED, commit.name(), "2"),
74+
new HoodieInstant(HoodieInstant.State.COMPLETED, replacecommit.name(), "3"),
75+
new HoodieInstant(HoodieInstant.State.COMPLETED, commit.name(), "4"),
76+
new HoodieInstant(HoodieInstant.State.COMPLETED, commit.name(), "5"),
77+
new HoodieInstant(HoodieInstant.State.COMPLETED, commit.name(), "6")
78+
);
79+
activeTimeline = new MockHoodieTimeline(instants);
80+
Assertions.assertFalse(inlineExecutor.isScheduleClustering(activeTimeline),
81+
"Enable inline clustering, 3 delta commits after replacecommit, should not schedule clustering");
82+
Assertions.assertTrue(notInlineExecutor.isScheduleClustering(activeTimeline),
83+
"Disable inline clustering, 3 delta commits after replacecommit, should schedule clustering");
84+
85+
instants =
86+
Arrays.asList(
87+
new HoodieInstant(HoodieInstant.State.COMPLETED, commit.name(), "1"),
88+
new HoodieInstant(HoodieInstant.State.COMPLETED, commit.name(), "2"),
89+
new HoodieInstant(HoodieInstant.State.COMPLETED, commit.name(), "3"),
90+
new HoodieInstant(HoodieInstant.State.COMPLETED, commit.name(), "4"),
91+
new HoodieInstant(HoodieInstant.State.COMPLETED, commit.name(), "5")
92+
);
93+
activeTimeline = new MockHoodieTimeline(instants);
94+
Assertions.assertTrue(inlineExecutor.isScheduleClustering(activeTimeline),
95+
"Enable inline clustering, 5 delta commits, should schedule clustering");
96+
Assertions.assertTrue(notInlineExecutor.isScheduleClustering(activeTimeline),
97+
"Disable inline clustering, 5 delta commits, should schedule clustering");
98+
}
99+
100+
private ClusteringPlanActionExecutor getClusteringPlanActionExecutor(boolean isInlineClustering) {
101+
HoodieEngineContext engineContext = new HoodieLocalEngineContext(new Configuration());
102+
String instantTime = "4";
103+
HoodieWriteConfig config =
104+
HoodieWriteConfig.newBuilder()
105+
.withPath("/db/tbl")
106+
.withClusteringConfig(
107+
HoodieClusteringConfig.newBuilder()
108+
.withAsyncClustering(false)
109+
.withInlineClustering(isInlineClustering)
110+
.withAsyncClusteringMaxCommits(3)
111+
.withInlineClusteringNumCommits(5)
112+
.build())
113+
.build();
114+
115+
return new ClusteringPlanActionExecutor(engineContext,
116+
config,
117+
null,
118+
instantTime,
119+
Option.empty());
120+
}
121+
}

hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestSparkConsistentBucketClustering.java

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -107,7 +107,9 @@ public void setup(int maxFileSize, Map<String, String> options) throws IOExcepti
107107
.withClusteringConfig(HoodieClusteringConfig.newBuilder()
108108
.withClusteringPlanStrategyClass(SparkConsistentBucketClusteringPlanStrategy.class.getName())
109109
.withClusteringExecutionStrategyClass(SparkConsistentBucketClusteringExecutionStrategy.class.getName())
110-
.withClusteringUpdatesStrategy(SparkConsistentBucketDuplicateUpdateStrategy.class.getName()).build())
110+
.withClusteringUpdatesStrategy(SparkConsistentBucketDuplicateUpdateStrategy.class.getName())
111+
.withInlineClusteringNumCommits(0)
112+
.withAsyncClusteringMaxCommits(0).build())
111113
.build();
112114

113115
writeClient = getHoodieWriteClient(config);

hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/cluster/ITTestHoodieFlinkClustering.java

Lines changed: 66 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -63,6 +63,7 @@
6363
import java.util.Map;
6464
import java.util.concurrent.TimeUnit;
6565

66+
import static org.junit.jupiter.api.Assertions.assertFalse;
6667
import static org.junit.jupiter.api.Assertions.assertTrue;
6768

6869
/**
@@ -220,4 +221,69 @@ public void testHoodieFlinkClusteringService() throws Exception {
220221

221222
TestData.checkWrittenData(tempFile, EXPECTED, 4);
222223
}
224+
225+
@Test
226+
public void testHoodieFlinkClusteringSchedule() throws Exception {
227+
// Create hoodie table and insert into data.
228+
EnvironmentSettings settings = EnvironmentSettings.newInstance().inBatchMode().build();
229+
TableEnvironment tableEnv = TableEnvironmentImpl.create(settings);
230+
Map<String, String> options = new HashMap<>();
231+
options.put(FlinkOptions.PATH.key(), tempFile.getAbsolutePath());
232+
233+
// use append mode
234+
options.put(FlinkOptions.OPERATION.key(), WriteOperationType.INSERT.value());
235+
options.put(FlinkOptions.INSERT_CLUSTER.key(), "false");
236+
237+
String hoodieTableDDL = TestConfigurations.getCreateHoodieTableDDL("t1", options);
238+
tableEnv.executeSql(hoodieTableDDL);
239+
tableEnv.executeSql(TestSQL.INSERT_T1).await();
240+
241+
// wait for the asynchronous commit to finish
242+
TimeUnit.SECONDS.sleep(3);
243+
244+
// Make configuration and setAvroSchema.
245+
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
246+
FlinkClusteringConfig cfg = new FlinkClusteringConfig();
247+
cfg.path = tempFile.getAbsolutePath();
248+
Configuration conf = FlinkClusteringConfig.toFlinkConfig(cfg);
249+
250+
// create metaClient
251+
HoodieTableMetaClient metaClient = StreamerUtil.createMetaClient(conf);
252+
253+
// set the table name
254+
conf.setString(FlinkOptions.TABLE_NAME, metaClient.getTableConfig().getTableName());
255+
256+
// set record key field
257+
conf.setString(FlinkOptions.RECORD_KEY_FIELD, metaClient.getTableConfig().getRecordKeyFieldProp());
258+
// set partition field
259+
conf.setString(FlinkOptions.PARTITION_PATH_FIELD, metaClient.getTableConfig().getPartitionFieldProp());
260+
261+
long ckpTimeout = env.getCheckpointConfig().getCheckpointTimeout();
262+
conf.setLong(FlinkOptions.WRITE_COMMIT_ACK_TIMEOUT, ckpTimeout);
263+
conf.setString(FlinkOptions.PARTITION_PATH_FIELD, "partition");
264+
conf.setInteger(FlinkOptions.CLUSTERING_DELTA_COMMITS, 2);
265+
conf.setBoolean(FlinkOptions.CLUSTERING_ASYNC_ENABLED, false);
266+
267+
// set table schema
268+
CompactionUtil.setAvroSchema(conf, metaClient);
269+
270+
// To compute the clustering instant time.
271+
String clusteringInstantTime = HoodieActiveTimeline.createNewInstantTime();
272+
273+
HoodieFlinkWriteClient writeClient = StreamerUtil.createWriteClient(conf);
274+
275+
boolean scheduled = writeClient.scheduleClusteringAtInstant(clusteringInstantTime, Option.empty());
276+
277+
assertFalse(scheduled, "1 delta commit, the clustering plan should not be scheduled");
278+
279+
tableEnv.executeSql(TestSQL.INSERT_T1).await();
280+
// wait for the asynchronous commit to finish
281+
TimeUnit.SECONDS.sleep(3);
282+
283+
clusteringInstantTime = HoodieActiveTimeline.createNewInstantTime();
284+
285+
scheduled = writeClient.scheduleClusteringAtInstant(clusteringInstantTime, Option.empty());
286+
287+
assertTrue(scheduled, "2 delta commits, the clustering plan should be scheduled");
288+
}
223289
}

hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RunClusteringProcedure.scala

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -75,6 +75,12 @@ class RunClusteringProcedure extends BaseProcedure
7575
val basePath: String = getBasePath(tableName, tablePath)
7676
val metaClient = HoodieTableMetaClient.builder.setConf(jsc.hadoopConfiguration()).setBasePath(basePath).build
7777
var conf: Map[String, String] = Map.empty
78+
79+
conf = conf ++ Map(
80+
HoodieClusteringConfig.ASYNC_CLUSTERING_MAX_COMMITS.key() -> "0",
81+
HoodieClusteringConfig.INLINE_CLUSTERING_MAX_COMMITS.key() -> "0"
82+
)
83+
7884
predicate match {
7985
case Some(p) =>
8086
val prunedPartitions = prunePartition(metaClient, p.asInstanceOf[String])

hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestClusteringProcedure.scala

Lines changed: 11 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -22,6 +22,7 @@ package org.apache.spark.sql.hudi.procedure
2222
import org.apache.hadoop.fs.Path
2323
import org.apache.hudi.common.table.timeline.{HoodieActiveTimeline, HoodieInstant, HoodieTimeline}
2424
import org.apache.hudi.common.util.{Option => HOption}
25+
import org.apache.hudi.config.HoodieClusteringConfig
2526
import org.apache.hudi.{HoodieCLIUtils, HoodieDataSourceHelpers}
2627

2728
import scala.collection.JavaConverters.asScalaIteratorConverter
@@ -52,7 +53,11 @@ class TestClusteringProcedure extends HoodieSparkProcedureTestBase {
5253
spark.sql(s"insert into $tableName values(1, 'a1', 10, 1000)")
5354
spark.sql(s"insert into $tableName values(2, 'a2', 10, 1001)")
5455
spark.sql(s"insert into $tableName values(3, 'a3', 10, 1002)")
55-
val client = HoodieCLIUtils.createHoodieClientFromPath(spark, basePath, Map.empty)
56+
57+
val configMap = Map(
58+
HoodieClusteringConfig.ASYNC_CLUSTERING_MAX_COMMITS.key() -> "0"
59+
)
60+
val client = HoodieCLIUtils.createHoodieClientFromPath(spark, basePath, configMap)
5661
// Generate the first clustering plan
5762
val firstScheduleInstant = HoodieActiveTimeline.createNewInstantTime
5863
client.scheduleClusteringAtInstant(firstScheduleInstant, HOption.empty())
@@ -153,7 +158,11 @@ class TestClusteringProcedure extends HoodieSparkProcedureTestBase {
153158
spark.sql(s"insert into $tableName values(1, 'a1', 10, 1000)")
154159
spark.sql(s"insert into $tableName values(2, 'a2', 10, 1001)")
155160
spark.sql(s"insert into $tableName values(3, 'a3', 10, 1002)")
156-
val client = HoodieCLIUtils.createHoodieClientFromPath(spark, basePath, Map.empty)
161+
162+
val configMap = Map(
163+
HoodieClusteringConfig.ASYNC_CLUSTERING_MAX_COMMITS.key() -> "0"
164+
)
165+
val client = HoodieCLIUtils.createHoodieClientFromPath(spark, basePath, configMap)
157166
// Generate the first clustering plan
158167
val firstScheduleInstant = HoodieActiveTimeline.createNewInstantTime
159168
client.scheduleClusteringAtInstant(firstScheduleInstant, HOption.empty())

0 commit comments

Comments
 (0)