Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -445,7 +445,7 @@ private IOException extractHiddenEof(Exception ex) {
&& 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;
}
Expand Down
Original file line number Diff line number Diff line change
@@ -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);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -474,8 +474,13 @@ private void interruptOrAbortWhenFail(ReplicationQueueOperation op) {
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);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -117,8 +117,9 @@ public final void run() {
} 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();
}
}
Expand Down