HBASE-19633 Clean up the replication queues in the postPeerModification stage when removing a peer

This commit is contained in:
zhangduo 2018-01-02 09:57:23 +08:00
parent f89920a60f
commit 19707a85dd
20 changed files with 134 additions and 130 deletions

View File

@ -27,8 +27,8 @@ import java.util.Set;
import java.util.TreeMap;
import org.apache.hadoop.hbase.TableName;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.yetus.audience.InterfaceAudience;
/**
* A configuration for the replication peer cluster.

View File

@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.mapreduce.replication;
import java.io.IOException;
import java.util.Arrays;
import java.util.UUID;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.conf.Configured;
import org.apache.hadoop.fs.FileSystem;
@ -45,13 +44,14 @@ import org.apache.hadoop.hbase.filter.PrefixFilter;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.mapreduce.TableInputFormat;
import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
import org.apache.hadoop.hbase.mapreduce.TableSnapshotInputFormat;
import org.apache.hadoop.hbase.mapreduce.TableMapper;
import org.apache.hadoop.hbase.mapreduce.TableSnapshotInputFormat;
import org.apache.hadoop.hbase.mapreduce.TableSplit;
import org.apache.hadoop.hbase.replication.ReplicationException;
import org.apache.hadoop.hbase.replication.ReplicationFactory;
import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
import org.apache.hadoop.hbase.replication.ReplicationPeerStorage;
import org.apache.hadoop.hbase.replication.ReplicationPeers;
import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
import org.apache.hadoop.hbase.snapshot.RestoreSnapshotHelper;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
@ -66,6 +66,7 @@ import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
/**
@ -333,19 +334,24 @@ public class VerifyReplication extends Configured implements Tool {
final Configuration conf, String peerId) throws IOException {
ZKWatcher localZKW = null;
try {
localZKW = new ZKWatcher(conf, "VerifyReplication",
new Abortable() {
@Override public void abort(String why, Throwable e) {}
@Override public boolean isAborted() {return false;}
});
localZKW = new ZKWatcher(conf, "VerifyReplication", new Abortable() {
@Override
public void abort(String why, Throwable e) {
}
ReplicationPeers rp = ReplicationFactory.getReplicationPeers(localZKW, conf);
rp.init();
return Pair.newPair(rp.getPeerConfig(peerId), rp.getPeerClusterConfiguration(peerId));
@Override
public boolean isAborted() {
return false;
}
});
ReplicationPeerStorage storage =
ReplicationStorageFactory.getReplicationPeerStorage(localZKW, conf);
ReplicationPeerConfig peerConfig = storage.getPeerConfig(peerId);
return Pair.newPair(peerConfig,
ReplicationPeers.getPeerClusterConfiguration(peerConfig, conf));
} catch (ReplicationException e) {
throw new IOException(
"An error occurred while trying to connect to the remove peer cluster", e);
throw new IOException("An error occurred while trying to connect to the remove peer cluster",
e);
} finally {
if (localZKW != null) {
localZKW.close();

View File

@ -27,8 +27,6 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.replication.ReplicationPeer.PeerState;
import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
@ -39,20 +37,22 @@ import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesti
@InterfaceAudience.Private
public class ReplicationPeers {
private static final Logger LOG = LoggerFactory.getLogger(ReplicationPeers.class);
private final Configuration conf;
// Map of peer clusters keyed by their id
private final ConcurrentMap<String, ReplicationPeerImpl> peerCache;
private final ReplicationPeerStorage peerStorage;
protected ReplicationPeers(ZKWatcher zookeeper, Configuration conf) {
ReplicationPeers(ZKWatcher zookeeper, Configuration conf) {
this.conf = conf;
this.peerCache = new ConcurrentHashMap<>();
this.peerStorage = ReplicationStorageFactory.getReplicationPeerStorage(zookeeper, conf);
}
public Configuration getConf() {
return conf;
}
public void init() throws ReplicationException {
// Loading all existing peerIds into peer cache.
for (String peerId : this.peerStorage.listPeerIds()) {
@ -120,22 +120,13 @@ public class ReplicationPeers {
return peerCache.keySet();
}
public ReplicationPeerConfig getPeerConfig(String peerId) {
ReplicationPeer replicationPeer = this.peerCache.get(peerId);
if (replicationPeer == null) {
throw new IllegalArgumentException("Peer with id= " + peerId + " is not cached");
}
return replicationPeer.getPeerConfig();
}
public Configuration getPeerClusterConfiguration(String peerId) throws ReplicationException {
ReplicationPeerConfig peerConfig = peerStorage.getPeerConfig(peerId);
public static Configuration getPeerClusterConfiguration(ReplicationPeerConfig peerConfig,
Configuration baseConf) throws ReplicationException {
Configuration otherConf;
try {
otherConf = HBaseConfiguration.createClusterConf(this.conf, peerConfig.getClusterKey());
otherConf = HBaseConfiguration.createClusterConf(baseConf, peerConfig.getClusterKey());
} catch (IOException e) {
throw new ReplicationException("Can't get peer configuration for peerId=" + peerId, e);
throw new ReplicationException("Can't get peer configuration for peer " + peerConfig, e);
}
if (!peerConfig.getConfiguration().isEmpty()) {
@ -172,8 +163,9 @@ public class ReplicationPeers {
* @return object representing the peer
*/
private ReplicationPeerImpl createPeer(String peerId) throws ReplicationException {
ReplicationPeerConfig peerConf = peerStorage.getPeerConfig(peerId);
ReplicationPeerConfig peerConfig = peerStorage.getPeerConfig(peerId);
boolean enabled = peerStorage.isPeerEnabled(peerId);
return new ReplicationPeerImpl(getPeerClusterConfiguration(peerId), peerId, enabled, peerConf);
return new ReplicationPeerImpl(getPeerClusterConfiguration(peerConfig, conf), peerId, enabled,
peerConfig);
}
}

View File

@ -27,7 +27,6 @@ import java.util.List;
import java.util.Set;
import java.util.SortedSet;
import java.util.TreeSet;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HConstants;
@ -50,7 +49,7 @@ import org.apache.zookeeper.data.Stat;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
/**
* ZK based replication queue storage.

View File

@ -19,13 +19,13 @@ package org.apache.hadoop.hbase.replication;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream;
import org.apache.hbase.thirdparty.com.google.protobuf.CodedOutputStream;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;

View File

@ -19,7 +19,6 @@ package org.apache.hadoop.hbase.replication;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
@ -238,12 +237,6 @@ public abstract class TestReplicationStateBasic {
} catch (ReplicationException e) {
}
try {
assertNull(rp.getPeerClusterConfiguration("bogus"));
fail("Should have thrown an ReplicationException when passed a bogus peerId");
} catch (ReplicationException e) {
}
assertNumberOfPeers(0);
// Add some peers
@ -258,7 +251,8 @@ public abstract class TestReplicationStateBasic {
fail("There are no connected peers, should have thrown an IllegalArgumentException");
} catch (IllegalArgumentException e) {
}
assertEquals(KEY_ONE, ZKConfig.getZooKeeperClusterKey(rp.getPeerClusterConfiguration(ID_ONE)));
assertEquals(KEY_ONE, ZKConfig.getZooKeeperClusterKey(ReplicationPeers
.getPeerClusterConfiguration(rp.getPeerStorage().getPeerConfig(ID_ONE), rp.getConf())));
rp.getPeerStorage().removePeer(ID_ONE);
rp.removePeer(ID_ONE);
assertNumberOfPeers(1);

View File

@ -19,7 +19,6 @@ package org.apache.hadoop.hbase.master;
import com.google.protobuf.Descriptors;
import com.google.protobuf.Service;
import java.io.IOException;
import java.io.InterruptedIOException;
import java.lang.reflect.Constructor;
@ -51,12 +50,10 @@ import java.util.concurrent.atomic.AtomicReference;
import java.util.function.Function;
import java.util.regex.Pattern;
import java.util.stream.Collectors;
import javax.servlet.ServletException;
import javax.servlet.http.HttpServlet;
import javax.servlet.http.HttpServletRequest;
import javax.servlet.http.HttpServletResponse;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.ClusterMetrics;
@ -207,6 +204,7 @@ import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionServerInfo;

View File

@ -18,7 +18,6 @@
package org.apache.hadoop.hbase.master.replication;
import java.io.IOException;
import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
@ -74,8 +73,8 @@ public class AddPeerProcedure extends ModifyPeerProcedure {
@Override
protected void postPeerModification(MasterProcedureEnv env) throws IOException {
LOG.info("Successfully added " + (enabled ? "ENABLED" : "DISABLED") + " peer " + peerId +
", config " + peerConfig);
LOG.info("Successfully added {} peer {}, config {}", enabled ? "ENABLED" : "DISABLED", peerId,
peerConfig);
MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
if (cpHost != null) {
env.getMasterCoprocessorHost().postAddReplicationPeer(peerId, peerConfig);

View File

@ -18,7 +18,6 @@
package org.apache.hadoop.hbase.master.replication;
import java.io.IOException;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
import org.apache.hadoop.hbase.replication.ReplicationException;
@ -62,7 +61,7 @@ public class DisablePeerProcedure extends ModifyPeerProcedure {
@Override
protected void postPeerModification(MasterProcedureEnv env) throws IOException {
LOG.info("Successfully disabled peer " + peerId);
LOG.info("Successfully disabled peer {}", peerId);
MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
if (cpHost != null) {
cpHost.postDisableReplicationPeer(peerId);

View File

@ -18,7 +18,6 @@
package org.apache.hadoop.hbase.master.replication;
import java.io.IOException;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
import org.apache.hadoop.hbase.replication.ReplicationException;
@ -62,7 +61,7 @@ public class EnablePeerProcedure extends ModifyPeerProcedure {
@Override
protected void postPeerModification(MasterProcedureEnv env) throws IOException {
LOG.info("Successfully enabled peer " + peerId);
LOG.info("Successfully enabled peer {}", peerId);
MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
if (cpHost != null) {
cpHost.postEnableReplicationPeer(peerId);

View File

@ -18,7 +18,6 @@
package org.apache.hadoop.hbase.master.replication;
import java.io.IOException;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
import org.apache.hadoop.hbase.master.procedure.PeerProcedureInterface;
import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch;
@ -84,10 +83,13 @@ public abstract class ModifyPeerProcedure
* Called before we finish the procedure. The implementation can do some logging work, and also
* call the coprocessor hook if any.
* <p>
* Notice that, since we have already done the actual work, throwing exception here will not fail
* this procedure, we will just ignore it and finish the procedure as suceeded.
* Notice that, since we have already done the actual work, throwing {@code IOException} here will
* not fail this procedure, we will just ignore it and finish the procedure as suceeded. If
* {@code ReplicationException} is thrown we will retry since this usually means we fails to
* update the peer storage.
*/
protected abstract void postPeerModification(MasterProcedureEnv env) throws IOException;
protected abstract void postPeerModification(MasterProcedureEnv env)
throws IOException, ReplicationException;
private void releaseLatch() {
ProcedurePrepareLatch.releaseLatch(latch, this);
@ -101,16 +103,14 @@ public abstract class ModifyPeerProcedure
try {
prePeerModification(env);
} catch (IOException e) {
LOG.warn(
getClass().getName() + " failed to call CP hook or the pre check is failed for peer " +
peerId + ", mark the procedure as failure and give up",
e);
LOG.warn("{} failed to call pre CP hook or the pre check is failed for peer {}, " +
"mark the procedure as failure and give up", getClass().getName(), peerId, e);
setFailure("master-" + getPeerOperationType().name().toLowerCase() + "-peer", e);
releaseLatch();
return Flow.NO_MORE_STATE;
} catch (ReplicationException e) {
LOG.warn(getClass().getName() + " failed to call prePeerModification for peer " + peerId +
", retry", e);
LOG.warn("{} failed to call prePeerModification for peer {}, retry", getClass().getName(),
peerId, e);
throw new ProcedureYieldException();
}
setNextState(PeerModificationState.UPDATE_PEER_STORAGE);
@ -119,8 +119,8 @@ public abstract class ModifyPeerProcedure
try {
updatePeerStorage(env);
} catch (ReplicationException e) {
LOG.warn(
getClass().getName() + " update peer storage for peer " + peerId + " failed, retry", e);
LOG.warn("{} update peer storage for peer {} failed, retry", getClass().getName(), peerId,
e);
throw new ProcedureYieldException();
}
setNextState(PeerModificationState.REFRESH_PEER_ON_RS);
@ -134,9 +134,13 @@ public abstract class ModifyPeerProcedure
case POST_PEER_MODIFICATION:
try {
postPeerModification(env);
} catch (ReplicationException e) {
LOG.warn("{} failed to call postPeerModification for peer {}, retry",
getClass().getName(), peerId, e);
throw new ProcedureYieldException();
} catch (IOException e) {
LOG.warn(getClass().getName() + " failed to call prePeerModification for peer " + peerId +
", ignore since the procedure has already done", e);
LOG.warn("{} failed to call post CP hook for peer {}, " +
"ignore since the procedure has already done", getClass().getName(), peerId, e);
}
releaseLatch();
return Flow.NO_MORE_STATE;
@ -175,7 +179,7 @@ public abstract class ModifyPeerProcedure
throws IOException, InterruptedException {
if (state == PeerModificationState.PRE_PEER_MODIFICATION) {
// actually the peer related operations has no rollback, but if we haven't done any
// modifications on the peer storage, we can just return.
// modifications on the peer storage yet, we can just return.
return;
}
throw new UnsupportedOperationException();

View File

@ -18,7 +18,6 @@
package org.apache.hadoop.hbase.master.replication;
import java.io.IOException;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
import org.apache.hadoop.hbase.master.procedure.PeerProcedureInterface;
@ -122,17 +121,15 @@ public class RefreshPeerProcedure extends Procedure<MasterProcedureEnv>
private void complete(MasterProcedureEnv env, Throwable error) {
if (event == null) {
LOG.warn("procedure event for " + getProcId() +
" is null, maybe the procedure is created when recovery",
new Exception());
LOG.warn("procedure event for {} is null, maybe the procedure is created when recovery",
getProcId());
return;
}
if (error != null) {
LOG.warn("Refresh peer " + peerId + " for " + type + " on " + targetServer + " failed",
error);
LOG.warn("Refresh peer {} for {} on {} failed", peerId, type, targetServer, error);
this.succ = false;
} else {
LOG.info("Refresh peer " + peerId + " for " + type + " on " + targetServer + " suceeded");
LOG.info("Refresh peer {} for {} on {} suceeded", peerId, type, targetServer);
this.succ = true;
}
@ -168,9 +165,9 @@ public class RefreshPeerProcedure extends Procedure<MasterProcedureEnv>
dispatched = false;
}
if (!env.getRemoteDispatcher().addOperationToNode(targetServer, this)) {
LOG.info("Can not add remote operation for refreshing peer " + peerId + " for " + type +
" to " + targetServer + ", this usually because the server is already dead," +
" give up and mark the procedure as complete");
LOG.info("Can not add remote operation for refreshing peer {} for {} to {}, " +
"this usually because the server is already dead, " +
"give up and mark the procedure as complete", peerId, type, targetServer);
return null;
}
dispatched = true;

View File

@ -18,7 +18,6 @@
package org.apache.hadoop.hbase.master.replication;
import java.io.IOException;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
import org.apache.hadoop.hbase.replication.ReplicationException;
@ -61,8 +60,10 @@ public class RemovePeerProcedure extends ModifyPeerProcedure {
}
@Override
protected void postPeerModification(MasterProcedureEnv env) throws IOException {
LOG.info("Successfully removed peer " + peerId);
protected void postPeerModification(MasterProcedureEnv env)
throws IOException, ReplicationException {
env.getReplicationPeerManager().removeAllQueuesAndHFileRefs(peerId);
LOG.info("Successfully removed peer {}", peerId);
MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
if (cpHost != null) {
cpHost.postRemoveReplicationPeer(peerId);

View File

@ -28,7 +28,6 @@ import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.regex.Pattern;
import java.util.stream.Collectors;
import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.DoNotRetryIOException;
@ -217,6 +216,36 @@ public class ReplicationPeerManager {
return desc != null ? Optional.of(desc.getPeerConfig()) : Optional.empty();
}
private void removeAllQueues0(String peerId) throws ReplicationException {
for (ServerName replicator : queueStorage.getListOfReplicators()) {
List<String> queueIds = queueStorage.getAllQueues(replicator);
for (String queueId : queueIds) {
ReplicationQueueInfo queueInfo = new ReplicationQueueInfo(queueId);
if (queueInfo.getPeerId().equals(peerId)) {
queueStorage.removeQueue(replicator, queueId);
}
}
queueStorage.removeReplicatorIfQueueIsEmpty(replicator);
}
}
public void removeAllQueuesAndHFileRefs(String peerId) throws ReplicationException {
// Here we need two passes to address the problem of claimQueue. Maybe a claimQueue is still
// on-going when the refresh peer config procedure is done, if a RS which has already been
// scanned claims the queue of a RS which has not been scanned yet, we will miss that queue in
// the scan here, and if the RS who has claimed the queue crashed before creating recovered
// source, then the queue will leave there until the another RS detects the crash and helps
// removing the queue.
// A two pass scan can solve the problem. Anyway, the queue will not disappear during the
// claiming, it will either under the old RS or under the new RS, and a queue can only be
// claimed once after the refresh peer procedure done(as the next claim queue will just delete
// it), so we can make sure that a two pass scan will finally find the queue and remove it,
// unless it has already been removed by others.
removeAllQueues0(peerId);
removeAllQueues0(peerId);
queueStorage.removePeerFromHFileRefs(peerId);
}
private void checkPeerConfig(ReplicationPeerConfig peerConfig) throws DoNotRetryIOException {
checkClusterKey(peerConfig.getClusterKey());

View File

@ -18,7 +18,6 @@
package org.apache.hadoop.hbase.master.replication;
import java.io.IOException;
import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
@ -70,7 +69,7 @@ public class UpdatePeerConfigProcedure extends ModifyPeerProcedure {
@Override
protected void postPeerModification(MasterProcedureEnv env) throws IOException {
LOG.info("Successfully updated peer config of " + peerId + " to " + peerConfig);
LOG.info("Successfully updated peer config of {} to {}", peerId, peerConfig);
MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
if (cpHost != null) {
cpHost.postUpdateReplicationPeerConfig(peerId, peerConfig);

View File

@ -177,6 +177,15 @@ import org.slf4j.LoggerFactory;
import sun.misc.Signal;
import sun.misc.SignalHandler;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
import org.apache.hbase.thirdparty.com.google.protobuf.BlockingRpcChannel;
import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
import org.apache.hbase.thirdparty.com.google.protobuf.TextFormat;
import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCall;
@ -207,15 +216,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProto
import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
import org.apache.hbase.thirdparty.com.google.protobuf.BlockingRpcChannel;
import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
import org.apache.hbase.thirdparty.com.google.protobuf.TextFormat;
import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
/**
* HRegionServer makes a set of HRegions available to clients. It checks in with
* the HMaster. There are many HRegionServers in a single HBase deployment.

View File

@ -28,7 +28,8 @@ import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.TextFormat;
import org.apache.hbase.thirdparty.com.google.protobuf.TextFormat;
import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RemoteProcedureResult;
import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportProcedureDoneRequest;

View File

@ -20,11 +20,12 @@ package org.apache.hadoop.hbase.replication.regionserver;
import org.apache.hadoop.hbase.executor.EventType;
import org.apache.hadoop.hbase.procedure2.RSProcedureCallable;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.PeerModificationType;
import org.apache.log4j.Logger;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
import org.apache.hbase.thirdparty.com.google.protobuf.InvalidProtocolBufferException;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.PeerModificationType;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RefreshPeerParameter;
/**

View File

@ -59,7 +59,6 @@ import org.apache.hadoop.hbase.replication.ReplicationEndpoint;
import org.apache.hadoop.hbase.replication.ReplicationException;
import org.apache.hadoop.hbase.replication.ReplicationListener;
import org.apache.hadoop.hbase.replication.ReplicationPeer;
import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
import org.apache.hadoop.hbase.replication.ReplicationPeers;
import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
@ -306,9 +305,8 @@ public class ReplicationSourceManager implements ReplicationListener {
*/
@VisibleForTesting
ReplicationSourceInterface addSource(String id) throws IOException, ReplicationException {
ReplicationPeerConfig peerConfig = replicationPeers.getPeerConfig(id);
ReplicationPeer peer = replicationPeers.getPeer(id);
ReplicationSourceInterface src = getReplicationSource(id, peerConfig, peer);
ReplicationSourceInterface src = getReplicationSource(id, peer);
synchronized (this.walsById) {
this.sources.add(src);
Map<String, SortedSet<String>> walsByGroup = new HashMap<>();
@ -499,8 +497,8 @@ public class ReplicationSourceManager implements ReplicationListener {
* @param peerId the id of the peer cluster
* @return the created source
*/
private ReplicationSourceInterface getReplicationSource(String peerId,
ReplicationPeerConfig peerConfig, ReplicationPeer replicationPeer) throws IOException {
private ReplicationSourceInterface getReplicationSource(String peerId, ReplicationPeer peer)
throws IOException {
RegionServerCoprocessorHost rsServerHost = null;
TableDescriptors tableDescriptors = null;
if (server instanceof HRegionServer) {
@ -512,24 +510,24 @@ public class ReplicationSourceManager implements ReplicationListener {
ReplicationEndpoint replicationEndpoint = null;
try {
String replicationEndpointImpl = peerConfig.getReplicationEndpointImpl();
String replicationEndpointImpl = peer.getPeerConfig().getReplicationEndpointImpl();
if (replicationEndpointImpl == null) {
// Default to HBase inter-cluster replication endpoint
replicationEndpointImpl = HBaseInterClusterReplicationEndpoint.class.getName();
}
replicationEndpoint = Class.forName(replicationEndpointImpl)
.asSubclass(ReplicationEndpoint.class).newInstance();
if(rsServerHost != null) {
ReplicationEndpoint newReplicationEndPoint = rsServerHost
.postCreateReplicationEndPoint(replicationEndpoint);
if(newReplicationEndPoint != null) {
if (rsServerHost != null) {
ReplicationEndpoint newReplicationEndPoint =
rsServerHost.postCreateReplicationEndPoint(replicationEndpoint);
if (newReplicationEndPoint != null) {
// Override the newly created endpoint from the hook with configured end point
replicationEndpoint = newReplicationEndPoint;
}
}
} catch (Exception e) {
LOG.warn("Passed replication endpoint implementation throws errors"
+ " while initializing ReplicationSource for peer: " + peerId, e);
LOG.warn("Passed replication endpoint implementation throws errors" +
" while initializing ReplicationSource for peer: " + peerId, e);
throw new IOException(e);
}
@ -539,8 +537,8 @@ public class ReplicationSourceManager implements ReplicationListener {
replicationEndpoint, walFileLengthProvider, metrics);
// init replication endpoint
replicationEndpoint.init(new ReplicationEndpoint.Context(conf, replicationPeer.getConfiguration(),
fs, peerId, clusterId, replicationPeer, metrics, tableDescriptors, server));
replicationEndpoint.init(new ReplicationEndpoint.Context(conf, peer.getConfiguration(), fs,
peerId, clusterId, peer, metrics, tableDescriptors, server));
return src;
}
@ -736,17 +734,6 @@ public class ReplicationSourceManager implements ReplicationListener {
abortWhenFail(() -> queueStorage.removeQueue(server.getServerName(), peerId));
continue;
}
ReplicationPeerConfig peerConfig = null;
try {
peerConfig = replicationPeers.getPeerConfig(actualPeerId);
} catch (Exception e) {
LOG.warn("Skipping failover for peer:" + actualPeerId + " of node " + deadRS
+ ", failed to read peer config", e);
abortWhenFail(() -> queueStorage.removeQueue(server.getServerName(), peerId));
continue;
}
// track sources in walsByIdRecoveredQueues
Map<String, SortedSet<String>> walsByGroup = new HashMap<>();
walsByIdRecoveredQueues.put(peerId, walsByGroup);
@ -761,7 +748,7 @@ public class ReplicationSourceManager implements ReplicationListener {
}
// enqueue sources
ReplicationSourceInterface src = getReplicationSource(peerId, peerConfig, peer);
ReplicationSourceInterface src = getReplicationSource(peerId, peer);
// synchronized on oldsources to avoid adding recovered source for the to-be-removed peer
// see removePeer
synchronized (oldsources) {

View File

@ -15,19 +15,15 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.client.replication;
import java.io.IOException;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
import org.apache.hadoop.hbase.replication.TestReplicationBase;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableList;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableMap;
import org.apache.hadoop.hbase.testclassification.ClientTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;
@ -39,6 +35,9 @@ import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList;
import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableMap;
@Category({ MediumTests.class, ClientTests.class })
public class TestReplicationAdminUsingProcedure extends TestReplicationBase {