HBASE-24120 Flakey Test: TestReplicationAdminWithClusters timeout (#1441)

Signed-off-by: stack <stack@apache.org>
Signed-off-by: Nick Dimiduk <ndimiduk@apache.org>
Signed-off-by: Duo Zhang <zhangduo@apache.org>
This commit is contained in:
huaxiangsun 2020-04-08 09:28:51 -07:00 committed by GitHub
parent c383bd1bb3
commit ed830222da
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
4 changed files with 51 additions and 5 deletions

View File

@ -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;
}

View File

@ -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);
}
}

View File

@ -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);
}

View File

@ -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();
}
}