n3nash commented on a change in pull request #2359: URL: https://github.com/apache/hudi/pull/2359#discussion_r553182089
########## File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/heartbeat/HoodieHeartbeatClient.java ########## @@ -0,0 +1,265 @@ +/* + * 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.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.HashMap; +import java.util.Map; +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); + + // path to the heartbeat folder where all writers are updating their heartbeats + private final transient FileSystem fs; + private final String basePath; + private String heartBeatFolderPath; + // heartbeat interval in millis + private final Long heartbeatIntervalInMs; + private Integer numTolerableHeartbeatMisses; + private final Long maxAllowableHeartbeatInternalInMs; + 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.maxAllowableHeartbeatInternalInMs = 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 numHeartBeatsForCurrentInstantTime = 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 getNumHeartBeatsForCurrentInstantTime() { + return numHeartBeatsForCurrentInstantTime; + } + + public void setNumHeartBeatsForCurrentInstantTime(Integer numHeartBeatsForCurrentInstantTime) { + this.numHeartBeatsForCurrentInstantTime = numHeartBeatsForCurrentInstantTime; + } + + public Timer getTimer() { + return timer; + } + + public void setTimer(Timer timer) { + this.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.putIfAbsent(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()) { + heartbeat.getTimer().cancel(); + heartbeat.setHeartbeatStopped(true); + LOG.info("Stopping heartbeat 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 boolean delete(String instantTime) throws IOException { Review comment: yeah, I agree, refactored and moved to utils class ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org