You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by hu...@apache.org on 2020/04/08 16:29:03 UTC
[hbase] branch master updated: HBASE-24120 Flakey Test:
TestReplicationAdminWithClusters timeout (#1441)
This is an automated email from the ASF dual-hosted git repository.
huaxiangsun pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hbase.git
The following commit(s) were added to refs/heads/master by this push:
new ed83022 HBASE-24120 Flakey Test: TestReplicationAdminWithClusters timeout (#1441)
ed83022 is described below
commit ed830222dad50520f5957703cfc31cfc36edd05d
Author: huaxiangsun <hu...@apache.org>
AuthorDate: Wed Apr 8 09:28:51 2020 -0700
HBASE-24120 Flakey Test: TestReplicationAdminWithClusters timeout (#1441)
Signed-off-by: stack <st...@apache.org>
Signed-off-by: Nick Dimiduk <nd...@apache.org>
Signed-off-by: Duo Zhang <zh...@apache.org>
---
.../hbase/regionserver/wal/ProtobufLogReader.java | 2 +-
.../regionserver/ReplicationRuntimeException.java | 40 ++++++++++++++++++++++
.../regionserver/ReplicationSourceManager.java | 9 +++--
.../regionserver/ReplicationSourceShipper.java | 5 +--
4 files changed, 51 insertions(+), 5 deletions(-)
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogReader.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogReader.java
index 96be61c..c48caad 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogReader.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogReader.java
@@ -445,7 +445,7 @@ public class ProtobufLogReader extends ReaderBase {
&& ex.getCause() != null && ex.getCause() instanceof IOException) {
ioEx = (IOException)ex.getCause();
}
- if (ioEx != null) {
+ if ((ioEx != null) && (ioEx.getMessage() != null)) {
if (ioEx.getMessage().contains("EOF")) return ioEx;
return null;
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationRuntimeException.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationRuntimeException.java
new file mode 100644
index 0000000..81ec0d9
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationRuntimeException.java
@@ -0,0 +1,40 @@
+/**
+ * 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.hadoop.hbase.replication.regionserver;
+
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * This exception is thrown when a replication source is terminated and source threads got
+ * interrupted.
+ *
+ * It is inherited from RuntimeException so that it can skip all the following processing logic
+ * and be propagated to the most top level and handled there.
+ */
+@InterfaceAudience.Private
+public class ReplicationRuntimeException extends RuntimeException {
+ private static final long serialVersionUID = 1L;
+
+ public ReplicationRuntimeException(String m, Throwable t) {
+ super(m, t);
+ }
+
+ public ReplicationRuntimeException(String m) {
+ super(m);
+ }
+}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
index 43afa79..6a4e863 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
@@ -579,8 +579,13 @@ public class ReplicationSourceManager implements ReplicationListener {
if (e.getCause() != null && e.getCause() instanceof KeeperException.SystemErrorException
&& e.getCause().getCause() != null && e.getCause()
.getCause() instanceof InterruptedException) {
- throw new RuntimeException(
- "Thread is interrupted, the replication source may be terminated");
+ // ReplicationRuntimeException(a RuntimeException) is thrown out here. The reason is
+ // that thread is interrupted deep down in the stack, it should pass the following
+ // processing logic and propagate to the most top layer which can handle this exception
+ // properly. In this specific case, the top layer is ReplicationSourceShipper#run().
+ throw new ReplicationRuntimeException(
+ "Thread is interrupted, the replication source may be terminated",
+ e.getCause().getCause());
}
server.abort("Failed to operate on replication queue", e);
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceShipper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceShipper.java
index cf42d4e..732b687 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceShipper.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceShipper.java
@@ -119,8 +119,9 @@ public class ReplicationSourceShipper extends Thread {
} else {
shipEdits(entryBatch);
}
- } catch (InterruptedException e) {
- LOG.trace("Interrupted while waiting for next replication entry batch", e);
+ } catch (InterruptedException | ReplicationRuntimeException e) {
+ // It is interrupted and needs to quit.
+ LOG.warn("Interrupted while waiting for next replication entry batch", e);
Thread.currentThread().interrupt();
}
}