1
0

[HUDI-1486] Remove inline inflight rollback in hoodie writer (#2359)

1. Refactor rollback and move cleaning failed commits logic into cleaner
2. Introduce hoodie heartbeat to ascertain failed commits
3. Fix test cases
This commit is contained in:
n3nash
2021-02-19 20:12:22 -08:00
committed by GitHub
parent c9fcf964b2
commit ffcfb58bac
64 changed files with 1541 additions and 306 deletions

View File

@@ -21,6 +21,7 @@ package org.apache.hudi.client;
import org.apache.hudi.client.embedded.EmbeddedTimelineServerHelper;
import org.apache.hudi.client.embedded.EmbeddedTimelineService;
import org.apache.hudi.common.engine.HoodieEngineContext;
import org.apache.hudi.client.heartbeat.HoodieHeartbeatClient;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion;
@@ -48,6 +49,7 @@ public abstract class AbstractHoodieClient implements Serializable, AutoCloseabl
protected final transient Configuration hadoopConf;
protected final HoodieWriteConfig config;
protected final String basePath;
protected final HoodieHeartbeatClient heartbeatClient;
/**
* Timeline Server has the same lifetime as that of Client. Any operations done on the same timeline service will be
@@ -70,6 +72,8 @@ public abstract class AbstractHoodieClient implements Serializable, AutoCloseabl
this.config = clientConfig;
this.timelineServer = timelineServer;
shouldStopTimelineServer = !timelineServer.isPresent();
this.heartbeatClient = new HoodieHeartbeatClient(this.fs, this.basePath,
clientConfig.getHoodieClientHeartbeatIntervalInMs(), clientConfig.getHoodieClientHeartbeatTolerableMisses());
startEmbeddedServerView();
initWrapperFSMetrics();
}
@@ -136,4 +140,8 @@ public abstract class AbstractHoodieClient implements Serializable, AutoCloseabl
public Option<EmbeddedTimelineService> getTimelineServer() {
return timelineServer;
}
public HoodieHeartbeatClient getHeartbeatClient() {
return heartbeatClient;
}
}

View File

@@ -29,8 +29,10 @@ import org.apache.hudi.callback.HoodieWriteCommitCallback;
import org.apache.hudi.callback.common.HoodieWriteCommitCallbackMessage;
import org.apache.hudi.callback.util.HoodieCommitCallbackFactory;
import org.apache.hudi.client.embedded.EmbeddedTimelineService;
import org.apache.hudi.client.heartbeat.HeartbeatUtils;
import org.apache.hudi.common.engine.HoodieEngineContext;
import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.model.HoodieWriteStat;
@@ -39,6 +41,7 @@ import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.util.CleanerUtils;
import org.apache.hudi.common.util.ClusteringUtils;
import org.apache.hudi.common.util.CommitUtils;
import org.apache.hudi.common.util.Option;
@@ -48,6 +51,7 @@ import org.apache.hudi.config.HoodieClusteringConfig;
import org.apache.hudi.config.HoodieCompactionConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieCommitException;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.exception.HoodieRestoreException;
import org.apache.hudi.exception.HoodieRollbackException;
@@ -96,43 +100,29 @@ public abstract class AbstractHoodieWriteClient<T extends HoodieRecordPayload, I
private transient WriteOperationType operationType;
private transient HoodieWriteCommitCallback commitCallback;
private transient AsyncCleanerService asyncCleanerService;
protected final boolean rollbackPending;
/**
* Create a write client, without cleaning up failed/inflight commits.
*
* @param context HoodieEngineContext
* @param clientConfig instance of HoodieWriteConfig
*/
public AbstractHoodieWriteClient(HoodieEngineContext context, HoodieWriteConfig clientConfig) {
this(context, clientConfig, false);
}
protected transient AsyncCleanerService asyncCleanerService;
/**
* Create a write client, with new hudi index.
*
* @param context HoodieEngineContext
* @param writeConfig instance of HoodieWriteConfig
* @param rollbackPending whether need to cleanup pending commits
*/
public AbstractHoodieWriteClient(HoodieEngineContext context, HoodieWriteConfig writeConfig, boolean rollbackPending) {
this(context, writeConfig, rollbackPending, Option.empty());
@Deprecated
public AbstractHoodieWriteClient(HoodieEngineContext context, HoodieWriteConfig writeConfig) {
this(context, writeConfig, Option.empty());
}
/**
* Create a write client, allows to specify all parameters.
*
* @param context HoodieEngineContext
* @param writeConfig instance of HoodieWriteConfig
* @param rollbackPending whether need to cleanup pending commits
* @param timelineService Timeline Service that runs as part of write client.
*/
public AbstractHoodieWriteClient(HoodieEngineContext context, HoodieWriteConfig writeConfig, boolean rollbackPending,
@Deprecated
public AbstractHoodieWriteClient(HoodieEngineContext context, HoodieWriteConfig writeConfig,
Option<EmbeddedTimelineService> timelineService) {
super(context, writeConfig, timelineService);
this.metrics = new HoodieMetrics(config, config.getTableName());
this.rollbackPending = rollbackPending;
this.index = createIndex(writeConfig);
}
@@ -181,7 +171,7 @@ public abstract class AbstractHoodieWriteClient<T extends HoodieRecordPayload, I
HoodieCommitMetadata metadata = CommitUtils.buildMetadata(stats, partitionToReplaceFileIds, extraMetadata, operationType, config.getSchema(), commitActionType);
// Finalize write
finalizeWrite(table, instantTime, stats);
HeartbeatUtils.abortIfHeartbeatExpired(instantTime, table, heartbeatClient, config);
try {
activeTimeline.saveAsComplete(new HoodieInstant(true, commitActionType, instantTime),
Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
@@ -236,17 +226,16 @@ public abstract class AbstractHoodieWriteClient<T extends HoodieRecordPayload, I
* Main API to run bootstrap to hudi.
*/
public void bootstrap(Option<Map<String, String>> extraMetadata) {
if (rollbackPending) {
rollBackInflightBootstrap();
}
// TODO : MULTIWRITER -> check if failed bootstrap files can be cleaned later
HoodieTable<T, I, K, O> table = getTableAndInitCtx(WriteOperationType.UPSERT, HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS);
rollbackFailedBootstrap();
table.bootstrap(context, extraMetadata);
}
/**
* Main API to rollback pending bootstrap.
* Main API to rollback failed bootstrap.
*/
protected void rollBackInflightBootstrap() {
public void rollbackFailedBootstrap() {
LOG.info("Rolling back pending bootstrap if present");
HoodieTable<T, I, K, O> table = createTable(config, hadoopConf);
HoodieTimeline inflightTimeline = table.getMetaClient().getCommitsTimeline().filterPendingExcludingCompaction();
@@ -258,7 +247,6 @@ public abstract class AbstractHoodieWriteClient<T extends HoodieRecordPayload, I
table.rollbackBootstrap(context, HoodieActiveTimeline.createNewInstantTime());
LOG.info("Finished rolling back pending bootstrap");
}
}
/**
@@ -369,9 +357,8 @@ public abstract class AbstractHoodieWriteClient<T extends HoodieRecordPayload, I
/**
* Common method containing steps to be performed before write (upsert/insert/...
*
* @param instantTime Instant Time
* @return Write Status
* @param instantTime
* @param writeOperationType
*/
protected void preWrite(String instantTime, WriteOperationType writeOperationType) {
setOperationType(writeOperationType);
@@ -424,15 +411,16 @@ public abstract class AbstractHoodieWriteClient<T extends HoodieRecordPayload, I
HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(config, table);
archiveLog.archiveIfRequired(context);
autoCleanOnCommit();
syncTableMetadata();
} catch (IOException ioe) {
throw new HoodieIOException(ioe.getMessage(), ioe);
} finally {
this.heartbeatClient.stop(instantTime);
}
}
protected void runAnyPendingCompactions(HoodieTable<T, I, K, O> table) {
table.getActiveTimeline().getCommitsAndCompactionTimeline().filterPendingCompactionTimeline().getInstants()
table.getActiveTimeline().getWriteTimeline().filterPendingCompactionTimeline().getInstants()
.forEach(instant -> {
LOG.info("Running previously failed inflight compaction at instant " + instant);
compact(instant.getTimestamp(), true);
@@ -533,11 +521,14 @@ public abstract class AbstractHoodieWriteClient<T extends HoodieRecordPayload, I
}
/**
* Rollback the inflight record changes with the given commit time.
* @Deprecated
* Rollback the inflight record changes with the given commit time. This
* will be removed in future in favor of {@link AbstractHoodieWriteClient#restoreToInstant(String)}
*
* @param commitInstantTime Instant time of the commit
* @throws HoodieRollbackException if rollback cannot be performed successfully
*/
@Deprecated
public boolean rollback(final String commitInstantTime) throws HoodieRollbackException {
LOG.info("Begin rollback of instant " + commitInstantTime);
final String rollbackInstantTime = HoodieActiveTimeline.createNewInstantTime();
@@ -598,6 +589,9 @@ public abstract class AbstractHoodieWriteClient<T extends HoodieRecordPayload, I
public HoodieCleanMetadata clean(String cleanInstantTime) throws HoodieIOException {
LOG.info("Cleaner started");
final Timer.Context timerContext = metrics.getCleanCtx();
LOG.info("Cleaned failed attempts if any");
CleanerUtils.rollbackFailedWrites(config.getFailedWritesCleanPolicy(),
HoodieTimeline.CLEAN_ACTION, () -> rollbackFailedWrites());
HoodieCleanMetadata metadata = createTable(config, hadoopConf).clean(context, cleanInstantTime);
if (timerContext != null && metadata != null) {
long durationMs = metrics.getDurationInMs(timerContext.stop());
@@ -617,11 +611,8 @@ public abstract class AbstractHoodieWriteClient<T extends HoodieRecordPayload, I
* Provides a new commit time for a write operation (insert/update/delete).
*/
public String startCommit() {
// NOTE : Need to ensure that rollback is done before a new commit is started
if (rollbackPending) {
// Only rollback pending commit/delta-commits. Do not touch compaction commits
rollbackPendingCommits();
}
CleanerUtils.rollbackFailedWrites(config.getFailedWritesCleanPolicy(),
HoodieTimeline.COMMIT_ACTION, () -> rollbackFailedWrites());
String instantTime = HoodieActiveTimeline.createNewInstantTime();
HoodieTableMetaClient metaClient = createMetaClient(true);
startCommit(instantTime, metaClient.getCommitActionType(), metaClient);
@@ -650,11 +641,8 @@ public abstract class AbstractHoodieWriteClient<T extends HoodieRecordPayload, I
* Completes a new commit time for a write operation (insert/update/delete) with specified action.
*/
private void startCommitWithTime(String instantTime, String actionType, HoodieTableMetaClient metaClient) {
// NOTE : Need to ensure that rollback is done before a new commit is started
if (rollbackPending) {
// Only rollback inflight commit/delta-commits. Do not touch compaction commits
rollbackPendingCommits();
}
CleanerUtils.rollbackFailedWrites(config.getFailedWritesCleanPolicy(),
HoodieTimeline.COMMIT_ACTION, () -> rollbackFailedWrites());
startCommit(instantTime, actionType, metaClient);
}
@@ -666,6 +654,9 @@ public abstract class AbstractHoodieWriteClient<T extends HoodieRecordPayload, I
HoodieTimeline.compareTimestamps(latestPending.getTimestamp(), HoodieTimeline.LESSER_THAN, instantTime),
"Latest pending compaction instant time must be earlier than this instant time. Latest Compaction :"
+ latestPending + ", Ingesting at " + instantTime));
if (config.getFailedWritesCleanPolicy().isLazy()) {
this.heartbeatClient.start(instantTime);
}
metaClient.getActiveTimeline().createNewInstant(new HoodieInstant(HoodieInstant.State.REQUESTED, actionType,
instantTime));
}
@@ -721,7 +712,7 @@ public abstract class AbstractHoodieWriteClient<T extends HoodieRecordPayload, I
/**
* Rollback failed compactions. Inflight rollbacks for compactions revert the .inflight file to the .requested file
*
* TODO : Deprecate this method and make it protected
* @param inflightInstant Inflight Compaction Instant
* @param table Hoodie Table
*/
@@ -749,22 +740,49 @@ public abstract class AbstractHoodieWriteClient<T extends HoodieRecordPayload, I
}
/**
* Cleanup all pending commits.
* Rollback all failed writes.
*/
private void rollbackPendingCommits() {
public Boolean rollbackFailedWrites() {
HoodieTable<T, I, K, O> table = createTable(config, hadoopConf);
HoodieTimeline inflightTimeline = getInflightTimelineExcludeCompactionAndClustering(table);
List<String> commits = inflightTimeline.getReverseOrderedInstants().map(HoodieInstant::getTimestamp)
.collect(Collectors.toList());
for (String commit : commits) {
if (HoodieTimeline.compareTimestamps(commit, HoodieTimeline.LESSER_THAN_OR_EQUALS,
List<String> instantsToRollback = getInstantsToRollback(table);
for (String instant : instantsToRollback) {
if (HoodieTimeline.compareTimestamps(instant, HoodieTimeline.LESSER_THAN_OR_EQUALS,
HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS)) {
rollBackInflightBootstrap();
rollbackFailedBootstrap();
break;
} else {
rollback(commit);
rollback(instant);
}
}
// Delete any heartbeat files for already rolled back commits
try {
HeartbeatUtils.cleanExpiredHeartbeats(this.heartbeatClient.getAllExistingHeartbeatInstants(),
createMetaClient(true), basePath);
} catch (IOException io) {
LOG.error("Unable to delete heartbeat files", io);
}
return true;
}
private List<String> getInstantsToRollback(HoodieTable<T, I, K, O> table) {
if (config.getFailedWritesCleanPolicy().isEager()) {
HoodieTimeline inflightTimeline = table.getMetaClient().getCommitsTimeline().filterPendingExcludingCompaction();
return inflightTimeline.getReverseOrderedInstants().map(HoodieInstant::getTimestamp)
.collect(Collectors.toList());
} else if (config.getFailedWritesCleanPolicy() == HoodieFailedWritesCleaningPolicy.NEVER) {
return Collections.EMPTY_LIST;
} else if (config.getFailedWritesCleanPolicy().isLazy()) {
return table.getMetaClient().getActiveTimeline()
.getCommitsTimeline().filterInflights().getReverseOrderedInstants().filter(instant -> {
try {
return heartbeatClient.isHeartbeatExpired(instant.getTimestamp());
} catch (IOException io) {
throw new HoodieException("Failed to check heartbeat for instant " + instant, io);
}
}).map(HoodieInstant::getTimestamp).collect(Collectors.toList());
} else {
throw new IllegalArgumentException("Invalid Failed Writes Cleaning Policy " + config.getFailedWritesCleanPolicy());
}
}
/**
@@ -912,5 +930,6 @@ public abstract class AbstractHoodieWriteClient<T extends HoodieRecordPayload, I
// Calling this here releases any resources used by your index, so make sure to finish any related operations
// before this point
this.index.close();
this.heartbeatClient.stop();
}
}

View File

@@ -0,0 +1,102 @@
/*
* 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.client.heartbeat;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.util.ValidationUtils;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.table.HoodieTable;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.io.File;
import java.io.IOException;
import java.util.List;
import java.util.Set;
import java.util.stream.Collectors;
/**
* Helper class to delete heartbeat for completed or failed instants with expired heartbeats.
*/
public class HeartbeatUtils {
private static final Logger LOG = LogManager.getLogger(HeartbeatUtils.class);
/**
* Deletes the heartbeat file for the specified instant.
* @param fs
* @param basePath
* @param instantTime
* @return
*/
public static boolean deleteHeartbeatFile(FileSystem fs, String basePath, String instantTime) {
boolean deleted = false;
try {
String heartbeatFolderPath = HoodieTableMetaClient.getHeartbeatFolderPath(basePath);
deleted = fs.delete(new Path(heartbeatFolderPath + File.separator + instantTime), false);
if (!deleted) {
LOG.error("Failed to delete heartbeat for instant " + instantTime);
}
} catch (IOException io) {
LOG.error("Unable to delete heartbeat for instant " + instantTime, io);
}
return deleted;
}
/**
* Deletes the heartbeat files for instants with expired heartbeats without any active instant.
* @param allExistingHeartbeatInstants
* @param metaClient
* @param basePath
*/
public static void cleanExpiredHeartbeats(List<String> allExistingHeartbeatInstants,
HoodieTableMetaClient metaClient, String basePath) {
Set<String> nonExpiredHeartbeatInstants = metaClient.getActiveTimeline()
.filterCompletedInstants().getInstants().map(HoodieInstant::getTimestamp).collect(Collectors.toSet());
allExistingHeartbeatInstants.stream().forEach(instant -> {
if (!nonExpiredHeartbeatInstants.contains(instant)) {
deleteHeartbeatFile(metaClient.getFs(), basePath, instant);
}
});
}
/**
* Check if the heartbeat corresponding to instantTime has expired. If yes, abort by throwing an exception.
* @param instantTime
* @param table
* @param heartbeatClient
* @param config
*/
public static void abortIfHeartbeatExpired(String instantTime, HoodieTable table,
HoodieHeartbeatClient heartbeatClient, HoodieWriteConfig config) {
ValidationUtils.checkArgument(heartbeatClient != null);
try {
if (config.getFailedWritesCleanPolicy().isLazy() && heartbeatClient.isHeartbeatExpired(instantTime)) {
throw new HoodieException("Heartbeat for instant " + instantTime + " has expired, last heartbeat "
+ heartbeatClient.getLastHeartbeatTime(table.getMetaClient().getFs(), config.getBasePath(), instantTime));
}
} catch (IOException io) {
throw new HoodieException("Unable to read heartbeat", io);
}
}
}

View File

@@ -0,0 +1,289 @@
/*
* 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.client.heartbeat;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.util.ValidationUtils;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieHeartbeatException;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import javax.annotation.concurrent.NotThreadSafe;
import java.io.File;
import java.io.IOException;
import java.io.OutputStream;
import java.io.Serializable;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
import java.util.Timer;
import java.util.TimerTask;
/**
* This class creates heartbeat for hudi client. This heartbeat is used to ascertain whether the running job is or not.
* NOTE: Due to CPU contention on the driver/client node, the heartbeats could be delayed, hence it's important to set
* the value high enough to avoid that possibility.
*/
@NotThreadSafe
public class HoodieHeartbeatClient implements AutoCloseable, Serializable {
private static final Logger LOG = LogManager.getLogger(HoodieHeartbeatClient.class);
private final transient FileSystem fs;
private final String basePath;
// path to the heartbeat folder where all writers are updating their heartbeats
private String heartbeatFolderPath;
// heartbeat interval in millis
private final Long heartbeatIntervalInMs;
private Integer numTolerableHeartbeatMisses;
private final Long maxAllowableHeartbeatIntervalInMs;
private Map<String, Heartbeat> instantToHeartbeatMap;
public HoodieHeartbeatClient(FileSystem fs, String basePath, Long heartbeatIntervalInMs,
Integer numTolerableHeartbeatMisses) {
ValidationUtils.checkArgument(heartbeatIntervalInMs >= 1000, "Cannot set heartbeat lower than 1 second");
this.fs = fs;
this.basePath = basePath;
this.heartbeatFolderPath = HoodieTableMetaClient.getHeartbeatFolderPath(basePath);
this.heartbeatIntervalInMs = heartbeatIntervalInMs;
this.numTolerableHeartbeatMisses = numTolerableHeartbeatMisses;
this.maxAllowableHeartbeatIntervalInMs = this.heartbeatIntervalInMs * this.numTolerableHeartbeatMisses;
this.instantToHeartbeatMap = new HashMap<>();
}
class Heartbeat {
private String instantTime;
private Boolean isHeartbeatStarted = false;
private Boolean isHeartbeatStopped = false;
private Long lastHeartbeatTime;
private Integer numHeartbeats = 0;
private Timer timer = new Timer();
public String getInstantTime() {
return instantTime;
}
public void setInstantTime(String instantTime) {
this.instantTime = instantTime;
}
public Boolean isHeartbeatStarted() {
return isHeartbeatStarted;
}
public void setHeartbeatStarted(Boolean heartbeatStarted) {
isHeartbeatStarted = heartbeatStarted;
}
public Boolean isHeartbeatStopped() {
return isHeartbeatStopped;
}
public void setHeartbeatStopped(Boolean heartbeatStopped) {
isHeartbeatStopped = heartbeatStopped;
}
public Long getLastHeartbeatTime() {
return lastHeartbeatTime;
}
public void setLastHeartbeatTime(Long lastHeartbeatTime) {
this.lastHeartbeatTime = lastHeartbeatTime;
}
public Integer getNumHeartbeats() {
return numHeartbeats;
}
public void setNumHeartbeats(Integer numHeartbeats) {
this.numHeartbeats = numHeartbeats;
}
public Timer getTimer() {
return timer;
}
public void setTimer(Timer timer) {
this.timer = timer;
}
@Override
public String toString() {
return "Heartbeat{"
+ "instantTime='" + instantTime + '\''
+ ", isHeartbeatStarted=" + isHeartbeatStarted
+ ", isHeartbeatStopped=" + isHeartbeatStopped
+ ", lastHeartbeatTime=" + lastHeartbeatTime
+ ", numHeartbeats=" + numHeartbeats
+ ", timer=" + timer
+ '}';
}
}
class HeartbeatTask extends TimerTask {
private final String instantTime;
HeartbeatTask(String instantTime) {
this.instantTime = instantTime;
}
@Override
public void run() {
updateHeartbeat(instantTime);
}
}
/**
* Start a new heartbeat for the specified instant. If there is already one running, this will be a NO_OP
* @param instantTime
*/
public void start(String instantTime) {
LOG.info("Received request to start heartbeat for instant time " + instantTime);
Heartbeat heartbeat = instantToHeartbeatMap.get(instantTime);
ValidationUtils.checkArgument(heartbeat == null || !heartbeat.isHeartbeatStopped(), "Cannot restart a stopped heartbeat for " + instantTime);
if (heartbeat != null && heartbeat.isHeartbeatStarted()) {
// heartbeat already started, NO_OP
return;
} else {
Heartbeat newHeartbeat = new Heartbeat();
newHeartbeat.setHeartbeatStarted(true);
instantToHeartbeatMap.put(instantTime, newHeartbeat);
// Ensure heartbeat is generated for the first time with this blocking call.
// Since timer submits the task to a thread, no guarantee when that thread will get CPU
// cycles to generate the first heartbeat.
updateHeartbeat(instantTime);
newHeartbeat.getTimer().scheduleAtFixedRate(new HeartbeatTask(instantTime), this.heartbeatIntervalInMs,
this.heartbeatIntervalInMs);
}
}
/**
* Stops the heartbeat for the specified instant.
* @param instantTime
* @throws HoodieException
*/
public void stop(String instantTime) throws HoodieException {
Heartbeat heartbeat = instantToHeartbeatMap.get(instantTime);
if (heartbeat != null && heartbeat.isHeartbeatStarted() && !heartbeat.isHeartbeatStopped()) {
LOG.info("Stopping heartbeat for instant " + instantTime);
heartbeat.getTimer().cancel();
heartbeat.setHeartbeatStopped(true);
LOG.info("Stopped heartbeat for instant " + instantTime);
HeartbeatUtils.deleteHeartbeatFile(fs, basePath, instantTime);
LOG.info("Deleted heartbeat file for instant " + instantTime);
}
}
/**
* Stops all heartbeats started via this instance of the client.
* @throws HoodieException
*/
public void stop() throws HoodieException {
instantToHeartbeatMap.values().stream().forEach(heartbeat -> stop(heartbeat.getInstantTime()));
}
public static Long getLastHeartbeatTime(FileSystem fs, String basePath, String instantTime) throws IOException {
Path heartbeatFilePath = new Path(HoodieTableMetaClient.getHeartbeatFolderPath(basePath) + File.separator + instantTime);
if (fs.exists(heartbeatFilePath)) {
return fs.getFileStatus(heartbeatFilePath).getModificationTime();
} else {
// NOTE : This can happen when a writer is upgraded to use lazy cleaning and the last write had failed
return 0L;
}
}
public static Boolean heartbeatExists(FileSystem fs, String basePath, String instantTime) throws IOException {
Path heartbeatFilePath = new Path(HoodieTableMetaClient.getHeartbeatFolderPath(basePath) + File.separator + instantTime);
if (fs.exists(heartbeatFilePath)) {
return true;
}
return false;
}
public boolean isHeartbeatExpired(String instantTime) throws IOException {
Long currentTime = System.currentTimeMillis();
Heartbeat lastHeartbeatForWriter = instantToHeartbeatMap.get(instantTime);
if (lastHeartbeatForWriter == null) {
LOG.info("Heartbeat not found in internal map, falling back to reading from DFS");
long lastHeartbeatForWriterTime = getLastHeartbeatTime(this.fs, basePath, instantTime);
lastHeartbeatForWriter = new Heartbeat();
lastHeartbeatForWriter.setLastHeartbeatTime(lastHeartbeatForWriterTime);
lastHeartbeatForWriter.setInstantTime(instantTime);
}
if (currentTime - lastHeartbeatForWriter.getLastHeartbeatTime() > this.maxAllowableHeartbeatIntervalInMs) {
LOG.warn("Heartbeat expired, currentTime = " + currentTime + ", last heartbeat = " + lastHeartbeatForWriter
+ ", heartbeat interval = " + this.heartbeatIntervalInMs);
return true;
}
return false;
}
public List<String> getAllExistingHeartbeatInstants() throws IOException {
Path heartbeatFolder = new Path(heartbeatFolderPath);
if (this.fs.exists(heartbeatFolder)) {
FileStatus[] fileStatus = this.fs.listStatus(new Path(heartbeatFolderPath));
return Arrays.stream(fileStatus).map(fs -> fs.getPath().getName()).collect(Collectors.toList());
}
return Collections.EMPTY_LIST;
}
private void updateHeartbeat(String instantTime) throws HoodieHeartbeatException {
try {
Long newHeartbeatTime = System.currentTimeMillis();
OutputStream outputStream =
this.fs.create(new Path(heartbeatFolderPath + File.separator + instantTime), true);
outputStream.close();
Heartbeat heartbeat = instantToHeartbeatMap.get(instantTime);
if (heartbeat.getLastHeartbeatTime() != null && isHeartbeatExpired(instantTime)) {
LOG.error("Aborting, missed generating heartbeat within allowable interval " + this.maxAllowableHeartbeatIntervalInMs);
// Since TimerTask allows only java.lang.Runnable, cannot throw an exception and bubble to the caller thread, hence
// explicitly interrupting the timer thread.
Thread.currentThread().interrupt();
}
heartbeat.setInstantTime(instantTime);
heartbeat.setLastHeartbeatTime(newHeartbeatTime);
heartbeat.setNumHeartbeats(heartbeat.getNumHeartbeats() + 1);
} catch (IOException io) {
throw new HoodieHeartbeatException("Unable to generate heartbeat ", io);
}
}
public String getHeartbeatFolderPath() {
return heartbeatFolderPath;
}
public Heartbeat getHeartbeat(String instantTime) {
return this.instantToHeartbeatMap.get(instantTime);
}
@Override
public void close() {
this.stop();
this.instantToHeartbeatMap.clear();
}
}

View File

@@ -20,6 +20,7 @@ package org.apache.hudi.config;
import org.apache.hudi.common.config.DefaultHoodieConfig;
import org.apache.hudi.common.model.HoodieCleaningPolicy;
import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy;
import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload;
import org.apache.hudi.common.util.ValidationUtils;
import org.apache.hudi.table.action.compact.CompactionTriggerStrategy;
@@ -108,6 +109,9 @@ public class HoodieCompactionConfig extends DefaultHoodieConfig {
public static final String COMPACTION_REVERSE_LOG_READ_ENABLED_PROP = "hoodie.compaction.reverse.log.read";
public static final String DEFAULT_COMPACTION_REVERSE_LOG_READ_ENABLED = "false";
private static final String DEFAULT_CLEANER_POLICY = HoodieCleaningPolicy.KEEP_LATEST_COMMITS.name();
public static final String FAILED_WRITES_CLEANER_POLICY_PROP = "hoodie.failed.writes.cleaner.policy";
private static final String DEFAULT_FAILED_WRITES_CLEANER_POLICY =
HoodieFailedWritesCleaningPolicy.EAGER.name();
private static final String DEFAULT_AUTO_CLEAN = "true";
private static final String DEFAULT_ASYNC_CLEAN = "false";
private static final String DEFAULT_INLINE_COMPACT = "false";
@@ -276,6 +280,11 @@ public class HoodieCompactionConfig extends DefaultHoodieConfig {
return this;
}
public Builder withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy failedWritesPolicy) {
props.setProperty(FAILED_WRITES_CLEANER_POLICY_PROP, failedWritesPolicy.name());
return this;
}
public HoodieCompactionConfig build() {
HoodieCompactionConfig config = new HoodieCompactionConfig(props);
setDefaultOnCondition(props, !props.containsKey(AUTO_CLEAN_PROP), AUTO_CLEAN_PROP, DEFAULT_AUTO_CLEAN);
@@ -328,6 +337,8 @@ public class HoodieCompactionConfig extends DefaultHoodieConfig {
COMMITS_ARCHIVAL_BATCH_SIZE_PROP, DEFAULT_COMMITS_ARCHIVAL_BATCH_SIZE);
setDefaultOnCondition(props, !props.containsKey(CLEANER_BOOTSTRAP_BASE_FILE_ENABLED),
CLEANER_BOOTSTRAP_BASE_FILE_ENABLED, DEFAULT_CLEANER_BOOTSTRAP_BASE_FILE_ENABLED);
setDefaultOnCondition(props, !props.containsKey(FAILED_WRITES_CLEANER_POLICY_PROP),
FAILED_WRITES_CLEANER_POLICY_PROP, DEFAULT_FAILED_WRITES_CLEANER_POLICY);
HoodieCleaningPolicy.valueOf(props.getProperty(CLEANER_POLICY_PROP));

View File

@@ -24,6 +24,7 @@ import org.apache.hudi.common.config.DefaultHoodieConfig;
import org.apache.hudi.common.config.HoodieMetadataConfig;
import org.apache.hudi.common.engine.EngineType;
import org.apache.hudi.common.fs.ConsistencyGuardConfig;
import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy;
import org.apache.hudi.common.model.HoodieCleaningPolicy;
import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload;
import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion;
@@ -136,6 +137,12 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
private static final String MERGE_ALLOW_DUPLICATE_ON_INSERTS = "hoodie.merge.allow.duplicate.on.inserts";
private static final String DEFAULT_MERGE_ALLOW_DUPLICATE_ON_INSERTS = "false";
public static final String CLIENT_HEARTBEAT_INTERVAL_IN_MS_PROP = "hoodie.client.heartbeat.interval_in_ms";
public static final Integer DEFAULT_CLIENT_HEARTBEAT_INTERVAL_IN_MS = 60 * 1000;
public static final String CLIENT_HEARTBEAT_NUM_TOLERABLE_MISSES_PROP = "hoodie.client.heartbeat.tolerable.misses";
public static final Integer DEFAULT_CLIENT_HEARTBEAT_NUM_TOLERABLE_MISSES = 2;
/**
* HUDI-858 : There are users who had been directly using RDD APIs and have relied on a behavior in 0.4.x to allow
* multiple write operations (upsert/buk-insert/...) to be executed within a single commit.
@@ -471,6 +478,11 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
return props.getProperty(HoodieClusteringConfig.CLUSTERING_UPDATES_STRATEGY_PROP);
}
public HoodieFailedWritesCleaningPolicy getFailedWritesCleanPolicy() {
return HoodieFailedWritesCleaningPolicy
.valueOf(props.getProperty(HoodieCompactionConfig.FAILED_WRITES_CLEANER_POLICY_PROP));
}
/**
* Clustering properties.
*/
@@ -905,6 +917,18 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
return Long.valueOf(props.getProperty(HoodieMemoryConfig.MAX_MEMORY_FOR_MERGE_PROP));
}
public int getMetadataCleanerCommitsRetained() {
return Integer.parseInt(props.getProperty(HoodieMetadataConfig.CLEANER_COMMITS_RETAINED_PROP));
}
public Long getHoodieClientHeartbeatIntervalInMs() {
return Long.valueOf(props.getProperty(CLIENT_HEARTBEAT_INTERVAL_IN_MS_PROP));
}
public Integer getHoodieClientHeartbeatTolerableMisses() {
return Integer.valueOf(props.getProperty(CLIENT_HEARTBEAT_NUM_TOLERABLE_MISSES_PROP));
}
/**
* File listing metadata configs.
*/
@@ -936,10 +960,6 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
return Integer.parseInt(props.getProperty(HoodieMetadataConfig.MIN_COMMITS_TO_KEEP_PROP));
}
public int getMetadataCleanerCommitsRetained() {
return Integer.parseInt(props.getProperty(HoodieMetadataConfig.CLEANER_COMMITS_RETAINED_PROP));
}
public static class Builder {
protected final Properties props = new Properties();
@@ -1202,6 +1222,16 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
return this;
}
public Builder withHeartbeatIntervalInMs(Integer heartbeatIntervalInMs) {
props.setProperty(CLIENT_HEARTBEAT_INTERVAL_IN_MS_PROP, String.valueOf(heartbeatIntervalInMs));
return this;
}
public Builder withHeartbeatTolerableMisses(Integer heartbeatTolerableMisses) {
props.setProperty(CLIENT_HEARTBEAT_NUM_TOLERABLE_MISSES_PROP, String.valueOf(heartbeatTolerableMisses));
return this;
}
public Builder withProperties(Properties properties) {
this.props.putAll(properties);
return this;
@@ -1258,6 +1288,10 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
MERGE_DATA_VALIDATION_CHECK_ENABLED, DEFAULT_MERGE_DATA_VALIDATION_CHECK_ENABLED);
setDefaultOnCondition(props, !props.containsKey(MERGE_ALLOW_DUPLICATE_ON_INSERTS),
MERGE_ALLOW_DUPLICATE_ON_INSERTS, DEFAULT_MERGE_ALLOW_DUPLICATE_ON_INSERTS);
setDefaultOnCondition(props, !props.containsKey(CLIENT_HEARTBEAT_INTERVAL_IN_MS_PROP),
CLIENT_HEARTBEAT_INTERVAL_IN_MS_PROP, String.valueOf(DEFAULT_CLIENT_HEARTBEAT_INTERVAL_IN_MS));
setDefaultOnCondition(props, !props.containsKey(CLIENT_HEARTBEAT_NUM_TOLERABLE_MISSES_PROP),
CLIENT_HEARTBEAT_NUM_TOLERABLE_MISSES_PROP, String.valueOf(DEFAULT_CLIENT_HEARTBEAT_NUM_TOLERABLE_MISSES));
// Make sure the props is propagated
setDefaultOnCondition(props, !isIndexConfigSet, HoodieIndexConfig.newBuilder().withEngineType(engineType).fromProperties(props).build());
@@ -1285,6 +1319,7 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
EXTERNAL_RECORD_AND_SCHEMA_TRANSFORMATION, DEFAULT_EXTERNAL_RECORD_AND_SCHEMA_TRANSFORMATION);
setDefaultOnCondition(props, !props.containsKey(TIMELINE_LAYOUT_VERSION), TIMELINE_LAYOUT_VERSION,
String.valueOf(TimelineLayoutVersion.CURR_VERSION));
}
private void validate() {

View File

@@ -30,6 +30,7 @@ import org.apache.hudi.common.fs.ConsistencyGuardConfig;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieCleaningPolicy;
import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy;
import org.apache.hudi.common.model.HoodieFileFormat;
import org.apache.hudi.common.model.HoodiePartitionMetadata;
import org.apache.hudi.common.model.HoodieRecord;
@@ -155,6 +156,7 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta
.withAutoClean(false)
.withCleanerParallelism(parallelism)
.withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS)
.withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.EAGER)
.retainCommits(writeConfig.getMetadataCleanerCommitsRetained())
.archiveCommitsWith(writeConfig.getMetadataMinCommitsToKeep(), writeConfig.getMetadataMaxCommitsToKeep())
// we will trigger compaction manually, to control the instant times

View File

@@ -33,6 +33,7 @@ import org.apache.hudi.common.model.ActionType;
import org.apache.hudi.common.model.HoodieArchivedLogFile;
import org.apache.hudi.common.model.HoodieAvroPayload;
import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy;
import org.apache.hudi.common.model.HoodieReplaceCommitMetadata;
import org.apache.hudi.common.model.HoodieRollingStatMetadata;
import org.apache.hudi.common.table.HoodieTableMetaClient;
@@ -168,13 +169,17 @@ public class HoodieTimelineArchiveLog<T extends HoodieAvroPayload, I, K, O> {
HoodieTimeline commitTimeline = table.getCompletedCommitsTimeline();
Option<HoodieInstant> oldestPendingCompactionInstant =
table.getActiveTimeline().filterPendingCompactionTimeline().firstInstant();
Option<HoodieInstant> oldestInflightCommitInstant =
table.getActiveTimeline()
.getTimelineOfActions(CollectionUtils.createSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION))
.filterInflights().firstInstant();
// We cannot have any holes in the commit timeline. We cannot archive any commits which are
// made after the first savepoint present.
Option<HoodieInstant> firstSavepoint = table.getCompletedSavepointTimeline().firstInstant();
if (!commitTimeline.empty() && commitTimeline.countInstants() > maxInstantsToKeep) {
// Actually do the commits
return commitTimeline.getInstants()
Stream<HoodieInstant> instantToArchiveStream = commitTimeline.getInstants()
.filter(s -> {
// if no savepoint present, then dont filter
return !(firstSavepoint.isPresent() && HoodieTimeline.compareTimestamps(firstSavepoint.get().getTimestamp(), LESSER_THAN_OR_EQUALS, s.getTimestamp()));
@@ -183,7 +188,15 @@ public class HoodieTimelineArchiveLog<T extends HoodieAvroPayload, I, K, O> {
return oldestPendingCompactionInstant
.map(instant -> HoodieTimeline.compareTimestamps(instant.getTimestamp(), GREATER_THAN, s.getTimestamp()))
.orElse(true);
}).limit(commitTimeline.countInstants() - minInstantsToKeep);
});
// We need this to ensure that when multiple writers are performing conflict resolution, eligible instants don't
// get archived, i.e, instants after the oldestInflight are retained on the timeline
if (config.getFailedWritesCleanPolicy() == HoodieFailedWritesCleaningPolicy.LAZY) {
instantToArchiveStream = instantToArchiveStream.filter(s -> oldestInflightCommitInstant.map(instant ->
HoodieTimeline.compareTimestamps(instant.getTimestamp(), GREATER_THAN, s.getTimestamp()))
.orElse(true));
}
return instantToArchiveStream.limit(commitTimeline.countInstants() - minInstantsToKeep);
} else {
return Stream.empty();
}

View File

@@ -62,7 +62,7 @@ public abstract class BaseScheduleCompactionActionExecutor<T extends HoodieRecor
// Committed and pending compaction instants should have strictly lower timestamps
List<HoodieInstant> conflictingInstants = table.getActiveTimeline()
.getCommitsAndCompactionTimeline().getInstants()
.getWriteTimeline().getInstants()
.filter(instant -> HoodieTimeline.compareTimestamps(
instant.getTimestamp(), HoodieTimeline.GREATER_THAN_OR_EQUALS, instantTime))
.collect(Collectors.toList());

View File

@@ -63,7 +63,7 @@ public abstract class BaseRestoreActionExecutor<T extends HoodieRecordPayload, I
restoreTimer.startTimer();
// Get all the commits on the timeline after the provided commit time
List<HoodieInstant> instantsToRollback = table.getActiveTimeline().getCommitsAndCompactionTimeline()
List<HoodieInstant> instantsToRollback = table.getActiveTimeline().getWriteTimeline()
.getReverseOrderedInstants()
.filter(instant -> HoodieActiveTimeline.GREATER_THAN.test(instant.getTimestamp(), restoreInstantTime))
.collect(Collectors.toList());

View File

@@ -19,6 +19,7 @@
package org.apache.hudi.table.action.rollback;
import org.apache.hudi.avro.model.HoodieRollbackMetadata;
import org.apache.hudi.client.heartbeat.HoodieHeartbeatClient;
import org.apache.hudi.common.HoodieRollbackStat;
import org.apache.hudi.common.bootstrap.index.BootstrapIndex;
import org.apache.hudi.common.engine.HoodieEngineContext;
@@ -130,23 +131,37 @@ public abstract class BaseRollbackActionExecutor<T extends HoodieRecordPayload,
}
private void validateRollbackCommitSequence() {
final String instantTimeToRollback = instantToRollback.getTimestamp();
HoodieTimeline commitTimeline = table.getCompletedCommitsTimeline();
HoodieTimeline inflightAndRequestedCommitTimeline = table.getPendingCommitTimeline();
// Make sure only the last n commits are being rolled back
// If there is a commit in-between or after that is not rolled back, then abort
if ((instantTimeToRollback != null) && !commitTimeline.empty()
&& !commitTimeline.findInstantsAfter(instantTimeToRollback, Integer.MAX_VALUE).empty()) {
throw new HoodieRollbackException(
"Found commits after time :" + instantTimeToRollback + ", please rollback greater commits first");
}
// Continue to provide the same behavior if policy is EAGER (similar to pendingRollback logic). This is required
// since with LAZY rollback we support parallel writing which can allow a new inflight while rollback is ongoing
// Remove this once we support LAZY rollback of failed writes by default as parallel writing becomes the default
// writer mode.
if (config.getFailedWritesCleanPolicy().isEager()) {
final String instantTimeToRollback = instantToRollback.getTimestamp();
HoodieTimeline commitTimeline = table.getCompletedCommitsTimeline();
HoodieTimeline inflightAndRequestedCommitTimeline = table.getPendingCommitTimeline();
// Make sure only the last n commits are being rolled back
// If there is a commit in-between or after that is not rolled back, then abort
if ((instantTimeToRollback != null) && !commitTimeline.empty()
&& !commitTimeline.findInstantsAfter(instantTimeToRollback, Integer.MAX_VALUE).empty()) {
// check if remnants are from a previous LAZY rollback config, if yes, let out of order rollback continue
try {
if (!HoodieHeartbeatClient.heartbeatExists(table.getMetaClient().getFs(),
config.getBasePath(), instantTimeToRollback)) {
throw new HoodieRollbackException(
"Found commits after time :" + instantTimeToRollback + ", please rollback greater commits first");
}
} catch (IOException io) {
throw new HoodieRollbackException("Unable to rollback commits ", io);
}
}
List<String> inflights = inflightAndRequestedCommitTimeline.getInstants().map(HoodieInstant::getTimestamp)
.collect(Collectors.toList());
if ((instantTimeToRollback != null) && !inflights.isEmpty()
&& (inflights.indexOf(instantTimeToRollback) != inflights.size() - 1)) {
throw new HoodieRollbackException(
"Found in-flight commits after time :" + instantTimeToRollback + ", please rollback greater commits first");
List<String> inflights = inflightAndRequestedCommitTimeline.getInstants().map(HoodieInstant::getTimestamp)
.collect(Collectors.toList());
if ((instantTimeToRollback != null) && !inflights.isEmpty()
&& (inflights.indexOf(instantTimeToRollback) != inflights.size() - 1)) {
throw new HoodieRollbackException(
"Found in-flight commits after time :" + instantTimeToRollback + ", please rollback greater commits first");
}
}
}

View File

@@ -52,7 +52,7 @@ public class SavepointHelpers {
// Make sure the restore was successful
table.getMetaClient().reloadActiveTimeline();
Option<HoodieInstant> lastInstant = table.getActiveTimeline()
.getCommitsAndCompactionTimeline()
.getWriteTimeline()
.filterCompletedAndCompactionInstants()
.lastInstant();
ValidationUtils.checkArgument(lastInstant.isPresent());

View File

@@ -0,0 +1,93 @@
/*
* 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.client.heartbeat;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.common.testutils.HoodieCommonTestHarness;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import java.io.IOException;
import static java.util.concurrent.TimeUnit.SECONDS;
import static org.awaitility.Awaitility.await;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertTrue;
public class TestHoodieHeartbeatClient extends HoodieCommonTestHarness {
private static String instantTime1 = "100";
private static String instantTime2 = "101";
private static Long heartBeatInterval = 1000L;
private static int numTolerableMisses = 1;
@BeforeEach
public void init() throws IOException {
initMetaClient();
}
@Test
public void testStartHeartbeat() throws IOException {
HoodieHeartbeatClient hoodieHeartbeatClient =
new HoodieHeartbeatClient(metaClient.getFs(), metaClient.getBasePath(), heartBeatInterval, numTolerableMisses);
hoodieHeartbeatClient.start(instantTime1);
FileStatus [] fs = metaClient.getFs().listStatus(new Path(hoodieHeartbeatClient.getHeartbeatFolderPath()));
assertTrue(fs.length == 1);
assertTrue(fs[0].getPath().toString().contains(instantTime1));
}
@Test
public void testStopHeartbeat() {
HoodieHeartbeatClient hoodieHeartbeatClient =
new HoodieHeartbeatClient(metaClient.getFs(), metaClient.getBasePath(), heartBeatInterval, numTolerableMisses);
hoodieHeartbeatClient.start(instantTime1);
hoodieHeartbeatClient.stop(instantTime1);
await().atMost(5, SECONDS).until(() -> hoodieHeartbeatClient.getHeartbeat(instantTime1).getNumHeartbeats() > 0);
Integer numHeartBeats = hoodieHeartbeatClient.getHeartbeat(instantTime1).getNumHeartbeats();
assertTrue(numHeartBeats == 1);
}
@Test
public void testIsHeartbeatExpired() throws IOException {
HoodieHeartbeatClient hoodieHeartbeatClient =
new HoodieHeartbeatClient(metaClient.getFs(), metaClient.getBasePath(), heartBeatInterval, numTolerableMisses);
hoodieHeartbeatClient.start(instantTime1);
hoodieHeartbeatClient.stop(instantTime1);
assertFalse(hoodieHeartbeatClient.isHeartbeatExpired(instantTime1));
}
@Test
public void testNumHeartbeatsGenerated() {
Long heartBeatInterval = 5000L;
HoodieHeartbeatClient hoodieHeartbeatClient =
new HoodieHeartbeatClient(metaClient.getFs(), metaClient.getBasePath(), heartBeatInterval, numTolerableMisses);
hoodieHeartbeatClient.start("100");
await().atMost(5, SECONDS).until(() -> hoodieHeartbeatClient.getHeartbeat(instantTime1).getNumHeartbeats() >= 1);
}
@Test
public void testDeleteWrongHeartbeat() throws IOException {
HoodieHeartbeatClient hoodieHeartbeatClient =
new HoodieHeartbeatClient(metaClient.getFs(), metaClient.getBasePath(), heartBeatInterval, numTolerableMisses);
hoodieHeartbeatClient.start(instantTime1);
hoodieHeartbeatClient.stop(instantTime1);
assertFalse(HeartbeatUtils.deleteHeartbeatFile(metaClient.getFs(), basePath, instantTime2));
}
}