You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by ro...@apache.org on 2023/10/11 08:57:28 UTC

[iotdb] branch rel/1.2 updated: Pipe: running count of the connector does not return to zero after exceptions occur (resulting in the failure of the automatic restart mechanism) (#11279) (#11284)

This is an automated email from the ASF dual-hosted git repository.

rong pushed a commit to branch rel/1.2
in repository https://gitbox.apache.org/repos/asf/iotdb.git


The following commit(s) were added to refs/heads/rel/1.2 by this push:
     new 5c9e1c06137 Pipe: running count of the connector does not return to zero after exceptions occur (resulting in the failure of the automatic restart mechanism) (#11279) (#11284)
5c9e1c06137 is described below

commit 5c9e1c0613718648dc6d18f965b6288fcc8001a2
Author: Steve Yurong Su <ro...@apache.org>
AuthorDate: Wed Oct 11 16:57:21 2023 +0800

    Pipe: running count of the connector does not return to zero after exceptions occur (resulting in the failure of the automatic restart mechanism) (#11279) (#11284)
    
    (cherry picked from commit 8951eb8ce045604914c38e261b03de503fd9916c)
---
 .../iotdb/db/pipe/agent/task/PipeTaskAgent.java    | 76 ++++++++++++++++++++++
 .../iotdb/db/pipe/task/subtask/PipeSubtask.java    | 10 +++
 .../PipeRuntimeConnectorCriticalException.java     |  6 +-
 .../pipe/PipeRuntimeCriticalException.java         |  5 --
 .../pipe/PipeRuntimeNonCriticalException.java      |  5 --
 .../iotdb/commons/pipe/task/meta/PipeTaskMeta.java | 26 ++++----
 6 files changed, 102 insertions(+), 26 deletions(-)

diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeTaskAgent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeTaskAgent.java
index a0cc1350949..bd26b0a0fe4 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeTaskAgent.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeTaskAgent.java
@@ -20,6 +20,7 @@
 package org.apache.iotdb.db.pipe.agent.task;
 
 import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId;
+import org.apache.iotdb.commons.exception.pipe.PipeRuntimeConnectorCriticalException;
 import org.apache.iotdb.commons.exception.pipe.PipeRuntimeCriticalException;
 import org.apache.iotdb.commons.exception.pipe.PipeRuntimeException;
 import org.apache.iotdb.commons.pipe.task.meta.PipeMeta;
@@ -36,10 +37,12 @@ import org.apache.iotdb.db.pipe.task.PipeBuilder;
 import org.apache.iotdb.db.pipe.task.PipeTask;
 import org.apache.iotdb.db.pipe.task.PipeTaskBuilder;
 import org.apache.iotdb.db.pipe.task.PipeTaskManager;
+import org.apache.iotdb.db.utils.DateTimeUtils;
 import org.apache.iotdb.mpp.rpc.thrift.THeartbeatResp;
 import org.apache.iotdb.mpp.rpc.thrift.TPipeHeartbeatReq;
 import org.apache.iotdb.mpp.rpc.thrift.TPipeHeartbeatResp;
 import org.apache.iotdb.mpp.rpc.thrift.TPushPipeMetaRespExceptionMessage;
+import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameters;
 
 import org.apache.thrift.TException;
 import org.slf4j.Logger;
@@ -51,6 +54,7 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
@@ -404,6 +408,71 @@ public class PipeTaskAgent {
   }
 
   private void stopAllPipesWithCriticalExceptionInternal() {
+    // 1. track exception in all pipe tasks that share the same connector that have critical
+    // exceptions.
+    final int currentDataNodeId = IoTDBDescriptor.getInstance().getConfig().getDataNodeId();
+    final Map<PipeParameters, PipeRuntimeConnectorCriticalException>
+        reusedConnectorParameters2ExceptionMap = new HashMap<>();
+
+    pipeMetaKeeper
+        .getPipeMetaList()
+        .forEach(
+            pipeMeta -> {
+              final PipeStaticMeta staticMeta = pipeMeta.getStaticMeta();
+              final PipeRuntimeMeta runtimeMeta = pipeMeta.getRuntimeMeta();
+
+              runtimeMeta
+                  .getConsensusGroupId2TaskMetaMap()
+                  .values()
+                  .forEach(
+                      pipeTaskMeta -> {
+                        if (pipeTaskMeta.getLeaderDataNodeId() != currentDataNodeId) {
+                          return;
+                        }
+
+                        for (final PipeRuntimeException e : pipeTaskMeta.getExceptionMessages()) {
+                          if (e instanceof PipeRuntimeConnectorCriticalException) {
+                            reusedConnectorParameters2ExceptionMap.putIfAbsent(
+                                staticMeta.getConnectorParameters(),
+                                (PipeRuntimeConnectorCriticalException) e);
+                          }
+                        }
+                      });
+            });
+    pipeMetaKeeper
+        .getPipeMetaList()
+        .forEach(
+            pipeMeta -> {
+              final PipeStaticMeta staticMeta = pipeMeta.getStaticMeta();
+              final PipeRuntimeMeta runtimeMeta = pipeMeta.getRuntimeMeta();
+
+              runtimeMeta
+                  .getConsensusGroupId2TaskMetaMap()
+                  .values()
+                  .forEach(
+                      pipeTaskMeta -> {
+                        if (pipeTaskMeta.getLeaderDataNodeId() == currentDataNodeId
+                            && reusedConnectorParameters2ExceptionMap.containsKey(
+                                staticMeta.getConnectorParameters())
+                            && !pipeTaskMeta.containsExceptionMessage(
+                                reusedConnectorParameters2ExceptionMap.get(
+                                    staticMeta.getConnectorParameters()))) {
+                          final PipeRuntimeConnectorCriticalException exception =
+                              reusedConnectorParameters2ExceptionMap.get(
+                                  staticMeta.getConnectorParameters());
+                          pipeTaskMeta.trackExceptionMessage(exception);
+                          LOGGER.warn(
+                              "Pipe {} (creation time = {}) will be stopped because of critical exception "
+                                  + "(occurred time {}) in connector {}.",
+                              staticMeta.getPipeName(),
+                              DateTimeUtils.convertLongToDate(staticMeta.getCreationTime(), "ms"),
+                              DateTimeUtils.convertLongToDate(exception.getTimeStamp(), "ms"),
+                              staticMeta.getConnectorParameters());
+                        }
+                      });
+            });
+
+    // 2. stop all pipes that have critical exceptions.
     pipeMetaKeeper
         .getPipeMetaList()
         .forEach(
@@ -420,6 +489,13 @@ public class PipeTaskAgent {
                           for (final PipeRuntimeException e : pipeTaskMeta.getExceptionMessages()) {
                             if (e instanceof PipeRuntimeCriticalException) {
                               stopPipe(staticMeta.getPipeName(), staticMeta.getCreationTime());
+                              LOGGER.warn(
+                                  "Pipe {} (creation time = {}) was stopped because of critical exception "
+                                      + "(occurred time {}).",
+                                  staticMeta.getPipeName(),
+                                  DateTimeUtils.convertLongToDate(
+                                      staticMeta.getCreationTime(), "ms"),
+                                  DateTimeUtils.convertLongToDate(e.getTimeStamp(), "ms"));
                               return;
                             }
                           }
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/subtask/PipeSubtask.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/subtask/PipeSubtask.java
index 5a01d4afe7c..d4e35c35200 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/subtask/PipeSubtask.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/subtask/PipeSubtask.java
@@ -137,6 +137,16 @@ public abstract class PipeSubtask
           this.getClass().getSimpleName(),
           retryCount.get(),
           MAX_RETRY_TIMES);
+      try {
+        Thread.sleep(1000L * retryCount.get());
+      } catch (InterruptedException e) {
+        LOGGER.warn(
+            "Interrupted when retrying to execute subtask {}({})",
+            taskID,
+            this.getClass().getSimpleName());
+        Thread.currentThread().interrupt();
+      }
+
       submitSelf();
     } else {
       final String errorMessage =
diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/exception/pipe/PipeRuntimeConnectorCriticalException.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/exception/pipe/PipeRuntimeConnectorCriticalException.java
index e5a8d71170c..1325ca55bda 100644
--- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/exception/pipe/PipeRuntimeConnectorCriticalException.java
+++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/exception/pipe/PipeRuntimeConnectorCriticalException.java
@@ -29,6 +29,7 @@ import java.nio.ByteBuffer;
 import java.util.Objects;
 
 public class PipeRuntimeConnectorCriticalException extends PipeRuntimeCriticalException {
+
   public PipeRuntimeConnectorCriticalException(String message) {
     super(message);
   }
@@ -44,11 +45,6 @@ public class PipeRuntimeConnectorCriticalException extends PipeRuntimeCriticalEx
         && Objects.equals(getTimeStamp(), ((PipeRuntimeException) obj).getTimeStamp());
   }
 
-  @Override
-  public int hashCode() {
-    return Objects.hash(getMessage(), getTimeStamp());
-  }
-
   @Override
   public void serialize(ByteBuffer byteBuffer) {
     PipeRuntimeExceptionType.CONNECTOR_CRITICAL_EXCEPTION.serialize(byteBuffer);
diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/exception/pipe/PipeRuntimeCriticalException.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/exception/pipe/PipeRuntimeCriticalException.java
index 9f9076be32d..0d3c0c2a786 100644
--- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/exception/pipe/PipeRuntimeCriticalException.java
+++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/exception/pipe/PipeRuntimeCriticalException.java
@@ -45,11 +45,6 @@ public class PipeRuntimeCriticalException extends PipeRuntimeException {
         && Objects.equals(getTimeStamp(), ((PipeRuntimeException) obj).getTimeStamp());
   }
 
-  @Override
-  public int hashCode() {
-    return Objects.hash(getMessage(), getTimeStamp());
-  }
-
   @Override
   public void serialize(ByteBuffer byteBuffer) {
     PipeRuntimeExceptionType.CRITICAL_EXCEPTION.serialize(byteBuffer);
diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/exception/pipe/PipeRuntimeNonCriticalException.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/exception/pipe/PipeRuntimeNonCriticalException.java
index 64c7a66b2d8..6b072a4b6ab 100644
--- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/exception/pipe/PipeRuntimeNonCriticalException.java
+++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/exception/pipe/PipeRuntimeNonCriticalException.java
@@ -45,11 +45,6 @@ public class PipeRuntimeNonCriticalException extends PipeRuntimeException {
         && Objects.equals(getTimeStamp(), ((PipeRuntimeException) obj).getTimeStamp());
   }
 
-  @Override
-  public int hashCode() {
-    return Objects.hash(getMessage(), getTimeStamp());
-  }
-
   @Override
   public void serialize(ByteBuffer byteBuffer) {
     PipeRuntimeExceptionType.NON_CRITICAL_EXCEPTION.serialize(byteBuffer);
diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/PipeTaskMeta.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/PipeTaskMeta.java
index d24ed912bf6..e5a80f681be 100644
--- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/PipeTaskMeta.java
+++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/PipeTaskMeta.java
@@ -34,10 +34,9 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
-import java.util.Arrays;
+import java.util.Map;
 import java.util.Objects;
-import java.util.Queue;
-import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReference;
 
@@ -55,7 +54,8 @@ public class PipeTaskMeta {
    * <p>The failure of them, respectively, will lead to the stop of the pipe, the stop of the pipes
    * sharing the same connector, and nothing.
    */
-  private final Queue<PipeRuntimeException> exceptionMessages = new ConcurrentLinkedQueue<>();
+  private final Map<PipeRuntimeException, PipeRuntimeException> exceptionMessages =
+      new ConcurrentHashMap<>();
 
   public PipeTaskMeta(/* @NotNull */ ProgressIndex progressIndex, int leaderDataNodeId) {
     this.progressIndex.set(progressIndex);
@@ -80,11 +80,15 @@ public class PipeTaskMeta {
   }
 
   public synchronized Iterable<PipeRuntimeException> getExceptionMessages() {
-    return new ArrayList<>(exceptionMessages);
+    return new ArrayList<>(exceptionMessages.values());
   }
 
   public synchronized void trackExceptionMessage(PipeRuntimeException exceptionMessage) {
-    exceptionMessages.add(exceptionMessage);
+    exceptionMessages.put(exceptionMessage, exceptionMessage);
+  }
+
+  public synchronized boolean containsExceptionMessage(PipeRuntimeException exceptionMessage) {
+    return exceptionMessages.containsKey(exceptionMessage);
   }
 
   public synchronized void clearExceptionMessages() {
@@ -97,7 +101,7 @@ public class PipeTaskMeta {
     ReadWriteIOUtils.write(leaderDataNodeId.get(), outputStream);
 
     ReadWriteIOUtils.write(exceptionMessages.size(), outputStream);
-    for (final PipeRuntimeException pipeRuntimeException : exceptionMessages) {
+    for (final PipeRuntimeException pipeRuntimeException : exceptionMessages.values()) {
       pipeRuntimeException.serialize(outputStream);
     }
   }
@@ -108,7 +112,7 @@ public class PipeTaskMeta {
     ReadWriteIOUtils.write(leaderDataNodeId.get(), outputStream);
 
     ReadWriteIOUtils.write(exceptionMessages.size(), outputStream);
-    for (final PipeRuntimeException pipeRuntimeException : exceptionMessages) {
+    for (final PipeRuntimeException pipeRuntimeException : exceptionMessages.values()) {
       pipeRuntimeException.serialize(outputStream);
     }
   }
@@ -123,7 +127,7 @@ public class PipeTaskMeta {
     for (int i = 0; i < size; ++i) {
       final PipeRuntimeException pipeRuntimeException =
           PipeRuntimeExceptionType.deserializeFrom(version, byteBuffer);
-      pipeTaskMeta.exceptionMessages.add(pipeRuntimeException);
+      pipeTaskMeta.exceptionMessages.put(pipeRuntimeException, pipeRuntimeException);
     }
     return pipeTaskMeta;
   }
@@ -139,7 +143,7 @@ public class PipeTaskMeta {
     for (int i = 0; i < size; ++i) {
       final PipeRuntimeException pipeRuntimeException =
           PipeRuntimeExceptionType.deserializeFrom(version, inputStream);
-      pipeTaskMeta.exceptionMessages.add(pipeRuntimeException);
+      pipeTaskMeta.exceptionMessages.put(pipeRuntimeException, pipeRuntimeException);
     }
     return pipeTaskMeta;
   }
@@ -155,7 +159,7 @@ public class PipeTaskMeta {
     PipeTaskMeta that = (PipeTaskMeta) obj;
     return progressIndex.get().equals(that.progressIndex.get())
         && leaderDataNodeId.get() == that.leaderDataNodeId.get()
-        && Arrays.equals(exceptionMessages.toArray(), that.exceptionMessages.toArray());
+        && exceptionMessages.equals(that.exceptionMessages);
   }
 
   @Override