[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.
This commit is contained in:
committed by
GitHub
parent
c319ee9cea
commit
52fe1c9fae
@@ -43,6 +43,7 @@ import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
|||||||
import org.apache.hudi.common.util.ClusteringUtils;
|
import org.apache.hudi.common.util.ClusteringUtils;
|
||||||
import org.apache.hudi.common.util.CompactionUtils;
|
import org.apache.hudi.common.util.CompactionUtils;
|
||||||
import org.apache.hudi.common.util.Option;
|
import org.apache.hudi.common.util.Option;
|
||||||
|
import org.apache.hudi.common.util.StringUtils;
|
||||||
import org.apache.hudi.common.util.ValidationUtils;
|
import org.apache.hudi.common.util.ValidationUtils;
|
||||||
import org.apache.hudi.common.util.collection.Pair;
|
import org.apache.hudi.common.util.collection.Pair;
|
||||||
import org.apache.hudi.config.HoodieClusteringConfig;
|
import org.apache.hudi.config.HoodieClusteringConfig;
|
||||||
@@ -403,6 +404,9 @@ public class HoodieDeltaStreamer implements Serializable {
|
|||||||
+ "https://spark.apache.org/docs/latest/job-scheduling.html")
|
+ "https://spark.apache.org/docs/latest/job-scheduling.html")
|
||||||
public Integer clusterSchedulingMinShare = 0;
|
public Integer clusterSchedulingMinShare = 0;
|
||||||
|
|
||||||
|
@Parameter(names = {"--post-write-termination-strategy-class"}, description = "Post writer termination strategy class to gracefully shutdown deltastreamer in continuous mode")
|
||||||
|
public String postWriteTerminationStrategyClass = "";
|
||||||
|
|
||||||
public boolean isAsyncCompactionEnabled() {
|
public boolean isAsyncCompactionEnabled() {
|
||||||
return continuousMode && !forceDisableCompaction
|
return continuousMode && !forceDisableCompaction
|
||||||
&& HoodieTableType.MERGE_ON_READ.equals(HoodieTableType.valueOf(tableType));
|
&& HoodieTableType.MERGE_ON_READ.equals(HoodieTableType.valueOf(tableType));
|
||||||
@@ -603,6 +607,8 @@ public class HoodieDeltaStreamer implements Serializable {
|
|||||||
*/
|
*/
|
||||||
private transient DeltaSync deltaSync;
|
private transient DeltaSync deltaSync;
|
||||||
|
|
||||||
|
private final Option<PostWriteTerminationStrategy> postWriteTerminationStrategy;
|
||||||
|
|
||||||
public DeltaSyncService(Config cfg, JavaSparkContext jssc, FileSystem fs, Configuration conf,
|
public DeltaSyncService(Config cfg, JavaSparkContext jssc, FileSystem fs, Configuration conf,
|
||||||
Option<TypedProperties> properties) throws IOException {
|
Option<TypedProperties> properties) throws IOException {
|
||||||
this.cfg = cfg;
|
this.cfg = cfg;
|
||||||
@@ -610,6 +616,8 @@ public class HoodieDeltaStreamer implements Serializable {
|
|||||||
this.sparkSession = SparkSession.builder().config(jssc.getConf()).getOrCreate();
|
this.sparkSession = SparkSession.builder().config(jssc.getConf()).getOrCreate();
|
||||||
this.asyncCompactService = Option.empty();
|
this.asyncCompactService = Option.empty();
|
||||||
this.asyncClusteringService = Option.empty();
|
this.asyncClusteringService = Option.empty();
|
||||||
|
this.postWriteTerminationStrategy = StringUtils.isNullOrEmpty(cfg.postWriteTerminationStrategyClass) ? Option.empty() :
|
||||||
|
TerminationStrategyUtils.createPostWriteTerminationStrategy(properties.get(), cfg.postWriteTerminationStrategyClass);
|
||||||
|
|
||||||
if (fs.exists(new Path(cfg.targetBasePath))) {
|
if (fs.exists(new Path(cfg.targetBasePath))) {
|
||||||
HoodieTableMetaClient meta =
|
HoodieTableMetaClient meta =
|
||||||
@@ -695,6 +703,14 @@ public class HoodieDeltaStreamer implements Serializable {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
// check if deltastreamer need to be shutdown
|
||||||
|
if (postWriteTerminationStrategy.isPresent()) {
|
||||||
|
if (postWriteTerminationStrategy.get().shouldShutdown(scheduledCompactionInstantAndRDD.isPresent() ? Option.of(scheduledCompactionInstantAndRDD.get().getRight()) :
|
||||||
|
Option.empty())) {
|
||||||
|
error = true;
|
||||||
|
shutdown(false);
|
||||||
|
}
|
||||||
|
}
|
||||||
long toSleepMs = cfg.minSyncIntervalSeconds * 1000 - (System.currentTimeMillis() - start);
|
long toSleepMs = cfg.minSyncIntervalSeconds * 1000 - (System.currentTimeMillis() - start);
|
||||||
if (toSleepMs > 0) {
|
if (toSleepMs > 0) {
|
||||||
LOG.info("Last sync ran less than min sync interval: " + cfg.minSyncIntervalSeconds + " s, sleep: "
|
LOG.info("Last sync ran less than min sync interval: " + cfg.minSyncIntervalSeconds + " s, sleep: "
|
||||||
|
|||||||
@@ -0,0 +1,56 @@
|
|||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.hudi.utilities.deltastreamer;
|
||||||
|
|
||||||
|
import org.apache.hudi.client.WriteStatus;
|
||||||
|
import org.apache.hudi.common.config.TypedProperties;
|
||||||
|
import org.apache.hudi.common.util.Option;
|
||||||
|
|
||||||
|
import org.apache.log4j.LogManager;
|
||||||
|
import org.apache.log4j.Logger;
|
||||||
|
import org.apache.spark.api.java.JavaRDD;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Post writer termination strategy for deltastreamer in continuous mode. This strategy is based on no new data for consecutive number of times.
|
||||||
|
*/
|
||||||
|
public class NoNewDataTerminationStrategy implements PostWriteTerminationStrategy {
|
||||||
|
|
||||||
|
private static final Logger LOG = LogManager.getLogger(NoNewDataTerminationStrategy.class);
|
||||||
|
|
||||||
|
public static final String MAX_ROUNDS_WITHOUT_NEW_DATA_TO_SHUTDOWN = "max.rounds.without.new.data.to.shutdown";
|
||||||
|
public static final int DEFAULT_MAX_ROUNDS_WITHOUT_NEW_DATA_TO_SHUTDOWN = 3;
|
||||||
|
|
||||||
|
private final int numTimesNoNewDataToShutdown;
|
||||||
|
private int numTimesNoNewData = 0;
|
||||||
|
|
||||||
|
public NoNewDataTerminationStrategy(TypedProperties properties) {
|
||||||
|
numTimesNoNewDataToShutdown = properties.getInteger(MAX_ROUNDS_WITHOUT_NEW_DATA_TO_SHUTDOWN, DEFAULT_MAX_ROUNDS_WITHOUT_NEW_DATA_TO_SHUTDOWN);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean shouldShutdown(Option<JavaRDD<WriteStatus>> writeStatuses) {
|
||||||
|
numTimesNoNewData = writeStatuses.isPresent() ? 0 : numTimesNoNewData + 1;
|
||||||
|
if (numTimesNoNewData >= numTimesNoNewDataToShutdown) {
|
||||||
|
LOG.info("Shutting down on continuous mode as there is no new data for " + numTimesNoNewData);
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
}
|
||||||
@@ -0,0 +1,39 @@
|
|||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.hudi.utilities.deltastreamer;
|
||||||
|
|
||||||
|
import org.apache.hudi.client.WriteStatus;
|
||||||
|
import org.apache.hudi.common.util.Option;
|
||||||
|
|
||||||
|
import org.apache.spark.api.java.JavaRDD;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Post write termination strategy for deltastreamer in continuous mode.
|
||||||
|
*/
|
||||||
|
public interface PostWriteTerminationStrategy {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns whether deltastreamer needs to be shutdown.
|
||||||
|
* @param writeStatuses optional pair of scheduled compaction instant and write statuses.
|
||||||
|
* @return true if deltastreamer has to be shutdown. false otherwise.
|
||||||
|
*/
|
||||||
|
boolean shouldShutdown(Option<JavaRDD<WriteStatus>> writeStatuses);
|
||||||
|
|
||||||
|
}
|
||||||
@@ -0,0 +1,45 @@
|
|||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.hudi.utilities.deltastreamer;
|
||||||
|
|
||||||
|
import org.apache.hudi.common.config.TypedProperties;
|
||||||
|
import org.apache.hudi.common.util.Option;
|
||||||
|
import org.apache.hudi.common.util.ReflectionUtils;
|
||||||
|
import org.apache.hudi.common.util.StringUtils;
|
||||||
|
import org.apache.hudi.exception.HoodieException;
|
||||||
|
|
||||||
|
public class TerminationStrategyUtils {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Create a PostWriteTerminationStrategy class via reflection,
|
||||||
|
* <br>
|
||||||
|
* if the class name of PostWriteTerminationStrategy is configured through the {@link HoodieDeltaStreamer.Config#postWriteTerminationStrategyClass}.
|
||||||
|
*/
|
||||||
|
public static Option<PostWriteTerminationStrategy> createPostWriteTerminationStrategy(TypedProperties properties, String postWriteTerminationStrategyClass)
|
||||||
|
throws HoodieException {
|
||||||
|
try {
|
||||||
|
return StringUtils.isNullOrEmpty(postWriteTerminationStrategyClass)
|
||||||
|
? Option.empty() :
|
||||||
|
Option.of((PostWriteTerminationStrategy) ReflectionUtils.loadClass(postWriteTerminationStrategyClass, properties));
|
||||||
|
} catch (Throwable e) {
|
||||||
|
throw new HoodieException("Could not create PostWritTerminationStrategy class " + postWriteTerminationStrategyClass, e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
@@ -60,6 +60,7 @@ import org.apache.hudi.utilities.HoodieClusteringJob;
|
|||||||
import org.apache.hudi.utilities.HoodieIndexer;
|
import org.apache.hudi.utilities.HoodieIndexer;
|
||||||
import org.apache.hudi.utilities.deltastreamer.DeltaSync;
|
import org.apache.hudi.utilities.deltastreamer.DeltaSync;
|
||||||
import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer;
|
import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer;
|
||||||
|
import org.apache.hudi.utilities.deltastreamer.NoNewDataTerminationStrategy;
|
||||||
import org.apache.hudi.utilities.schema.FilebasedSchemaProvider;
|
import org.apache.hudi.utilities.schema.FilebasedSchemaProvider;
|
||||||
import org.apache.hudi.utilities.schema.SchemaProvider;
|
import org.apache.hudi.utilities.schema.SchemaProvider;
|
||||||
import org.apache.hudi.utilities.schema.SparkAvroPostProcessor;
|
import org.apache.hudi.utilities.schema.SparkAvroPostProcessor;
|
||||||
@@ -738,18 +739,30 @@ public class TestHoodieDeltaStreamer extends HoodieDeltaStreamerTestBase {
|
|||||||
testUpsertsContinuousMode(HoodieTableType.COPY_ON_WRITE, "continuous_cow");
|
testUpsertsContinuousMode(HoodieTableType.COPY_ON_WRITE, "continuous_cow");
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testUpsertsCOWContinuousModeShutdownGracefully() throws Exception {
|
||||||
|
testUpsertsContinuousMode(HoodieTableType.COPY_ON_WRITE, "continuous_cow", true);
|
||||||
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testUpsertsMORContinuousMode() throws Exception {
|
public void testUpsertsMORContinuousMode() throws Exception {
|
||||||
testUpsertsContinuousMode(HoodieTableType.MERGE_ON_READ, "continuous_mor");
|
testUpsertsContinuousMode(HoodieTableType.MERGE_ON_READ, "continuous_mor");
|
||||||
}
|
}
|
||||||
|
|
||||||
private void testUpsertsContinuousMode(HoodieTableType tableType, String tempDir) throws Exception {
|
private void testUpsertsContinuousMode(HoodieTableType tableType, String tempDir) throws Exception {
|
||||||
|
testUpsertsContinuousMode(tableType, tempDir, false);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void testUpsertsContinuousMode(HoodieTableType tableType, String tempDir, boolean testShutdownGracefully) throws Exception {
|
||||||
String tableBasePath = dfsBasePath + "/" + tempDir;
|
String tableBasePath = dfsBasePath + "/" + tempDir;
|
||||||
// Keep it higher than batch-size to test continuous mode
|
// Keep it higher than batch-size to test continuous mode
|
||||||
int totalRecords = 3000;
|
int totalRecords = 3000;
|
||||||
// Initial bulk insert
|
// Initial bulk insert
|
||||||
HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(tableBasePath, WriteOperationType.UPSERT);
|
HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(tableBasePath, WriteOperationType.UPSERT);
|
||||||
cfg.continuousMode = true;
|
cfg.continuousMode = true;
|
||||||
|
if (testShutdownGracefully) {
|
||||||
|
cfg.postWriteTerminationStrategyClass = NoNewDataTerminationStrategy.class.getName();
|
||||||
|
}
|
||||||
cfg.tableType = tableType.name();
|
cfg.tableType = tableType.name();
|
||||||
cfg.configs.add(String.format("%s=%d", SourceConfigs.MAX_UNIQUE_RECORDS_PROP, totalRecords));
|
cfg.configs.add(String.format("%s=%d", SourceConfigs.MAX_UNIQUE_RECORDS_PROP, totalRecords));
|
||||||
cfg.configs.add(String.format("%s=false", HoodieCompactionConfig.AUTO_CLEAN.key()));
|
cfg.configs.add(String.format("%s=false", HoodieCompactionConfig.AUTO_CLEAN.key()));
|
||||||
@@ -763,6 +776,9 @@ public class TestHoodieDeltaStreamer extends HoodieDeltaStreamerTestBase {
|
|||||||
}
|
}
|
||||||
TestHelpers.assertRecordCount(totalRecords, tableBasePath, sqlContext);
|
TestHelpers.assertRecordCount(totalRecords, tableBasePath, sqlContext);
|
||||||
TestHelpers.assertDistanceCount(totalRecords, tableBasePath, sqlContext);
|
TestHelpers.assertDistanceCount(totalRecords, tableBasePath, sqlContext);
|
||||||
|
if (testShutdownGracefully) {
|
||||||
|
TestDataSource.returnEmptyBatch = true;
|
||||||
|
}
|
||||||
return true;
|
return true;
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
@@ -781,9 +797,36 @@ public class TestHoodieDeltaStreamer extends HoodieDeltaStreamerTestBase {
|
|||||||
}
|
}
|
||||||
});
|
});
|
||||||
TestHelpers.waitTillCondition(condition, dsFuture, 360);
|
TestHelpers.waitTillCondition(condition, dsFuture, 360);
|
||||||
|
if (cfg != null && !cfg.postWriteTerminationStrategyClass.isEmpty()) {
|
||||||
|
awaitDeltaStreamerShutdown(ds);
|
||||||
|
} else {
|
||||||
ds.shutdownGracefully();
|
ds.shutdownGracefully();
|
||||||
dsFuture.get();
|
dsFuture.get();
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
static void awaitDeltaStreamerShutdown(HoodieDeltaStreamer ds) throws InterruptedException {
|
||||||
|
// await until deltastreamer shuts down on its own
|
||||||
|
boolean shutDownRequested = false;
|
||||||
|
int timeSoFar = 0;
|
||||||
|
while (!shutDownRequested) {
|
||||||
|
shutDownRequested = ds.getDeltaSyncService().isShutdownRequested();
|
||||||
|
Thread.sleep(500);
|
||||||
|
timeSoFar += 500;
|
||||||
|
if (timeSoFar > (2 * 60 * 1000)) {
|
||||||
|
Assertions.fail("Deltastreamer should have shutdown by now");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
boolean shutdownComplete = false;
|
||||||
|
while (!shutdownComplete) {
|
||||||
|
shutdownComplete = ds.getDeltaSyncService().isShutdown();
|
||||||
|
Thread.sleep(500);
|
||||||
|
timeSoFar += 500;
|
||||||
|
if (timeSoFar > (2 * 60 * 1000)) {
|
||||||
|
Assertions.fail("Deltastreamer should have shutdown by now");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
static void deltaStreamerTestRunner(HoodieDeltaStreamer ds, Function<Boolean, Boolean> condition) throws Exception {
|
static void deltaStreamerTestRunner(HoodieDeltaStreamer ds, Function<Boolean, Boolean> condition) throws Exception {
|
||||||
deltaStreamerTestRunner(ds, null, condition);
|
deltaStreamerTestRunner(ds, null, condition);
|
||||||
|
|||||||
@@ -39,11 +39,14 @@ import java.util.stream.Collectors;
|
|||||||
public class TestDataSource extends AbstractBaseTestSource {
|
public class TestDataSource extends AbstractBaseTestSource {
|
||||||
|
|
||||||
private static final Logger LOG = LogManager.getLogger(TestDataSource.class);
|
private static final Logger LOG = LogManager.getLogger(TestDataSource.class);
|
||||||
|
public static boolean returnEmptyBatch = false;
|
||||||
|
private static int counter = 0;
|
||||||
|
|
||||||
public TestDataSource(TypedProperties props, JavaSparkContext sparkContext, SparkSession sparkSession,
|
public TestDataSource(TypedProperties props, JavaSparkContext sparkContext, SparkSession sparkSession,
|
||||||
SchemaProvider schemaProvider) {
|
SchemaProvider schemaProvider) {
|
||||||
super(props, sparkContext, sparkSession, schemaProvider);
|
super(props, sparkContext, sparkSession, schemaProvider);
|
||||||
initDataGen();
|
initDataGen();
|
||||||
|
returnEmptyBatch = false;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@@ -54,9 +57,13 @@ public class TestDataSource extends AbstractBaseTestSource {
|
|||||||
LOG.info("Source Limit is set to " + sourceLimit);
|
LOG.info("Source Limit is set to " + sourceLimit);
|
||||||
|
|
||||||
// No new data.
|
// No new data.
|
||||||
if (sourceLimit <= 0) {
|
if (sourceLimit <= 0 || returnEmptyBatch) {
|
||||||
|
LOG.warn("Return no new data from Test Data source " + counter + ", source limit " + sourceLimit);
|
||||||
return new InputBatch<>(Option.empty(), lastCheckpointStr.orElse(null));
|
return new InputBatch<>(Option.empty(), lastCheckpointStr.orElse(null));
|
||||||
|
} else {
|
||||||
|
LOG.warn("Returning valid data from Test Data source " + counter + ", source limit " + sourceLimit);
|
||||||
}
|
}
|
||||||
|
counter++;
|
||||||
|
|
||||||
List<GenericRecord> records =
|
List<GenericRecord> records =
|
||||||
fetchNextBatch(props, (int) sourceLimit, instantTime, DEFAULT_PARTITION_NUM).collect(Collectors.toList());
|
fetchNextBatch(props, (int) sourceLimit, instantTime, DEFAULT_PARTITION_NUM).collect(Collectors.toList());
|
||||||
|
|||||||
Reference in New Issue
Block a user