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



##########
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java
##########
@@ -232,17 +250,18 @@ void emitCommitMetrics(String instantTime, 
HoodieCommitMetadata metadata, String
    * 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);
+    if (isEager(config.getFailedWritesCleanPolicy())) {

Review comment:
       `config` is a member variable right? why do we pass it in to the checks? 
Can we just do `eagerCleanFailedWrites()`, which does he if block and the call 
to `rollbackFailedBootstrap()`?

##########
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieClient.java
##########
@@ -70,6 +72,8 @@ protected AbstractHoodieClient(HoodieEngineContext context, 
HoodieWriteConfig cl
     this.config = clientConfig;
     this.timelineServer = timelineServer;
     shouldStopTimelineServer = !timelineServer.isPresent();
+    this.heartbeatClient = new HoodieHeartbeatClient(this.fs, this.basePath,
+        clientConfig.getHoodieClientHeartbeatIntervalInMs(), 
clientConfig.getHoodieClientHeartbeatTolerableMisses());

Review comment:
       rename to be shorter? drop the `HoodieClient` or `Hoodie` part in the 
names?
   

##########
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java
##########
@@ -707,24 +739,51 @@ public void rollbackInflightCompaction(HoodieInstant 
inflightInstant, HoodieTabl
   }
 
   /**
-   * Cleanup all pending commits.
+   * Rollback all failed commits.
    */
-  private void rollbackPendingCommits() {
+  public void rollbackFailedCommits() {
     HoodieTable<T, I, K, O> table = createTable(config, hadoopConf);
-    HoodieTimeline inflightTimeline = 
table.getMetaClient().getCommitsTimeline().filterPendingExcludingCompaction();
-    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 the heartbeats from DFS
+      if (!isEager(config.getFailedWritesCleanPolicy())) {
+        try {
+          this.heartbeatClient.delete(instant);
+        } catch (IOException io) {
+          LOG.error(io);
+        }
       }
     }
   }
 
+  private List<String> getInstantsToRollback(HoodieTable<T, I, K, O> table) {
+    if (isEager(config.getFailedWritesCleanPolicy())) {
+      HoodieTimeline inflightTimeline = 
table.getMetaClient().getCommitsTimeline().filterPendingExcludingCompaction();
+      return 
inflightTimeline.getReverseOrderedInstants().map(HoodieInstant::getTimestamp)
+          .collect(Collectors.toList());
+    } else if (!isEager(config.getFailedWritesCleanPolicy())) {
+      return table.getMetaClient().getActiveTimeline()
+          
.getCommitsTimeline().filterInflights().getReverseOrderedInstants().filter(instant
 -> {
+            try {
+              return 
heartbeatClient.isHeartbeatExpired(instant.getTimestamp(), 
System.currentTimeMillis());

Review comment:
       drop the second argument to `isHeartbeatExpired()` and have it get the 
current time epoch inside?

##########
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java
##########
@@ -707,24 +739,51 @@ public void rollbackInflightCompaction(HoodieInstant 
inflightInstant, HoodieTabl
   }
 
   /**
-   * Cleanup all pending commits.
+   * Rollback all failed commits.
    */
-  private void rollbackPendingCommits() {
+  public void rollbackFailedCommits() {
     HoodieTable<T, I, K, O> table = createTable(config, hadoopConf);
-    HoodieTimeline inflightTimeline = 
table.getMetaClient().getCommitsTimeline().filterPendingExcludingCompaction();
-    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 the heartbeats from DFS
+      if (!isEager(config.getFailedWritesCleanPolicy())) {
+        try {
+          this.heartbeatClient.delete(instant);

Review comment:
       how do we cleanup if we fail after rolling back, but before deleting the 
heartbeat?

##########
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;

Review comment:
       typo: interval

##########
File path: 
hudi-common/src/main/java/org/apache/hudi/common/model/HoodieFailedWritesCleaningPolicy.java
##########
@@ -0,0 +1,23 @@
+/*
+ * 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.common.model;
+
+public enum HoodieFailedWritesCleaningPolicy {
+  EAGER, LAZY, NEVER

Review comment:
       docs for all these three modes please. 

##########
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java
##########
@@ -408,13 +426,14 @@ protected void postCommit(HoodieTable<T, I, K, O> table, 
HoodieCommitMetadata me
       HoodieTimelineArchiveLog archiveLog = new 
HoodieTimelineArchiveLog(config, table);
       archiveLog.archiveIfRequired(context);
       autoCleanOnCommit(instantTime);
+      this.heartbeatClient.stop(instantTime);
     } catch (IOException ioe) {
       throw new HoodieIOException(ioe.getMessage(), ioe);
     }
   }
 
   protected void runAnyPendingCompactions(HoodieTable<T, I, K, O> table) {
-    
table.getActiveTimeline().getCommitsAndCompactionTimeline().filterPendingCompactionTimeline().getInstants()
+    
table.getActiveTimeline().getCommitsCompactionAndReplaceTimeline().filterPendingCompactionTimeline().getInstants()

Review comment:
       should we just call `getCommitsCompactionAndReplaceTimeline()` -> 
`getCompletedWriteTimeline()` ? we use writes already for other things

##########
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java
##########
@@ -580,6 +602,11 @@ public HoodieRestoreMetadata restoreToInstant(final String 
instantTime) throws H
   public HoodieCleanMetadata clean(String cleanInstantTime) throws 
HoodieIOException {
     LOG.info("Cleaner started");
     final Timer.Context timerContext = metrics.getCleanCtx();
+    LOG.info("Cleaning failed attempts if any");
+    if (!isEager(config.getFailedWritesCleanPolicy())) {

Review comment:
       pull both lines into a method that takes 0 args?

##########
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java
##########
@@ -707,24 +739,51 @@ public void rollbackInflightCompaction(HoodieInstant 
inflightInstant, HoodieTabl
   }
 
   /**
-   * Cleanup all pending commits.
+   * Rollback all failed commits.
    */
-  private void rollbackPendingCommits() {
+  public void rollbackFailedCommits() {
     HoodieTable<T, I, K, O> table = createTable(config, hadoopConf);
-    HoodieTimeline inflightTimeline = 
table.getMetaClient().getCommitsTimeline().filterPendingExcludingCompaction();
-    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 the heartbeats from DFS
+      if (!isEager(config.getFailedWritesCleanPolicy())) {
+        try {
+          this.heartbeatClient.delete(instant);
+        } catch (IOException io) {
+          LOG.error(io);

Review comment:
       some context on the error?

##########
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java
##########
@@ -580,6 +602,11 @@ public HoodieRestoreMetadata restoreToInstant(final String 
instantTime) throws H
   public HoodieCleanMetadata clean(String cleanInstantTime) throws 
HoodieIOException {
     LOG.info("Cleaner started");
     final Timer.Context timerContext = metrics.getCleanCtx();
+    LOG.info("Cleaning failed attempts if any");
+    if (!isEager(config.getFailedWritesCleanPolicy())) {
+      rollbackFailedCommits();

Review comment:
       rename to `rollbackFailedWrites()` 

##########
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java
##########
@@ -707,24 +739,51 @@ public void rollbackInflightCompaction(HoodieInstant 
inflightInstant, HoodieTabl
   }
 
   /**
-   * Cleanup all pending commits.
+   * Rollback all failed commits.
    */
-  private void rollbackPendingCommits() {
+  public void rollbackFailedCommits() {
     HoodieTable<T, I, K, O> table = createTable(config, hadoopConf);
-    HoodieTimeline inflightTimeline = 
table.getMetaClient().getCommitsTimeline().filterPendingExcludingCompaction();
-    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);

Review comment:
       To confirm my understanding, this will log rollback blocks for a much 
older data block and as long as that instant is on the active timeline, the 
merging/reading part will continue to ignore the blocks/files corresponding to 
the failed write.
   
   In terms of multiple writing, one of the writers has to be responsible for 
doing all "table services" (cleaning, compaction, clustering, failedwrite 
cleaning, archiving) and another writer could be writing a new data block into 
the file group, while this failed cleanup is happening? and we can handle this, 
right?

##########
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java
##########
@@ -859,6 +931,10 @@ protected void 
setWriteSchemaForDeletes(HoodieTableMetaClient metaClient) {
     }
   }
 
+  private boolean isEager(HoodieFailedWritesCleaningPolicy cleaningPolicy) {

Review comment:
       Like I mentioned above, we can make turn this into 
`eagerCleanFailedWrites()` returning boolean, working off the member variable 
`config` ? In its current form, this should probably reside in 
`HoodieFailedWritesCleaningPolicy` itself

##########
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCompactionConfig.java
##########
@@ -260,6 +264,11 @@ public Builder withCleanBootstrapBaseFileEnabled(Boolean 
cleanBootstrapSourceFil
       return this;
     }
 
+    public Builder 
withCleanFailedWritesPolicy(HoodieFailedWritesCleaningPolicy 
failedWritesPolicy) {

Review comment:
       withFailedWritesCleaningPolicy(). can we make this consistent everywhere.

##########
File path: hudi-client/hudi-spark-client/pom.xml
##########
@@ -127,6 +127,14 @@
       <scope>test</scope>
     </dependency>
 
+    <!-- Other Utils -->

Review comment:
       can we pull this into parent pom, introduce a property for the version 
and just use this without the `<version>` tag here. just like other deps/

##########
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;

Review comment:
       just `numHeartBeats`? 

##########
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java
##########
@@ -712,24 +742,43 @@ public void rollbackInflightCompaction(HoodieInstant 
inflightInstant, HoodieTabl
   }
 
   /**
-   * Cleanup all pending commits.
+   * Rollback all failed commits.
    */
-  private void rollbackPendingCommits() {
+  public void rollbackFailedCommits() {

Review comment:
       okay makes sense.  left a comment above for renaming.

##########
File path: 
hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/heartbeat/TestHoodieHeartbeatClient.java
##########
@@ -0,0 +1,152 @@
+/*
+ * 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.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.Test;
+
+import java.io.IOException;
+import java.io.OutputStream;
+
+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 final Logger LOG = 
LogManager.getLogger(TestHoodieHeartbeatClient.class);
+
+  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(2, SECONDS).until(() -> 
hoodieHeartbeatClient.getHeartbeat(instantTime1).getNumHeartBeatsForCurrentInstantTime()
 > 0);

Review comment:
       the 2 second limit is a bit tight. the test jvm can gc etc. can we make 
this longer.  same check for other time based tests here.

##########
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);

Review comment:
       is there a reason to make this map synchronzied/concurrent ?

##########
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:
       I find this a bit odd, for heartBeat to have a `delete()` method? can 
this clean up be done as part of `stop()` ? esp given delete() requires a stop 
first

##########
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTimelineArchiveLog.java
##########
@@ -165,13 +167,17 @@ public boolean archiveIfRequired(HoodieEngineContext 
context) throws IOException
     HoodieTimeline commitTimeline = table.getCompletedCommitsTimeline();
     Option<HoodieInstant> oldestPendingCompactionInstant =
         
table.getActiveTimeline().filterPendingCompactionTimeline().firstInstant();
+    Option<HoodieInstant> oldestInflightInstant =
+        table.getActiveTimeline()
+            
.getTimelineOfActions(CollectionUtils.createSet(HoodieTimeline.COMMIT_ACTION, 
HoodieTimeline.DELTA_COMMIT_ACTION))

Review comment:
       makes sense to have the lazy cleaning for just the true write actiosn. 




----------------------------------------------------------------
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


Reply via email to