n3nash commented on a change in pull request #2359:
URL: https://github.com/apache/hudi/pull/2359#discussion_r549937742



##########
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/heartbeat/HoodieHeartbeatClient.java
##########
@@ -0,0 +1,196 @@
+/*
+ * 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.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import javax.annotation.concurrent.NotThreadSafe;
+import java.io.File;
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.io.OutputStream;
+import java.io.Serializable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+/**
+ * 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 {
+
+  // TODO : Throw new exception from thread
+  private static final Logger LOG = 
LogManager.getLogger(HoodieHeartbeatClient.class);
+
+  // path to the hearbeat folder where all writers are updating their 
heartbeats
+  private final transient FileSystem fs;
+  private final String basePath;
+  private String heartBeatFolderPath;
+  private String currentInstantTime;
+  private int numHeartBeatsForCurrentInstantTime;
+  // heartbeat interval in millis
+  private final long heartbeatIntervalInMillis;
+  private final transient ExecutorService executorService = 
Executors.newSingleThreadExecutor();
+  private volatile boolean shutdownRequested;
+  private boolean isHearbeatStarted = false;
+  private boolean isHeatBeatStopped = false;
+  private transient Future<?> heartBeatFuture;
+  private Long lastHeartBeatTime;
+  // This is required for testing. Problem : if we set 
heartbeatIntervalInSeconds really large, test takes longer, if
+  // we set it small and you are debugging with breakpoint, causes issues. 
Need to implement a Mock in tests
+  private final Boolean skipHeartBeatCheck;
+
+  public HoodieHeartbeatClient(FileSystem fs, String basePath, long 
heartbeatIntervalInSeconds) {
+    ValidationUtils.checkArgument(heartbeatIntervalInSeconds >= 1, "Cannot set 
heartbeat lower than 1 second");
+    this.fs = fs;
+    this.basePath = basePath;
+    this.heartBeatFolderPath = 
HoodieTableMetaClient.getHeartbeatFolderPath(basePath);
+    this.heartbeatIntervalInMillis = heartbeatIntervalInSeconds * 1000L;
+    this.skipHeartBeatCheck = true;
+  }
+
+  public void start(String instantTime) {
+    LOG.info("Received request to start heartbeat for instant time " + 
instantTime);
+    if (isHearbeatStarted && instantTime == currentInstantTime) {
+      // heartbeat already started, NO_OP
+      return;
+    } else if (instantTime != currentInstantTime) {
+      if (this.currentInstantTime != null) {
+        LOG.warn("Stopping heartbeat for previous instant time " + 
this.currentInstantTime);
+        this.stop(this.currentInstantTime);
+      }
+      this.currentInstantTime = instantTime;
+      this.numHeartBeatsForCurrentInstantTime = 0;
+      this.shutdownRequested = false;
+    }
+    try {
+      updateHeartbeat(instantTime);
+    } catch (Exception e) {
+      throw new RuntimeException("Unable to generate heartbeat");
+    }
+    this.heartBeatFuture = executorService.submit(() -> {
+      while (!shutdownRequested) {
+        try {
+          Thread.sleep(this.heartbeatIntervalInMillis);
+          updateHeartbeat(instantTime);
+        } catch (InterruptedIOException | InterruptedException ioe) {
+          LOG.warn("Thread controlling heartbeat was interrupted");
+        } catch (IOException ioe) {
+          LOG.error("Unable to create heartbeat file", ioe);
+          throw new RuntimeException(ioe);
+        }
+      }
+    });
+    this.isHearbeatStarted = true;
+    this.isHeatBeatStopped = false;
+  }
+
+  public void stop(String instantTime) throws HoodieException {
+    this.shutdownRequested = true;
+    if (isHearbeatStarted && !isHeatBeatStopped && this.heartBeatFuture != 
null) {
+      // TODO : Substract the amount of time it takes to execute 
updateHeartbeat from the sleep time to avoid race conditions
+      boolean isCancelled = this.heartBeatFuture.cancel(true);
+      if (!isCancelled) {
+        throw new HoodieException("Could not stop heartbeat client");
+      }
+      this.isHeatBeatStopped = true;
+      LOG.info("Stopping heartbeat for instant " + instantTime);
+    }
+  }
+
+  public void stop() throws HoodieException {
+    if (this.currentInstantTime != null) {
+      stop(this.currentInstantTime);
+    }
+  }
+
+  public boolean delete(String instantTime) throws IOException {
+    // TODO ensure that we are not deleting the current instant heartbeat, 
this might need storing the instant time
+    // and implementing another heartbeat utils class for everything else
+    if (this.currentInstantTime == instantTime && !this.isHeatBeatStopped) {
+      LOG.error("Cannot delete a currently running heartbeat, stop it first");
+      return false;
+    } else {
+      boolean deleted = this.fs.delete(new Path(heartBeatFolderPath + 
File.separator + instantTime), false);
+      if (!deleted) {
+        LOG.error("Failed to delete heartbeat for instant " + instantTime);
+      }
+      return deleted;
+    }
+  }
+
+  public static Long getLastHeartbeat(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 boolean checkIfConcurrentWriterRunning(String instantTime) throws 
IOException {
+    long lastHeartBeatForWriter = getLastHeartbeat(fs, basePath, instantTime);
+    long currentTime = System.currentTimeMillis();
+    if (currentTime - lastHeartBeatForWriter > this.heartbeatIntervalInMillis) 
{
+      return false;
+    }
+    return true;
+  }
+
+  private void updateHeartbeat(String instantTime) throws IOException {
+    Long newHeartBeatTime = System.currentTimeMillis();
+    OutputStream outputStream =

Review comment:
       Yes, a new file creation is atomic, overwrite is NOT. That is why I 
don't read the contents of the file since that causes race-conditions and the 
attempt to read fail, the modification time is an indicator of when was the 
last attempt to update heartbeat which is all we need. Also from the 
documentation here -> 
https://hadoop.apache.org/docs/r3.1.2/hadoop-project-dist/hadoop-common/filesystem/introduction.html
   `When OutputStream.close() is called, all remaining data is written, the 
file closed and the NameNode updated with the final size of the file. The 
modification time is set to the time the file was closed.`
   




----------------------------------------------------------------
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:
[email protected]


Reply via email to