Skip to content

Commit 52fe1c9

Browse files
authored
[HUDI-3675] Adding post write termination strategy to deltastreamer continuous mode (#5073)
- Added a postWriteTerminationStrategy to deltastreamer continuous mode. One can enable by setting the appropriate termination strategy using DeltastreamerConfig.postWriteTerminationStrategyClass. If not, continuous mode is expected to run forever. - Added one concrete impl for termination strategy as NoNewDataTerminationStrategy which shuts down deltastreamer if there is no new data to consume from source for N consecutive rounds.
1 parent c319ee9 commit 52fe1c9

6 files changed

Lines changed: 209 additions & 3 deletions

File tree

hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java

Lines changed: 16 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -43,6 +43,7 @@
4343
import org.apache.hudi.common.util.ClusteringUtils;
4444
import org.apache.hudi.common.util.CompactionUtils;
4545
import org.apache.hudi.common.util.Option;
46+
import org.apache.hudi.common.util.StringUtils;
4647
import org.apache.hudi.common.util.ValidationUtils;
4748
import org.apache.hudi.common.util.collection.Pair;
4849
import org.apache.hudi.config.HoodieClusteringConfig;
@@ -403,6 +404,9 @@ public static class Config implements Serializable {
403404
+ "https://spark.apache.org/docs/latest/job-scheduling.html")
404405
public Integer clusterSchedulingMinShare = 0;
405406

407+
@Parameter(names = {"--post-write-termination-strategy-class"}, description = "Post writer termination strategy class to gracefully shutdown deltastreamer in continuous mode")
408+
public String postWriteTerminationStrategyClass = "";
409+
406410
public boolean isAsyncCompactionEnabled() {
407411
return continuousMode && !forceDisableCompaction
408412
&& HoodieTableType.MERGE_ON_READ.equals(HoodieTableType.valueOf(tableType));
@@ -603,13 +607,17 @@ public static class DeltaSyncService extends HoodieAsyncService {
603607
*/
604608
private transient DeltaSync deltaSync;
605609

610+
private final Option<PostWriteTerminationStrategy> postWriteTerminationStrategy;
611+
606612
public DeltaSyncService(Config cfg, JavaSparkContext jssc, FileSystem fs, Configuration conf,
607613
Option<TypedProperties> properties) throws IOException {
608614
this.cfg = cfg;
609615
this.jssc = jssc;
610616
this.sparkSession = SparkSession.builder().config(jssc.getConf()).getOrCreate();
611617
this.asyncCompactService = Option.empty();
612618
this.asyncClusteringService = Option.empty();
619+
this.postWriteTerminationStrategy = StringUtils.isNullOrEmpty(cfg.postWriteTerminationStrategyClass) ? Option.empty() :
620+
TerminationStrategyUtils.createPostWriteTerminationStrategy(properties.get(), cfg.postWriteTerminationStrategyClass);
613621

614622
if (fs.exists(new Path(cfg.targetBasePath))) {
615623
HoodieTableMetaClient meta =
@@ -695,6 +703,14 @@ protected Pair<CompletableFuture, ExecutorService> startService() {
695703
}
696704
}
697705
}
706+
// check if deltastreamer need to be shutdown
707+
if (postWriteTerminationStrategy.isPresent()) {
708+
if (postWriteTerminationStrategy.get().shouldShutdown(scheduledCompactionInstantAndRDD.isPresent() ? Option.of(scheduledCompactionInstantAndRDD.get().getRight()) :
709+
Option.empty())) {
710+
error = true;
711+
shutdown(false);
712+
}
713+
}
698714
long toSleepMs = cfg.minSyncIntervalSeconds * 1000 - (System.currentTimeMillis() - start);
699715
if (toSleepMs > 0) {
700716
LOG.info("Last sync ran less than min sync interval: " + cfg.minSyncIntervalSeconds + " s, sleep: "
Lines changed: 56 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,56 @@
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.utilities.deltastreamer;
21+
22+
import org.apache.hudi.client.WriteStatus;
23+
import org.apache.hudi.common.config.TypedProperties;
24+
import org.apache.hudi.common.util.Option;
25+
26+
import org.apache.log4j.LogManager;
27+
import org.apache.log4j.Logger;
28+
import org.apache.spark.api.java.JavaRDD;
29+
30+
/**
31+
* Post writer termination strategy for deltastreamer in continuous mode. This strategy is based on no new data for consecutive number of times.
32+
*/
33+
public class NoNewDataTerminationStrategy implements PostWriteTerminationStrategy {
34+
35+
private static final Logger LOG = LogManager.getLogger(NoNewDataTerminationStrategy.class);
36+
37+
public static final String MAX_ROUNDS_WITHOUT_NEW_DATA_TO_SHUTDOWN = "max.rounds.without.new.data.to.shutdown";
38+
public static final int DEFAULT_MAX_ROUNDS_WITHOUT_NEW_DATA_TO_SHUTDOWN = 3;
39+
40+
private final int numTimesNoNewDataToShutdown;
41+
private int numTimesNoNewData = 0;
42+
43+
public NoNewDataTerminationStrategy(TypedProperties properties) {
44+
numTimesNoNewDataToShutdown = properties.getInteger(MAX_ROUNDS_WITHOUT_NEW_DATA_TO_SHUTDOWN, DEFAULT_MAX_ROUNDS_WITHOUT_NEW_DATA_TO_SHUTDOWN);
45+
}
46+
47+
@Override
48+
public boolean shouldShutdown(Option<JavaRDD<WriteStatus>> writeStatuses) {
49+
numTimesNoNewData = writeStatuses.isPresent() ? 0 : numTimesNoNewData + 1;
50+
if (numTimesNoNewData >= numTimesNoNewDataToShutdown) {
51+
LOG.info("Shutting down on continuous mode as there is no new data for " + numTimesNoNewData);
52+
return true;
53+
}
54+
return false;
55+
}
56+
}
Lines changed: 39 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,39 @@
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.utilities.deltastreamer;
21+
22+
import org.apache.hudi.client.WriteStatus;
23+
import org.apache.hudi.common.util.Option;
24+
25+
import org.apache.spark.api.java.JavaRDD;
26+
27+
/**
28+
* Post write termination strategy for deltastreamer in continuous mode.
29+
*/
30+
public interface PostWriteTerminationStrategy {
31+
32+
/**
33+
* Returns whether deltastreamer needs to be shutdown.
34+
* @param writeStatuses optional pair of scheduled compaction instant and write statuses.
35+
* @return true if deltastreamer has to be shutdown. false otherwise.
36+
*/
37+
boolean shouldShutdown(Option<JavaRDD<WriteStatus>> writeStatuses);
38+
39+
}
Lines changed: 45 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,45 @@
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.utilities.deltastreamer;
21+
22+
import org.apache.hudi.common.config.TypedProperties;
23+
import org.apache.hudi.common.util.Option;
24+
import org.apache.hudi.common.util.ReflectionUtils;
25+
import org.apache.hudi.common.util.StringUtils;
26+
import org.apache.hudi.exception.HoodieException;
27+
28+
public class TerminationStrategyUtils {
29+
30+
/**
31+
* Create a PostWriteTerminationStrategy class via reflection,
32+
* <br>
33+
* if the class name of PostWriteTerminationStrategy is configured through the {@link HoodieDeltaStreamer.Config#postWriteTerminationStrategyClass}.
34+
*/
35+
public static Option<PostWriteTerminationStrategy> createPostWriteTerminationStrategy(TypedProperties properties, String postWriteTerminationStrategyClass)
36+
throws HoodieException {
37+
try {
38+
return StringUtils.isNullOrEmpty(postWriteTerminationStrategyClass)
39+
? Option.empty() :
40+
Option.of((PostWriteTerminationStrategy) ReflectionUtils.loadClass(postWriteTerminationStrategyClass, properties));
41+
} catch (Throwable e) {
42+
throw new HoodieException("Could not create PostWritTerminationStrategy class " + postWriteTerminationStrategyClass, e);
43+
}
44+
}
45+
}

hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java

Lines changed: 45 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -60,6 +60,7 @@
6060
import org.apache.hudi.utilities.HoodieIndexer;
6161
import org.apache.hudi.utilities.deltastreamer.DeltaSync;
6262
import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer;
63+
import org.apache.hudi.utilities.deltastreamer.NoNewDataTerminationStrategy;
6364
import org.apache.hudi.utilities.schema.FilebasedSchemaProvider;
6465
import org.apache.hudi.utilities.schema.SchemaProvider;
6566
import org.apache.hudi.utilities.schema.SparkAvroPostProcessor;
@@ -738,18 +739,30 @@ public void testUpsertsCOWContinuousMode() throws Exception {
738739
testUpsertsContinuousMode(HoodieTableType.COPY_ON_WRITE, "continuous_cow");
739740
}
740741

742+
@Test
743+
public void testUpsertsCOWContinuousModeShutdownGracefully() throws Exception {
744+
testUpsertsContinuousMode(HoodieTableType.COPY_ON_WRITE, "continuous_cow", true);
745+
}
746+
741747
@Test
742748
public void testUpsertsMORContinuousMode() throws Exception {
743749
testUpsertsContinuousMode(HoodieTableType.MERGE_ON_READ, "continuous_mor");
744750
}
745751

746752
private void testUpsertsContinuousMode(HoodieTableType tableType, String tempDir) throws Exception {
753+
testUpsertsContinuousMode(tableType, tempDir, false);
754+
}
755+
756+
private void testUpsertsContinuousMode(HoodieTableType tableType, String tempDir, boolean testShutdownGracefully) throws Exception {
747757
String tableBasePath = dfsBasePath + "/" + tempDir;
748758
// Keep it higher than batch-size to test continuous mode
749759
int totalRecords = 3000;
750760
// Initial bulk insert
751761
HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(tableBasePath, WriteOperationType.UPSERT);
752762
cfg.continuousMode = true;
763+
if (testShutdownGracefully) {
764+
cfg.postWriteTerminationStrategyClass = NoNewDataTerminationStrategy.class.getName();
765+
}
753766
cfg.tableType = tableType.name();
754767
cfg.configs.add(String.format("%s=%d", SourceConfigs.MAX_UNIQUE_RECORDS_PROP, totalRecords));
755768
cfg.configs.add(String.format("%s=false", HoodieCompactionConfig.AUTO_CLEAN.key()));
@@ -763,6 +776,9 @@ private void testUpsertsContinuousMode(HoodieTableType tableType, String tempDir
763776
}
764777
TestHelpers.assertRecordCount(totalRecords, tableBasePath, sqlContext);
765778
TestHelpers.assertDistanceCount(totalRecords, tableBasePath, sqlContext);
779+
if (testShutdownGracefully) {
780+
TestDataSource.returnEmptyBatch = true;
781+
}
766782
return true;
767783
});
768784
}
@@ -781,8 +797,35 @@ static void deltaStreamerTestRunner(HoodieDeltaStreamer ds, HoodieDeltaStreamer.
781797
}
782798
});
783799
TestHelpers.waitTillCondition(condition, dsFuture, 360);
784-
ds.shutdownGracefully();
785-
dsFuture.get();
800+
if (cfg != null && !cfg.postWriteTerminationStrategyClass.isEmpty()) {
801+
awaitDeltaStreamerShutdown(ds);
802+
} else {
803+
ds.shutdownGracefully();
804+
dsFuture.get();
805+
}
806+
}
807+
808+
static void awaitDeltaStreamerShutdown(HoodieDeltaStreamer ds) throws InterruptedException {
809+
// await until deltastreamer shuts down on its own
810+
boolean shutDownRequested = false;
811+
int timeSoFar = 0;
812+
while (!shutDownRequested) {
813+
shutDownRequested = ds.getDeltaSyncService().isShutdownRequested();
814+
Thread.sleep(500);
815+
timeSoFar += 500;
816+
if (timeSoFar > (2 * 60 * 1000)) {
817+
Assertions.fail("Deltastreamer should have shutdown by now");
818+
}
819+
}
820+
boolean shutdownComplete = false;
821+
while (!shutdownComplete) {
822+
shutdownComplete = ds.getDeltaSyncService().isShutdown();
823+
Thread.sleep(500);
824+
timeSoFar += 500;
825+
if (timeSoFar > (2 * 60 * 1000)) {
826+
Assertions.fail("Deltastreamer should have shutdown by now");
827+
}
828+
}
786829
}
787830

788831
static void deltaStreamerTestRunner(HoodieDeltaStreamer ds, Function<Boolean, Boolean> condition) throws Exception {

hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestDataSource.java

Lines changed: 8 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -39,11 +39,14 @@
3939
public class TestDataSource extends AbstractBaseTestSource {
4040

4141
private static final Logger LOG = LogManager.getLogger(TestDataSource.class);
42+
public static boolean returnEmptyBatch = false;
43+
private static int counter = 0;
4244

4345
public TestDataSource(TypedProperties props, JavaSparkContext sparkContext, SparkSession sparkSession,
4446
SchemaProvider schemaProvider) {
4547
super(props, sparkContext, sparkSession, schemaProvider);
4648
initDataGen();
49+
returnEmptyBatch = false;
4750
}
4851

4952
@Override
@@ -54,9 +57,13 @@ protected InputBatch<JavaRDD<GenericRecord>> fetchNewData(Option<String> lastChe
5457
LOG.info("Source Limit is set to " + sourceLimit);
5558

5659
// No new data.
57-
if (sourceLimit <= 0) {
60+
if (sourceLimit <= 0 || returnEmptyBatch) {
61+
LOG.warn("Return no new data from Test Data source " + counter + ", source limit " + sourceLimit);
5862
return new InputBatch<>(Option.empty(), lastCheckpointStr.orElse(null));
63+
} else {
64+
LOG.warn("Returning valid data from Test Data source " + counter + ", source limit " + sourceLimit);
5965
}
66+
counter++;
6067

6168
List<GenericRecord> records =
6269
fetchNextBatch(props, (int) sourceLimit, instantTime, DEFAULT_PARTITION_NUM).collect(Collectors.toList());

0 commit comments

Comments
 (0)