From 523b6278ef366b5bec83378ed5feaf6d2d248994 Mon Sep 17 00:00:00 2001 From: zhangduo Date: Mon, 8 Jan 2018 15:27:28 +0800 Subject: [PATCH] HBASE-19397 Design procedures for ReplicationManager to notify peer change event from master --- .../java/org/apache/hadoop/hbase/client/Admin.java | 87 +- .../org/apache/hadoop/hbase/client/HBaseAdmin.java | 149 ++- .../hadoop/hbase/client/RawAsyncHBaseAdmin.java | 82 +- .../replication/ReplicationPeerConfigUtil.java | 10 +- .../apache/hadoop/hbase/protobuf/ProtobufUtil.java | 11 +- .../hbase/replication/ReplicationPeerConfig.java | 3 +- .../hadoop/hbase/shaded/protobuf/ProtobufUtil.java | 13 +- .../apache/hadoop/hbase/util/CollectionUtils.java | 3 + .../mapreduce/replication/VerifyReplication.java | 46 +- .../hbase/procedure2/LockedResourceType.java | 4 +- .../procedure2/RemoteProcedureDispatcher.java | 22 +- .../src/main/protobuf/Admin.proto | 9 +- .../src/main/protobuf/LockService.proto | 1 + .../src/main/protobuf/MasterProcedure.proto | 45 + .../src/main/protobuf/RegionServerStatus.proto | 19 + .../src/main/protobuf/Replication.proto | 5 + hbase-replication/pom.xml | 12 + .../hbase/replication/ReplicationFactory.java | 31 +- .../hbase/replication/ReplicationListener.java | 14 - .../hadoop/hbase/replication/ReplicationPeer.java | 44 +- .../hbase/replication/ReplicationPeerImpl.java | 125 +++ .../hbase/replication/ReplicationPeerStorage.java | 73 ++ .../hbase/replication/ReplicationPeerZKImpl.java | 314 ------ .../hadoop/hbase/replication/ReplicationPeers.java | 209 ++-- .../hbase/replication/ReplicationPeersZKImpl.java | 553 ---------- .../hbase/replication/ReplicationQueueInfo.java | 23 +- .../hbase/replication/ReplicationQueueStorage.java | 172 ++++ .../hbase/replication/ReplicationQueues.java | 161 --- .../replication/ReplicationQueuesArguments.java | 70 -- .../hbase/replication/ReplicationQueuesClient.java | 93 -- .../ReplicationQueuesClientArguments.java | 40 - .../replication/ReplicationQueuesClientZKImpl.java | 177 ---- .../hbase/replication/ReplicationQueuesZKImpl.java | 408 -------- .../hbase/replication/ReplicationStateZKBase.java | 156 --- .../replication/ReplicationStorageFactory.java | 49 + .../hbase/replication/ReplicationTableBase.java | 441 -------- .../replication/ReplicationTrackerZKImpl.java | 105 +- .../hadoop/hbase/replication/ReplicationUtils.java | 125 +++ .../replication/ZKReplicationPeerStorage.java | 169 +++ .../replication/ZKReplicationQueueStorage.java | 496 +++++++++ .../replication/ZKReplicationStorageBase.java | 77 ++ .../replication/TestReplicationStateBasic.java | 317 ++++++ .../replication/TestReplicationStateZKImpl.java | 89 ++ .../replication/TestZKReplicationPeerStorage.java | 173 ++++ .../replication/TestZKReplicationQueueStorage.java | 245 +++++ .../hbase/coprocessor/RegionServerObserver.java | 14 + .../apache/hadoop/hbase/executor/EventType.java | 26 +- .../apache/hadoop/hbase/executor/ExecutorType.java | 3 +- .../org/apache/hadoop/hbase/master/HMaster.java | 174 ++-- .../hadoop/hbase/master/MasterRpcServices.java | 42 +- .../apache/hadoop/hbase/master/MasterServices.java | 26 +- .../assignment/RegionTransitionProcedure.java | 29 +- .../master/cleaner/ReplicationZKNodeCleaner.java | 206 ---- .../cleaner/ReplicationZKNodeCleanerChore.java | 55 - .../hbase/master/procedure/MasterProcedureEnv.java | 23 +- .../master/procedure/MasterProcedureScheduler.java | 262 +++-- .../master/procedure/PeerProcedureInterface.java | 34 + .../master/procedure/ProcedurePrepareLatch.java | 2 +- .../master/procedure/RSProcedureDispatcher.java | 94 +- .../hbase/master/replication/AddPeerProcedure.java | 98 ++ .../master/replication/DisablePeerProcedure.java | 70 ++ .../master/replication/EnablePeerProcedure.java | 70 ++ .../master/replication/ModifyPeerProcedure.java | 214 ++++ .../master/replication/RefreshPeerProcedure.java | 204 ++++ .../master/replication/RemovePeerProcedure.java | 72 ++ .../master/replication/ReplicationManager.java | 203 ---- .../master/replication/ReplicationPeerManager.java | 344 +++++++ .../replication/UpdatePeerConfigProcedure.java | 92 ++ .../hbase/procedure2/RSProcedureCallable.java | 43 + .../hadoop/hbase/regionserver/HRegionServer.java | 54 +- .../hadoop/hbase/regionserver/RSRpcServices.java | 62 +- .../regionserver/RegionServerCoprocessorHost.java | 18 + .../RemoteProcedureResultReporter.java | 112 ++ .../regionserver/ReplicationSourceService.java | 12 +- .../regionserver/handler/RSProcedureHandler.java | 54 + .../hbase/replication/BaseReplicationEndpoint.java | 2 +- .../master/ReplicationHFileCleaner.java | 109 +- .../replication/master/ReplicationLogCleaner.java | 35 +- .../master/ReplicationPeerConfigUpgrader.java | 134 ++- .../regionserver/DumpReplicationQueues.java | 85 +- .../HBaseInterClusterReplicationEndpoint.java | 17 +- .../regionserver/PeerProcedureHandler.java | 41 + .../regionserver/PeerProcedureHandlerImpl.java | 110 ++ .../regionserver/RecoveredReplicationSource.java | 46 +- .../RecoveredReplicationSourceShipper.java | 30 +- .../regionserver/RefreshPeerCallable.java | 92 ++ .../replication/regionserver/Replication.java | 65 +- .../regionserver/ReplicationSource.java | 292 +++--- .../regionserver/ReplicationSourceFactory.java | 4 +- .../regionserver/ReplicationSourceInterface.java | 23 +- .../regionserver/ReplicationSourceManager.java | 1080 ++++++++++---------- .../regionserver/ReplicationSourceShipper.java | 6 +- .../regionserver/ReplicationSourceWALReader.java | 2 +- .../regionserver/ReplicationSyncUp.java | 29 +- .../hbase/security/access/AccessController.java | 24 +- .../org/apache/hadoop/hbase/util/HBaseFsck.java | 16 +- .../hadoop/hbase/util/hbck/ReplicationChecker.java | 119 ++- .../hadoop/hbase/TestJMXConnectorServer.java | 7 + .../hbase/client/TestAsyncReplicationAdminApi.java | 29 +- .../client/replication/TestReplicationAdmin.java | 142 ++- .../hbase/master/MockNoopMasterServices.java | 31 +- .../hadoop/hbase/master/TestMasterNoCluster.java | 4 +- .../master/assignment/TestAssignmentManager.java | 20 +- .../hbase/master/cleaner/TestLogsCleaner.java | 42 +- .../cleaner/TestReplicationHFileCleaner.java | 92 +- .../cleaner/TestReplicationZKNodeCleaner.java | 115 --- .../procedure/TestMasterProcedureScheduler.java | 65 +- .../TestMasterProcedureSchedulerConcurrency.java | 135 +++ .../hbase/replication/ReplicationSourceDummy.java | 13 +- .../replication/TestMultiSlaveReplication.java | 2 - .../replication/TestNamespaceReplication.java | 57 +- .../TestReplicationDisableInactivePeer.java | 6 +- .../replication/TestReplicationProcedureRetry.java | 200 ++++ .../hbase/replication/TestReplicationSource.java | 27 +- .../replication/TestReplicationStateBasic.java | 378 ------- .../replication/TestReplicationStateZKImpl.java | 227 ---- .../replication/TestReplicationSyncUpTool.java | 6 +- .../replication/TestReplicationTrackerZKImpl.java | 90 +- .../replication/master/TestTableCFsUpdater.java | 41 +- .../regionserver/TestReplicationSourceManager.java | 160 ++- .../TestReplicationSourceManagerZkImpl.java | 141 +-- .../security/access/TestAccessController.java | 17 +- .../hbase/util/TestHBaseFsckReplication.java | 101 ++ .../hadoop/hbase/util/hbck/HbckTestingUtil.java | 6 +- hbase-shell/src/test/ruby/shell/list_locks_test.rb | 19 + .../apache/hadoop/hbase/HBaseZKTestingUtility.java | 3 +- 126 files changed, 6853 insertions(+), 5866 deletions(-) create mode 100644 hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerImpl.java create mode 100644 hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerStorage.java delete mode 100644 hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerZKImpl.java delete mode 100644 hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java create mode 100644 hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueStorage.java delete mode 100644 hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java delete mode 100644 hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesArguments.java delete mode 100644 hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClient.java delete mode 100644 hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientArguments.java delete mode 100644 hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java delete mode 100644 hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java delete mode 100644 hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java create mode 100644 hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStorageFactory.java delete mode 100644 hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationTableBase.java create mode 100644 hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationUtils.java create mode 100644 hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationPeerStorage.java create mode 100644 hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java create mode 100644 hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationStorageBase.java create mode 100644 hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java create mode 100644 hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java create mode 100644 hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationPeerStorage.java create mode 100644 hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationQueueStorage.java delete mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKNodeCleaner.java delete mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKNodeCleanerChore.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/PeerProcedureInterface.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AddPeerProcedure.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/DisablePeerProcedure.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/EnablePeerProcedure.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerProcedure.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RemovePeerProcedure.java delete mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationManager.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/UpdatePeerConfigProcedure.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/procedure2/RSProcedureCallable.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RemoteProcedureResultReporter.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/RSProcedureHandler.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandler.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandlerImpl.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RefreshPeerCallable.java delete mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationZKNodeCleaner.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationProcedureRetry.java delete mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java delete mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckReplication.java diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java index f61b32e..6729473 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java @@ -2466,7 +2466,7 @@ public interface Admin extends Abortable, Closeable { /** * Add a new replication peer for replicating data to slave cluster. * @param peerId a short name that identifies the peer - * @param peerConfig configuration for the replication slave cluster + * @param peerConfig configuration for the replication peer * @throws IOException if a remote or network exception occurs */ default void addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig) @@ -2477,7 +2477,7 @@ public interface Admin extends Abortable, Closeable { /** * Add a new replication peer for replicating data to slave cluster. * @param peerId a short name that identifies the peer - * @param peerConfig configuration for the replication slave cluster + * @param peerConfig configuration for the replication peer * @param enabled peer state, true if ENABLED and false if DISABLED * @throws IOException if a remote or network exception occurs */ @@ -2485,6 +2485,37 @@ public interface Admin extends Abortable, Closeable { throws IOException; /** + * Add a new replication peer but does not block and wait for it. + *

+ * You can use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw + * ExecutionException if there was an error while executing the operation or TimeoutException in + * case the wait timeout was not long enough to allow the operation to complete. + * @param peerId a short name that identifies the peer + * @param peerConfig configuration for the replication peer + * @return the result of the async operation + * @throws IOException IOException if a remote or network exception occurs + */ + default Future addReplicationPeerAsync(String peerId, ReplicationPeerConfig peerConfig) + throws IOException { + return addReplicationPeerAsync(peerId, peerConfig, true); + } + + /** + * Add a new replication peer but does not block and wait for it. + *

+ * You can use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw + * ExecutionException if there was an error while executing the operation or TimeoutException in + * case the wait timeout was not long enough to allow the operation to complete. + * @param peerId a short name that identifies the peer + * @param peerConfig configuration for the replication peer + * @param enabled peer state, true if ENABLED and false if DISABLED + * @return the result of the async operation + * @throws IOException IOException if a remote or network exception occurs + */ + Future addReplicationPeerAsync(String peerId, ReplicationPeerConfig peerConfig, + boolean enabled) throws IOException; + + /** * Remove a peer and stop the replication. * @param peerId a short name that identifies the peer * @throws IOException if a remote or network exception occurs @@ -2492,6 +2523,18 @@ public interface Admin extends Abortable, Closeable { void removeReplicationPeer(String peerId) throws IOException; /** + * Remove a replication peer but does not block and wait for it. + *

+ * You can use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw + * ExecutionException if there was an error while executing the operation or TimeoutException in + * case the wait timeout was not long enough to allow the operation to complete. + * @param peerId a short name that identifies the peer + * @return the result of the async operation + * @throws IOException IOException if a remote or network exception occurs + */ + Future removeReplicationPeerAsync(String peerId) throws IOException; + + /** * Restart the replication stream to the specified peer. * @param peerId a short name that identifies the peer * @throws IOException if a remote or network exception occurs @@ -2499,6 +2542,18 @@ public interface Admin extends Abortable, Closeable { void enableReplicationPeer(String peerId) throws IOException; /** + * Enable a replication peer but does not block and wait for it. + *

+ * You can use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw + * ExecutionException if there was an error while executing the operation or TimeoutException in + * case the wait timeout was not long enough to allow the operation to complete. + * @param peerId a short name that identifies the peer + * @return the result of the async operation + * @throws IOException IOException if a remote or network exception occurs + */ + Future enableReplicationPeerAsync(String peerId) throws IOException; + + /** * Stop the replication stream to the specified peer. * @param peerId a short name that identifies the peer * @throws IOException if a remote or network exception occurs @@ -2506,6 +2561,18 @@ public interface Admin extends Abortable, Closeable { void disableReplicationPeer(String peerId) throws IOException; /** + * Disable a replication peer but does not block and wait for it. + *

+ * You can use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw + * ExecutionException if there was an error while executing the operation or TimeoutException in + * case the wait timeout was not long enough to allow the operation to complete. + * @param peerId a short name that identifies the peer + * @return the result of the async operation + * @throws IOException IOException if a remote or network exception occurs + */ + Future disableReplicationPeerAsync(String peerId) throws IOException; + + /** * Returns the configured ReplicationPeerConfig for the specified peer. * @param peerId a short name that identifies the peer * @return ReplicationPeerConfig for the peer @@ -2516,13 +2583,27 @@ public interface Admin extends Abortable, Closeable { /** * Update the peerConfig for the specified peer. * @param peerId a short name that identifies the peer - * @param peerConfig new config for the peer + * @param peerConfig new config for the replication peer * @throws IOException if a remote or network exception occurs */ void updateReplicationPeerConfig(String peerId, ReplicationPeerConfig peerConfig) throws IOException; /** + * Update the peerConfig for the specified peer but does not block and wait for it. + *

+ * You can use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw + * ExecutionException if there was an error while executing the operation or TimeoutException in + * case the wait timeout was not long enough to allow the operation to complete. + * @param peerId a short name that identifies the peer + * @param peerConfig new config for the replication peer + * @return the result of the async operation + * @throws IOException IOException if a remote or network exception occurs + */ + Future updateReplicationPeerConfigAsync(String peerId, ReplicationPeerConfig peerConfig) + throws IOException; + + /** * Append the replicable table column family config from the specified peer. * @param id a short that identifies the cluster * @param tableCfs A map from tableName to column family names diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java index 4ac1c21..1a87b48 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -39,6 +39,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Supplier; import java.util.regex.Pattern; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -200,7 +201,12 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTa import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.DisableReplicationPeerResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.EnableReplicationPeerResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetReplicationPeerConfigResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos; /** @@ -3772,6 +3778,25 @@ public class HBaseAdmin implements Admin { } } + @InterfaceAudience.Private + @InterfaceStability.Evolving + private static class ReplicationFuture extends ProcedureFuture { + private final String peerId; + private final Supplier getOperation; + + public ReplicationFuture(HBaseAdmin admin, String peerId, Long procId, + Supplier getOperation) { + super(admin, procId); + this.peerId = peerId; + this.getOperation = getOperation; + } + + @Override + public String toString() { + return "Operation: " + getOperation.get() + ", peerId: " + peerId; + } + } + @Override public List getSecurityCapabilities() throws IOException { try { @@ -3844,50 +3869,82 @@ public class HBaseAdmin implements Admin { @Override public void addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled) throws IOException { - executeCallable(new MasterCallable(getConnection(), getRpcControllerFactory()) { - @Override - protected Void rpcCall() throws Exception { - master.addReplicationPeer(getRpcController(), - RequestConverter.buildAddReplicationPeerRequest(peerId, peerConfig, enabled)); - return null; - } - }); + get(addReplicationPeerAsync(peerId, peerConfig, enabled), this.syncWaitTimeout, + TimeUnit.MILLISECONDS); + } + + @Override + public Future addReplicationPeerAsync(String peerId, ReplicationPeerConfig peerConfig, + boolean enabled) throws IOException { + AddReplicationPeerResponse response = executeCallable( + new MasterCallable(getConnection(), getRpcControllerFactory()) { + @Override + protected AddReplicationPeerResponse rpcCall() throws Exception { + return master.addReplicationPeer(getRpcController(), + RequestConverter.buildAddReplicationPeerRequest(peerId, peerConfig, enabled)); + } + }); + return new ReplicationFuture(this, peerId, response.getProcId(), () -> "ADD_REPLICATION_PEER"); } @Override public void removeReplicationPeer(String peerId) throws IOException { - executeCallable(new MasterCallable(getConnection(), getRpcControllerFactory()) { - @Override - protected Void rpcCall() throws Exception { - master.removeReplicationPeer(getRpcController(), - RequestConverter.buildRemoveReplicationPeerRequest(peerId)); - return null; - } - }); + get(removeReplicationPeerAsync(peerId), this.syncWaitTimeout, TimeUnit.MILLISECONDS); + } + + @Override + public Future removeReplicationPeerAsync(String peerId) throws IOException { + RemoveReplicationPeerResponse response = + executeCallable(new MasterCallable(getConnection(), + getRpcControllerFactory()) { + @Override + protected RemoveReplicationPeerResponse rpcCall() throws Exception { + return master.removeReplicationPeer(getRpcController(), + RequestConverter.buildRemoveReplicationPeerRequest(peerId)); + } + }); + return new ReplicationFuture(this, peerId, response.getProcId(), + () -> "REMOVE_REPLICATION_PEER"); } @Override public void enableReplicationPeer(final String peerId) throws IOException { - executeCallable(new MasterCallable(getConnection(), getRpcControllerFactory()) { - @Override - protected Void rpcCall() throws Exception { - master.enableReplicationPeer(getRpcController(), - RequestConverter.buildEnableReplicationPeerRequest(peerId)); - return null; - } - }); + get(enableReplicationPeerAsync(peerId), this.syncWaitTimeout, TimeUnit.MILLISECONDS); + } + + @Override + public Future enableReplicationPeerAsync(final String peerId) throws IOException { + EnableReplicationPeerResponse response = + executeCallable(new MasterCallable(getConnection(), + getRpcControllerFactory()) { + @Override + protected EnableReplicationPeerResponse rpcCall() throws Exception { + return master.enableReplicationPeer(getRpcController(), + RequestConverter.buildEnableReplicationPeerRequest(peerId)); + } + }); + return new ReplicationFuture(this, peerId, response.getProcId(), + () -> "ENABLE_REPLICATION_PEER"); } @Override public void disableReplicationPeer(final String peerId) throws IOException { - executeCallable(new MasterCallable(getConnection(), getRpcControllerFactory()) { - @Override - protected Void rpcCall() throws Exception { - master.disableReplicationPeer(getRpcController(), - RequestConverter.buildDisableReplicationPeerRequest(peerId)); - return null; - } - }); + get(disableReplicationPeerAsync(peerId), this.syncWaitTimeout, TimeUnit.MILLISECONDS); + } + + @Override + public Future disableReplicationPeerAsync(final String peerId) throws IOException { + DisableReplicationPeerResponse response = + executeCallable(new MasterCallable(getConnection(), + getRpcControllerFactory()) { + @Override + protected DisableReplicationPeerResponse rpcCall() throws Exception { + return master.disableReplicationPeer(getRpcController(), + RequestConverter.buildDisableReplicationPeerRequest(peerId)); + } + }); + return new ReplicationFuture(this, peerId, response.getProcId(), + () -> "DISABLE_REPLICATION_PEER"); } @Override @@ -3906,14 +3963,24 @@ public class HBaseAdmin implements Admin { @Override public void updateReplicationPeerConfig(final String peerId, final ReplicationPeerConfig peerConfig) throws IOException { - executeCallable(new MasterCallable(getConnection(), getRpcControllerFactory()) { - @Override - protected Void rpcCall() throws Exception { - master.updateReplicationPeerConfig(getRpcController(), - RequestConverter.buildUpdateReplicationPeerConfigRequest(peerId, peerConfig)); - return null; - } - }); + get(updateReplicationPeerConfigAsync(peerId, peerConfig), this.syncWaitTimeout, + TimeUnit.MILLISECONDS); + } + + @Override + public Future updateReplicationPeerConfigAsync(final String peerId, + final ReplicationPeerConfig peerConfig) throws IOException { + UpdateReplicationPeerConfigResponse response = + executeCallable(new MasterCallable(getConnection(), + getRpcControllerFactory()) { + @Override + protected UpdateReplicationPeerConfigResponse rpcCall() throws Exception { + return master.updateReplicationPeerConfig(getRpcController(), + RequestConverter.buildUpdateReplicationPeerConfigRequest(peerId, peerConfig)); + } + }); + return new ReplicationFuture(this, peerId, response.getProcId(), + () -> "UPDATE_REPLICATION_PEER_CONFIG"); } @Override diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java index ac00234..4b94687 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java @@ -36,6 +36,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; import java.util.function.BiConsumer; import java.util.function.Function; +import java.util.function.Supplier; import java.util.regex.Pattern; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -1525,47 +1526,34 @@ class RawAsyncHBaseAdmin implements AsyncAdmin { @Override public CompletableFuture addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled) { - return this - . newMasterCaller() - .action( - (controller, stub) -> this - . call(controller, stub, - RequestConverter.buildAddReplicationPeerRequest(peerId, peerConfig, enabled), (s, - c, req, done) -> s.addReplicationPeer(c, req, done), (resp) -> null)).call(); + return this. procedureCall( + RequestConverter.buildAddReplicationPeerRequest(peerId, peerConfig, enabled), + (s, c, req, done) -> s.addReplicationPeer(c, req, done), (resp) -> resp.getProcId(), + new ReplicationProcedureBiConsumer(peerId, () -> "ADD_REPLICATION_PEER")); } @Override public CompletableFuture removeReplicationPeer(String peerId) { - return this - . newMasterCaller() - .action( - (controller, stub) -> this - . call(controller, - stub, RequestConverter.buildRemoveReplicationPeerRequest(peerId), - (s, c, req, done) -> s.removeReplicationPeer(c, req, done), (resp) -> null)).call(); + return this. procedureCall( + RequestConverter.buildRemoveReplicationPeerRequest(peerId), + (s, c, req, done) -> s.removeReplicationPeer(c, req, done), (resp) -> resp.getProcId(), + new ReplicationProcedureBiConsumer(peerId, () -> "REMOVE_REPLICATION_PEER")); } @Override public CompletableFuture enableReplicationPeer(String peerId) { - return this - . newMasterCaller() - .action( - (controller, stub) -> this - . call(controller, - stub, RequestConverter.buildEnableReplicationPeerRequest(peerId), - (s, c, req, done) -> s.enableReplicationPeer(c, req, done), (resp) -> null)).call(); + return this. procedureCall( + RequestConverter.buildEnableReplicationPeerRequest(peerId), + (s, c, req, done) -> s.enableReplicationPeer(c, req, done), (resp) -> resp.getProcId(), + new ReplicationProcedureBiConsumer(peerId, () -> "ENABLE_REPLICATION_PEER")); } @Override public CompletableFuture disableReplicationPeer(String peerId) { - return this - . newMasterCaller() - .action( - (controller, stub) -> this - . call( - controller, stub, RequestConverter.buildDisableReplicationPeerRequest(peerId), (s, - c, req, done) -> s.disableReplicationPeer(c, req, done), (resp) -> null)) - .call(); + return this. procedureCall( + RequestConverter.buildDisableReplicationPeerRequest(peerId), + (s, c, req, done) -> s.disableReplicationPeer(c, req, done), (resp) -> resp.getProcId(), + new ReplicationProcedureBiConsumer(peerId, () -> "DISABLE_REPLICATION_PEER")); } @Override @@ -1584,13 +1572,11 @@ class RawAsyncHBaseAdmin implements AsyncAdmin { public CompletableFuture updateReplicationPeerConfig(String peerId, ReplicationPeerConfig peerConfig) { return this - . newMasterCaller() - .action( - (controller, stub) -> this - . call( - controller, stub, RequestConverter.buildUpdateReplicationPeerConfigRequest(peerId, - peerConfig), (s, c, req, done) -> s.updateReplicationPeerConfig(c, req, done), ( - resp) -> null)).call(); + . procedureCall( + RequestConverter.buildUpdateReplicationPeerConfigRequest(peerId, peerConfig), + (s, c, req, done) -> s.updateReplicationPeerConfig(c, req, done), + (resp) -> resp.getProcId(), + new ReplicationProcedureBiConsumer(peerId, () -> "UPDATE_REPLICATION_PEER_CONFIG")); } @Override @@ -2549,6 +2535,30 @@ class RawAsyncHBaseAdmin implements AsyncAdmin { } } + private class ReplicationProcedureBiConsumer extends ProcedureBiConsumer { + private final String peerId; + private final Supplier getOperation; + + ReplicationProcedureBiConsumer(String peerId, Supplier getOperation) { + this.peerId = peerId; + this.getOperation = getOperation; + } + + String getDescription() { + return "Operation: " + getOperation.get() + ", peerId: " + peerId; + } + + @Override + void onFinished() { + LOG.info(getDescription() + " completed"); + } + + @Override + void onError(Throwable error) { + LOG.info(getDescription() + " failed with " + error.getMessage()); + } + } + private CompletableFuture waitProcedureResult(CompletableFuture procFuture) { CompletableFuture future = new CompletableFuture<>(); procFuture.whenComplete((procId, error) -> { diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationPeerConfigUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationPeerConfigUtil.java index 022bf64..a234a9b 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationPeerConfigUtil.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationPeerConfigUtil.java @@ -247,22 +247,22 @@ public final class ReplicationPeerConfigUtil { public static ReplicationPeerConfig parsePeerFrom(final byte[] bytes) throws DeserializationException { if (ProtobufUtil.isPBMagicPrefix(bytes)) { - int pblen = ProtobufUtil.lengthOfPBMagic(); + int pbLen = ProtobufUtil.lengthOfPBMagic(); ReplicationProtos.ReplicationPeer.Builder builder = ReplicationProtos.ReplicationPeer.newBuilder(); ReplicationProtos.ReplicationPeer peer; try { - ProtobufUtil.mergeFrom(builder, bytes, pblen, bytes.length - pblen); + ProtobufUtil.mergeFrom(builder, bytes, pbLen, bytes.length - pbLen); peer = builder.build(); } catch (IOException e) { throw new DeserializationException(e); } return convert(peer); } else { - if (bytes.length > 0) { - return ReplicationPeerConfig.newBuilder().setClusterKey(Bytes.toString(bytes)).build(); + if (bytes == null || bytes.length <= 0) { + throw new DeserializationException("Bytes to deserialize should not be empty."); } - return ReplicationPeerConfig.newBuilder().setClusterKey("").build(); + return ReplicationPeerConfig.newBuilder().setClusterKey(Bytes.toString(bytes)).build(); } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java index d549607..2f2dc86 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java @@ -17,6 +17,8 @@ */ package org.apache.hadoop.hbase.protobuf; +import static org.apache.hadoop.hbase.protobuf.ProtobufMagic.PB_MAGIC; + import com.google.protobuf.ByteString; import com.google.protobuf.CodedInputStream; import com.google.protobuf.InvalidProtocolBufferException; @@ -199,7 +201,7 @@ public final class ProtobufUtil { * byte array that is bytes.length plus {@link ProtobufMagic#PB_MAGIC}.length. */ public static byte [] prependPBMagic(final byte [] bytes) { - return Bytes.add(ProtobufMagic.PB_MAGIC, bytes); + return Bytes.add(PB_MAGIC, bytes); } /** @@ -224,10 +226,11 @@ public final class ProtobufUtil { * @param bytes bytes to check * @throws DeserializationException if we are missing the pb magic prefix */ - public static void expectPBMagicPrefix(final byte [] bytes) throws DeserializationException { + public static void expectPBMagicPrefix(final byte[] bytes) throws DeserializationException { if (!isPBMagicPrefix(bytes)) { - throw new DeserializationException("Missing pb magic " + - Bytes.toString(ProtobufMagic.PB_MAGIC) + " prefix"); + String bytesPrefix = bytes == null ? "null" : Bytes.toStringBinary(bytes, 0, PB_MAGIC.length); + throw new DeserializationException( + "Missing pb magic " + Bytes.toString(PB_MAGIC) + " prefix, bytes: " + bytesPrefix); } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfig.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfig.java index ab75dff..d52354d 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfig.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfig.java @@ -25,10 +25,9 @@ import java.util.List; import java.util.Map; 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. diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java index f90d9dd..06d9a3c 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java @@ -17,6 +17,8 @@ */ package org.apache.hadoop.hbase.shaded.protobuf; +import static org.apache.hadoop.hbase.protobuf.ProtobufMagic.PB_MAGIC; + import java.io.ByteArrayOutputStream; import java.io.IOException; import java.io.InputStream; @@ -279,7 +281,7 @@ public final class ProtobufUtil { * byte array that is bytes.length plus {@link ProtobufMagic#PB_MAGIC}.length. */ public static byte [] prependPBMagic(final byte [] bytes) { - return Bytes.add(ProtobufMagic.PB_MAGIC, bytes); + return Bytes.add(PB_MAGIC, bytes); } /** @@ -304,10 +306,11 @@ public final class ProtobufUtil { * @param bytes bytes to check * @throws DeserializationException if we are missing the pb magic prefix */ - public static void expectPBMagicPrefix(final byte [] bytes) throws DeserializationException { + public static void expectPBMagicPrefix(final byte[] bytes) throws DeserializationException { if (!isPBMagicPrefix(bytes)) { - throw new DeserializationException("Missing pb magic " + - Bytes.toString(ProtobufMagic.PB_MAGIC) + " prefix"); + String bytesPrefix = bytes == null ? "null" : Bytes.toStringBinary(bytes, 0, PB_MAGIC.length); + throw new DeserializationException( + "Missing pb magic " + Bytes.toString(PB_MAGIC) + " prefix" + ", bytes: " + bytesPrefix); } } @@ -1945,7 +1948,7 @@ public final class ProtobufUtil { public static byte [] toDelimitedByteArray(final Message m) throws IOException { // Allocate arbitrary big size so we avoid resizing. ByteArrayOutputStream baos = new ByteArrayOutputStream(4096); - baos.write(ProtobufMagic.PB_MAGIC); + baos.write(PB_MAGIC); m.writeDelimitedTo(baos); return baos.toByteArray(); } diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/CollectionUtils.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/CollectionUtils.java index 875b124..8bbb6f1 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/CollectionUtils.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/CollectionUtils.java @@ -107,6 +107,9 @@ public class CollectionUtils { return list.get(list.size() - 1); } + public static List nullToEmpty(List list) { + return list != null ? list : Collections.emptyList(); + } /** * In HBASE-16648 we found that ConcurrentHashMap.get is much faster than computeIfAbsent if the * value already exists. Notice that the implementation does not guarantee that the supplier will diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java index 9065f4e..fac4875 100644 --- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java +++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java @@ -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,14 +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.ReplicationPeerZKImpl; -import org.apache.hadoop.hbase.replication.ReplicationPeers; +import org.apache.hadoop.hbase.replication.ReplicationPeerStorage; +import org.apache.hadoop.hbase.replication.ReplicationStorageFactory; +import org.apache.hadoop.hbase.replication.ReplicationUtils; import org.apache.hadoop.hbase.snapshot.RestoreSnapshotHelper; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSUtils; @@ -67,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,30 +333,26 @@ public class VerifyReplication extends Configured implements Tool { private static Pair getPeerQuorumConfig( final Configuration conf, String peerId) throws IOException { ZKWatcher localZKW = null; - ReplicationPeerZKImpl peer = null; try { - localZKW = new ZKWatcher(conf, "VerifyReplication", - new Abortable() { - @Override public void abort(String why, Throwable e) {} - @Override public boolean isAborted() {return false;} - }); - - ReplicationPeers rp = ReplicationFactory.getReplicationPeers(localZKW, conf, localZKW); - rp.init(); - - Pair pair = rp.getPeerConf(peerId); - if (pair == null) { - throw new IOException("Couldn't get peer conf!"); - } + localZKW = new ZKWatcher(conf, "VerifyReplication", new Abortable() { + @Override + public void abort(String why, Throwable e) { + } - return pair; + @Override + public boolean isAborted() { + return false; + } + }); + ReplicationPeerStorage storage = + ReplicationStorageFactory.getReplicationPeerStorage(localZKW, conf); + ReplicationPeerConfig peerConfig = storage.getPeerConfig(peerId); + return Pair.newPair(peerConfig, + ReplicationUtils.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 (peer != null) { - peer.close(); - } if (localZKW != null) { localZKW.close(); } diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/LockedResourceType.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/LockedResourceType.java index c5fe62b..dc9b5d4 100644 --- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/LockedResourceType.java +++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/LockedResourceType.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -22,5 +22,5 @@ import org.apache.yetus.audience.InterfaceAudience; @InterfaceAudience.Private public enum LockedResourceType { - SERVER, NAMESPACE, TABLE, REGION + SERVER, NAMESPACE, TABLE, REGION, PEER } diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RemoteProcedureDispatcher.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RemoteProcedureDispatcher.java index 8bbfcec..bdff1ca 100644 --- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RemoteProcedureDispatcher.java +++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RemoteProcedureDispatcher.java @@ -226,13 +226,29 @@ public abstract class RemoteProcedureDispatcher - * @param */ public interface RemoteProcedure { + /** + * For building the remote operation. + */ RemoteOperation remoteCallBuild(TEnv env, TRemote remote); - void remoteCallCompleted(TEnv env, TRemote remote, RemoteOperation response); + + /** + * Called when the executeProcedure call is failed. + */ void remoteCallFailed(TEnv env, TRemote remote, IOException exception); + + /** + * Called when RS tells the remote procedure is succeeded through the + * {@code reportProcedureDone} method. + */ + void remoteOperationCompleted(TEnv env); + + /** + * Called when RS tells the remote procedure is failed through the {@code reportProcedureDone} + * method. + */ + void remoteOperationFailed(TEnv env, RemoteProcedureException error); } /** diff --git a/hbase-protocol-shaded/src/main/protobuf/Admin.proto b/hbase-protocol-shaded/src/main/protobuf/Admin.proto index 118c79b..ddcc266 100644 --- a/hbase-protocol-shaded/src/main/protobuf/Admin.proto +++ b/hbase-protocol-shaded/src/main/protobuf/Admin.proto @@ -256,14 +256,19 @@ message ClearRegionBlockCacheResponse { required CacheEvictionStats stats = 1; } +message RemoteProcedureRequest { + required uint64 proc_id = 1; + required string proc_class = 2; + optional bytes proc_data = 3; +} + message ExecuteProceduresRequest { repeated OpenRegionRequest open_region = 1; repeated CloseRegionRequest close_region = 2; + repeated RemoteProcedureRequest proc = 3; } message ExecuteProceduresResponse { - repeated OpenRegionResponse open_region = 1; - repeated CloseRegionResponse close_region = 2; } service AdminService { diff --git a/hbase-protocol-shaded/src/main/protobuf/LockService.proto b/hbase-protocol-shaded/src/main/protobuf/LockService.proto index b8d180c..0675070 100644 --- a/hbase-protocol-shaded/src/main/protobuf/LockService.proto +++ b/hbase-protocol-shaded/src/main/protobuf/LockService.proto @@ -77,6 +77,7 @@ enum LockedResourceType { NAMESPACE = 2; TABLE = 3; REGION = 4; + PEER = 5; } message LockedResource { diff --git a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto index f9b8807..ae676ea 100644 --- a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto +++ b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto @@ -27,6 +27,7 @@ option optimize_for = SPEED; import "HBase.proto"; import "RPC.proto"; import "Snapshot.proto"; +import "Replication.proto"; // ============================================================================ // WARNING - Compatibility rules @@ -365,3 +366,47 @@ message GCMergedRegionsStateData { required RegionInfo parent_b = 2; required RegionInfo merged_child = 3; } + +enum PeerModificationState { + PRE_PEER_MODIFICATION = 1; + UPDATE_PEER_STORAGE = 2; + REFRESH_PEER_ON_RS = 3; + POST_PEER_MODIFICATION = 4; +} + +message PeerModificationStateData { + required string peer_id = 1; +} + +enum PeerModificationType { + ADD_PEER = 1; + REMOVE_PEER = 2; + ENABLE_PEER = 3; + DISABLE_PEER = 4; + UPDATE_PEER_CONFIG = 5; +} + +message RefreshPeerStateData { + required string peer_id = 1; + required PeerModificationType type = 2; + required ServerName target_server = 3; +} + +message RefreshPeerParameter { + required string peer_id = 1; + required PeerModificationType type = 2; + required ServerName target_server = 3; +} + +message ModifyPeerStateData { + required string peer_id = 1; +} + +message AddPeerStateData { + required ReplicationPeer peer_config = 1; + required bool enabled = 2; +} + +message UpdatePeerConfigStateData { + required ReplicationPeer peer_config = 1; +} \ No newline at end of file diff --git a/hbase-protocol-shaded/src/main/protobuf/RegionServerStatus.proto b/hbase-protocol-shaded/src/main/protobuf/RegionServerStatus.proto index f83bb20..3f836cd 100644 --- a/hbase-protocol-shaded/src/main/protobuf/RegionServerStatus.proto +++ b/hbase-protocol-shaded/src/main/protobuf/RegionServerStatus.proto @@ -28,6 +28,7 @@ option optimize_for = SPEED; import "HBase.proto"; import "ClusterStatus.proto"; +import "ErrorHandling.proto"; message RegionServerStartupRequest { /** Port number this regionserver is up on */ @@ -143,7 +144,22 @@ message RegionSpaceUseReportRequest { } message RegionSpaceUseReportResponse { +} + +message RemoteProcedureResult { + required uint64 proc_id = 1; + enum Status { + SUCCESS = 1; + ERROR = 2; + } + required Status status = 2; + optional ForeignExceptionMessage error = 3; +} +message ReportProcedureDoneRequest { + repeated RemoteProcedureResult result = 1; +} +message ReportProcedureDoneResponse { } service RegionServerStatusService { @@ -181,4 +197,7 @@ service RegionServerStatusService { */ rpc ReportRegionSpaceUse(RegionSpaceUseReportRequest) returns(RegionSpaceUseReportResponse); + + rpc ReportProcedureDone(ReportProcedureDoneRequest) + returns(ReportProcedureDoneResponse); } diff --git a/hbase-protocol-shaded/src/main/protobuf/Replication.proto b/hbase-protocol-shaded/src/main/protobuf/Replication.proto index 8657c25..9f7b4c2 100644 --- a/hbase-protocol-shaded/src/main/protobuf/Replication.proto +++ b/hbase-protocol-shaded/src/main/protobuf/Replication.proto @@ -84,6 +84,7 @@ message AddReplicationPeerRequest { } message AddReplicationPeerResponse { + optional uint64 proc_id = 1; } message RemoveReplicationPeerRequest { @@ -91,6 +92,7 @@ message RemoveReplicationPeerRequest { } message RemoveReplicationPeerResponse { + optional uint64 proc_id = 1; } message EnableReplicationPeerRequest { @@ -98,6 +100,7 @@ message EnableReplicationPeerRequest { } message EnableReplicationPeerResponse { + optional uint64 proc_id = 1; } message DisableReplicationPeerRequest { @@ -105,6 +108,7 @@ message DisableReplicationPeerRequest { } message DisableReplicationPeerResponse { + optional uint64 proc_id = 1; } message GetReplicationPeerConfigRequest { @@ -122,6 +126,7 @@ message UpdateReplicationPeerConfigRequest { } message UpdateReplicationPeerConfigResponse { + optional uint64 proc_id = 1; } message ListReplicationPeersRequest { diff --git a/hbase-replication/pom.xml b/hbase-replication/pom.xml index cd4f74d..f44db12 100644 --- a/hbase-replication/pom.xml +++ b/hbase-replication/pom.xml @@ -137,6 +137,18 @@ org.apache.hbase hbase-zookeeper + + org.apache.hbase + hbase-common + test-jar + test + + + org.apache.hbase + hbase-zookeeper + test-jar + test + org.apache.commons diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java index 6c83d6a..2f17e11 100644 --- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java +++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -18,7 +17,6 @@ */ package org.apache.hadoop.hbase.replication; -import org.apache.commons.lang3.reflect.ConstructorUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Abortable; import org.apache.hadoop.hbase.Stoppable; @@ -33,31 +31,12 @@ public final class ReplicationFactory { private ReplicationFactory() { } - public static ReplicationQueues getReplicationQueues(ReplicationQueuesArguments args) - throws Exception { - return (ReplicationQueues) ConstructorUtils.invokeConstructor(ReplicationQueuesZKImpl.class, - args); - } - - public static ReplicationQueuesClient - getReplicationQueuesClient(ReplicationQueuesClientArguments args) throws Exception { - return (ReplicationQueuesClient) ConstructorUtils - .invokeConstructor(ReplicationQueuesClientZKImpl.class, args); - } - - public static ReplicationPeers getReplicationPeers(final ZKWatcher zk, Configuration conf, - Abortable abortable) { - return getReplicationPeers(zk, conf, null, abortable); - } - - public static ReplicationPeers getReplicationPeers(final ZKWatcher zk, Configuration conf, - final ReplicationQueuesClient queuesClient, Abortable abortable) { - return new ReplicationPeersZKImpl(zk, conf, queuesClient, abortable); + public static ReplicationPeers getReplicationPeers(ZKWatcher zk, Configuration conf) { + return new ReplicationPeers(zk, conf); } - public static ReplicationTracker getReplicationTracker(ZKWatcher zookeeper, - final ReplicationPeers replicationPeers, Configuration conf, Abortable abortable, + public static ReplicationTracker getReplicationTracker(ZKWatcher zookeeper, Abortable abortable, Stoppable stopper) { - return new ReplicationTrackerZKImpl(zookeeper, replicationPeers, conf, abortable, stopper); + return new ReplicationTrackerZKImpl(zookeeper, abortable, stopper); } } diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationListener.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationListener.java index 3edfcf9..f040bf9 100644 --- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationListener.java +++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationListener.java @@ -18,8 +18,6 @@ */ package org.apache.hadoop.hbase.replication; -import java.util.List; - import org.apache.yetus.audience.InterfaceAudience; /** @@ -36,16 +34,4 @@ public interface ReplicationListener { * @param regionServer the removed region server */ public void regionServerRemoved(String regionServer); - - /** - * A peer cluster has been removed (i.e. unregistered) from replication. - * @param peerId The peer id of the cluster that has been removed - */ - public void peerRemoved(String peerId); - - /** - * The list of registered peer clusters has changed. - * @param peerIds A list of all currently registered peer clusters - */ - public void peerListChanged(List peerIds); } diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeer.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeer.java index 97e2ddb..2da3cce 100644 --- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeer.java +++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeer.java @@ -26,7 +26,6 @@ import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.TableName; import org.apache.yetus.audience.InterfaceAudience; - /** * ReplicationPeer manages enabled / disabled state for the peer. */ @@ -49,41 +48,60 @@ public interface ReplicationPeer { String getId(); /** - * Get the peer config object - * @return the ReplicationPeerConfig for this peer + * Returns the state of the peer by reading local cache. + * @return the enabled state */ - public ReplicationPeerConfig getPeerConfig(); + PeerState getPeerState(); /** - * Returns the state of the peer - * @return the enabled state + * Test whether the peer is enabled. + * @return {@code true} if enabled, otherwise {@code false}. */ - PeerState getPeerState(); + default boolean isPeerEnabled() { + return getPeerState() == PeerState.ENABLED; + } + + /** + * Get the peer config object + * @return the ReplicationPeerConfig for this peer + */ + ReplicationPeerConfig getPeerConfig(); /** * Get the configuration object required to communicate with this peer * @return configuration object */ - public Configuration getConfiguration(); + Configuration getConfiguration(); /** * Get replicable (table, cf-list) map of this peer * @return the replicable (table, cf-list) map */ - public Map> getTableCFs(); + Map> getTableCFs(); /** * Get replicable namespace set of this peer * @return the replicable namespaces set */ - public Set getNamespaces(); + Set getNamespaces(); /** * Get the per node bandwidth upper limit for this peer * @return the bandwidth upper limit */ - public long getPeerBandwidth(); + long getPeerBandwidth(); - void trackPeerConfigChanges(ReplicationPeerConfigListener listener); + /** + * Register a peer config listener to catch the peer config change event. + * @param listener listener to catch the peer config change event. + */ + void registerPeerConfigListener(ReplicationPeerConfigListener listener); -} + /** + * @deprecated Use {@link #registerPeerConfigListener(ReplicationPeerConfigListener)} instead. + */ + @Deprecated + default void trackPeerConfigChanges(ReplicationPeerConfigListener listener) { + registerPeerConfigListener(listener); + } +} \ No newline at end of file diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerImpl.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerImpl.java new file mode 100644 index 0000000..604e0bb --- /dev/null +++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerImpl.java @@ -0,0 +1,125 @@ +/** + * 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; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Set; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.TableName; +import org.apache.yetus.audience.InterfaceAudience; + +@InterfaceAudience.Private +public class ReplicationPeerImpl implements ReplicationPeer { + + private final Configuration conf; + + private final String id; + + private volatile ReplicationPeerConfig peerConfig; + + private volatile PeerState peerState; + + private final List peerConfigListeners; + + /** + * Constructor that takes all the objects required to communicate with the specified peer, except + * for the region server addresses. + * @param conf configuration object to this peer + * @param id string representation of this peer's identifier + * @param peerConfig configuration for the replication peer + */ + public ReplicationPeerImpl(Configuration conf, String id, boolean peerState, + ReplicationPeerConfig peerConfig) { + this.conf = conf; + this.id = id; + this.peerState = peerState ? PeerState.ENABLED : PeerState.DISABLED; + this.peerConfig = peerConfig; + this.peerConfigListeners = new ArrayList<>(); + } + + void setPeerState(boolean enabled) { + this.peerState = enabled ? PeerState.ENABLED : PeerState.DISABLED; + } + + void setPeerConfig(ReplicationPeerConfig peerConfig) { + this.peerConfig = peerConfig; + peerConfigListeners.forEach(listener -> listener.peerConfigUpdated(peerConfig)); + } + + /** + * Get the identifier of this peer + * @return string representation of the id (short) + */ + @Override + public String getId() { + return id; + } + + @Override + public PeerState getPeerState() { + return peerState; + } + + /** + * Get the peer config object + * @return the ReplicationPeerConfig for this peer + */ + @Override + public ReplicationPeerConfig getPeerConfig() { + return peerConfig; + } + + /** + * Get the configuration object required to communicate with this peer + * @return configuration object + */ + @Override + public Configuration getConfiguration() { + return conf; + } + + /** + * Get replicable (table, cf-list) map of this peer + * @return the replicable (table, cf-list) map + */ + @Override + public Map> getTableCFs() { + return this.peerConfig.getTableCFsMap(); + } + + /** + * Get replicable namespace set of this peer + * @return the replicable namespaces set + */ + @Override + public Set getNamespaces() { + return this.peerConfig.getNamespaces(); + } + + @Override + public long getPeerBandwidth() { + return this.peerConfig.getBandwidth(); + } + + @Override + public void registerPeerConfigListener(ReplicationPeerConfigListener listener) { + this.peerConfigListeners.add(listener); + } +} diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerStorage.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerStorage.java new file mode 100644 index 0000000..1adda02 --- /dev/null +++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerStorage.java @@ -0,0 +1,73 @@ +/** + * 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; + +import java.util.List; + +import org.apache.yetus.audience.InterfaceAudience; + +/** + * Perform read/write to the replication peer storage. + */ +@InterfaceAudience.Private +public interface ReplicationPeerStorage { + + /** + * Add a replication peer. + * @throws ReplicationException if there are errors accessing the storage service. + */ + void addPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled) + throws ReplicationException; + + /** + * Remove a replication peer. + * @throws ReplicationException if there are errors accessing the storage service. + */ + void removePeer(String peerId) throws ReplicationException; + + /** + * Set the state of peer, {@code true} to {@code ENABLED}, otherwise to {@code DISABLED}. + * @throws ReplicationException if there are errors accessing the storage service. + */ + void setPeerState(String peerId, boolean enabled) throws ReplicationException; + + /** + * Update the config a replication peer. + * @throws ReplicationException if there are errors accessing the storage service. + */ + void updatePeerConfig(String peerId, ReplicationPeerConfig peerConfig) + throws ReplicationException; + + /** + * Return the peer ids of all replication peers. + * @throws ReplicationException if there are errors accessing the storage service. + */ + List listPeerIds() throws ReplicationException; + + /** + * Test whether a replication peer is enabled. + * @throws ReplicationException if there are errors accessing the storage service. + */ + boolean isPeerEnabled(String peerId) throws ReplicationException; + + /** + * Get the peer config of a replication peer. + * @throws ReplicationException if there are errors accessing the storage service. + */ + ReplicationPeerConfig getPeerConfig(String peerId) throws ReplicationException; +} diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerZKImpl.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerZKImpl.java deleted file mode 100644 index 15523e7..0000000 --- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerZKImpl.java +++ /dev/null @@ -1,314 +0,0 @@ -/* - * - * 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; - -import java.io.Closeable; -import java.io.IOException; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Set; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.Abortable; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil; -import org.apache.hadoop.hbase.exceptions.DeserializationException; -import org.apache.hadoop.hbase.log.HBaseMarkers; -import org.apache.hadoop.hbase.zookeeper.ZKNodeTracker; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; -import org.apache.hadoop.hbase.zookeeper.ZKWatcher; -import org.apache.yetus.audience.InterfaceAudience; -import org.apache.zookeeper.KeeperException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos; - -@InterfaceAudience.Private -public class ReplicationPeerZKImpl extends ReplicationStateZKBase - implements ReplicationPeer, Abortable, Closeable { - private static final Logger LOG = LoggerFactory.getLogger(ReplicationPeerZKImpl.class); - - private ReplicationPeerConfig peerConfig; - private final String id; - private volatile PeerState peerState; - private volatile Map> tableCFs = new HashMap<>(); - private final Configuration conf; - private PeerStateTracker peerStateTracker; - private PeerConfigTracker peerConfigTracker; - - - /** - * Constructor that takes all the objects required to communicate with the specified peer, except - * for the region server addresses. - * @param conf configuration object to this peer - * @param id string representation of this peer's identifier - * @param peerConfig configuration for the replication peer - */ - public ReplicationPeerZKImpl(ZKWatcher zkWatcher, Configuration conf, - String id, ReplicationPeerConfig peerConfig, - Abortable abortable) - throws ReplicationException { - super(zkWatcher, conf, abortable); - this.conf = conf; - this.peerConfig = peerConfig; - this.id = id; - } - - /** - * start a state tracker to check whether this peer is enabled or not - * - * @param peerStateNode path to zk node which stores peer state - * @throws KeeperException if creating the znode fails - */ - public void startStateTracker(String peerStateNode) throws KeeperException { - ensurePeerEnabled(peerStateNode); - this.peerStateTracker = new PeerStateTracker(peerStateNode, zookeeper, this); - this.peerStateTracker.start(); - try { - this.readPeerStateZnode(); - } catch (DeserializationException e) { - throw ZKUtil.convert(e); - } - } - - private void readPeerStateZnode() throws DeserializationException { - this.peerState = - isStateEnabled(this.peerStateTracker.getData(false)) - ? PeerState.ENABLED - : PeerState.DISABLED; - } - - /** - * start a table-cfs tracker to listen the (table, cf-list) map change - * @param peerConfigNode path to zk node which stores table-cfs - */ - public void startPeerConfigTracker(String peerConfigNode) throws KeeperException { - this.peerConfigTracker = new PeerConfigTracker(peerConfigNode, zookeeper, - this); - this.peerConfigTracker.start(); - this.readPeerConfig(); - } - - private ReplicationPeerConfig readPeerConfig() { - try { - byte[] data = peerConfigTracker.getData(false); - if (data != null) { - this.peerConfig = ReplicationPeerConfigUtil.parsePeerFrom(data); - } - } catch (DeserializationException e) { - LOG.error("", e); - } - return this.peerConfig; - } - - @Override - public PeerState getPeerState() { - return peerState; - } - - /** - * Get the identifier of this peer - * @return string representation of the id (short) - */ - @Override - public String getId() { - return id; - } - - /** - * Get the peer config object - * @return the ReplicationPeerConfig for this peer - */ - @Override - public ReplicationPeerConfig getPeerConfig() { - return peerConfig; - } - - /** - * Get the configuration object required to communicate with this peer - * @return configuration object - */ - @Override - public Configuration getConfiguration() { - return conf; - } - - /** - * Get replicable (table, cf-list) map of this peer - * @return the replicable (table, cf-list) map - */ - @Override - public Map> getTableCFs() { - this.tableCFs = peerConfig.getTableCFsMap(); - return this.tableCFs; - } - - /** - * Get replicable namespace set of this peer - * @return the replicable namespaces set - */ - @Override - public Set getNamespaces() { - return this.peerConfig.getNamespaces(); - } - - @Override - public long getPeerBandwidth() { - return this.peerConfig.getBandwidth(); - } - - @Override - public void trackPeerConfigChanges(ReplicationPeerConfigListener listener) { - if (this.peerConfigTracker != null){ - this.peerConfigTracker.setListener(listener); - } - } - - @Override - public void abort(String why, Throwable e) { - LOG.error(HBaseMarkers.FATAL, "The ReplicationPeer corresponding to peer " + - peerConfig + " was aborted for the following reason(s):" + why, e); - } - - @Override - public boolean isAborted() { - // Currently the replication peer is never "Aborted", we just log when the - // abort method is called. - return false; - } - - @Override - public void close() throws IOException { - // TODO: stop zkw? - } - - /** - * Parse the raw data from ZK to get a peer's state - * @param bytes raw ZK data - * @return True if the passed in bytes are those of a pb serialized ENABLED state. - * @throws DeserializationException if parsing the state fails - */ - public static boolean isStateEnabled(final byte[] bytes) throws DeserializationException { - ReplicationProtos.ReplicationState.State state = parseStateFrom(bytes); - return ReplicationProtos.ReplicationState.State.ENABLED == state; - } - - /** - * @param bytes Content of a state znode. - * @return State parsed from the passed bytes. - * @throws DeserializationException if a ProtoBuf operation fails - */ - private static ReplicationProtos.ReplicationState.State parseStateFrom(final byte[] bytes) - throws DeserializationException { - ProtobufUtil.expectPBMagicPrefix(bytes); - int pblen = ProtobufUtil.lengthOfPBMagic(); - ReplicationProtos.ReplicationState.Builder builder = - ReplicationProtos.ReplicationState.newBuilder(); - ReplicationProtos.ReplicationState state; - try { - ProtobufUtil.mergeFrom(builder, bytes, pblen, bytes.length - pblen); - state = builder.build(); - return state.getState(); - } catch (IOException e) { - throw new DeserializationException(e); - } - } - - /** - * Utility method to ensure an ENABLED znode is in place; if not present, we create it. - * @param path Path to znode to check - * @return True if we created the znode. - * @throws KeeperException if creating the znode fails - */ - private boolean ensurePeerEnabled(final String path) throws KeeperException { - if (ZKUtil.checkExists(zookeeper, path) == -1) { - // There is a race b/w PeerWatcher and ReplicationZookeeper#add method to create the - // peer-state znode. This happens while adding a peer. - // The peer state data is set as "ENABLED" by default. - ZKUtil.createNodeIfNotExistsAndWatch(zookeeper, path, - ReplicationStateZKBase.ENABLED_ZNODE_BYTES); - return true; - } - return false; - } - - /** - * Tracker for state of this peer - */ - public class PeerStateTracker extends ZKNodeTracker { - - public PeerStateTracker(String peerStateZNode, ZKWatcher watcher, - Abortable abortable) { - super(watcher, peerStateZNode, abortable); - } - - @Override - public synchronized void nodeDataChanged(String path) { - if (path.equals(node)) { - super.nodeDataChanged(path); - try { - readPeerStateZnode(); - } catch (DeserializationException e) { - LOG.warn("Failed deserializing the content of " + path, e); - } - } - } - } - - /** - * Tracker for PeerConfigNode of this peer - */ - public class PeerConfigTracker extends ZKNodeTracker { - - ReplicationPeerConfigListener listener; - - public PeerConfigTracker(String peerConfigNode, ZKWatcher watcher, - Abortable abortable) { - super(watcher, peerConfigNode, abortable); - } - - public synchronized void setListener(ReplicationPeerConfigListener listener){ - this.listener = listener; - } - - @Override - public synchronized void nodeCreated(String path) { - if (path.equals(node)) { - super.nodeCreated(path); - ReplicationPeerConfig config = readPeerConfig(); - if (listener != null){ - listener.peerConfigUpdated(config); - } - } - } - - @Override - public synchronized void nodeDataChanged(String path) { - //superclass calls nodeCreated - if (path.equals(node)) { - super.nodeDataChanged(path); - } - - } - - } -} diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java index 542d892..eacb2f4 100644 --- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java +++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -18,58 +17,51 @@ */ package org.apache.hadoop.hbase.replication; -import java.util.Collection; -import java.util.List; -import java.util.Map; +import java.util.Collections; import java.util.Set; - +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.util.Pair; +import org.apache.hadoop.hbase.replication.ReplicationPeer.PeerState; +import org.apache.hadoop.hbase.zookeeper.ZKWatcher; import org.apache.yetus.audience.InterfaceAudience; +import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; + /** - * This provides an interface for maintaining a set of peer clusters. These peers are remote slave - * clusters that data is replicated to. A peer cluster can be in three different states: - * - * 1. Not-Registered - There is no notion of the peer cluster. - * 2. Registered - The peer has an id and is being tracked but there is no connection. - * 3. Connected - There is an active connection to the remote peer. - * - * In the registered or connected state, a peer cluster can either be enabled or disabled. + * This provides an class for maintaining a set of peer clusters. These peers are remote slave + * clusters that data is replicated to. */ @InterfaceAudience.Private -public interface ReplicationPeers { +public class ReplicationPeers { - /** - * Initialize the ReplicationPeers interface. - */ - void init() throws ReplicationException; + private final Configuration conf; - /** - * Add a new remote slave cluster for replication. - * @param peerId a short that identifies the cluster - * @param peerConfig configuration for the replication slave cluster - */ - default void registerPeer(String peerId, ReplicationPeerConfig peerConfig) - throws ReplicationException { - registerPeer(peerId, peerConfig, true); + // Map of peer clusters keyed by their id + private final ConcurrentMap peerCache; + private final ReplicationPeerStorage peerStorage; + + ReplicationPeers(ZKWatcher zookeeper, Configuration conf) { + this.conf = conf; + this.peerCache = new ConcurrentHashMap<>(); + this.peerStorage = ReplicationStorageFactory.getReplicationPeerStorage(zookeeper, conf); } - /** - * Add a new remote slave cluster for replication. - * @param peerId a short that identifies the cluster - * @param peerConfig configuration for the replication slave cluster - * @param enabled peer state, true if ENABLED and false if DISABLED - */ - void registerPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled) - throws ReplicationException; + public Configuration getConf() { + return conf; + } - /** - * Removes a remote slave cluster and stops the replication to it. - * @param peerId a short that identifies the cluster - */ - void unregisterPeer(String peerId) throws ReplicationException; + public void init() throws ReplicationException { + // Loading all existing peerIds into peer cache. + for (String peerId : this.peerStorage.listPeerIds()) { + addPeer(peerId); + } + } + + @VisibleForTesting + public ReplicationPeerStorage getPeerStorage() { + return this.peerStorage; + } /** * Method called after a peer has been connected. It will create a ReplicationPeer to track the @@ -78,111 +70,66 @@ public interface ReplicationPeers { * @return whether a ReplicationPeer was successfully created * @throws ReplicationException if connecting to the peer fails */ - boolean peerConnected(String peerId) throws ReplicationException; - - /** - * Method called after a peer has been disconnected. It will remove the ReplicationPeer that - * tracked the disconnected cluster. - * @param peerId a short that identifies the cluster - */ - void peerDisconnected(String peerId); - - /** - * Restart the replication to the specified remote slave cluster. - * @param peerId a short that identifies the cluster - */ - void enablePeer(String peerId) throws ReplicationException; - - /** - * Stop the replication to the specified remote slave cluster. - * @param peerId a short that identifies the cluster - */ - void disablePeer(String peerId) throws ReplicationException; + public boolean addPeer(String peerId) throws ReplicationException { + if (this.peerCache.containsKey(peerId)) { + return false; + } - /** - * Get the table and column-family list string of the peer from the underlying storage. - * @param peerId a short that identifies the cluster - */ - public Map> getPeerTableCFsConfig(String peerId) - throws ReplicationException; + peerCache.put(peerId, createPeer(peerId)); + return true; + } - /** - * Set the table and column-family list string of the peer to the underlying storage. - * @param peerId a short that identifies the cluster - * @param tableCFs the table and column-family list which will be replicated for this peer - */ - public void setPeerTableCFsConfig(String peerId, - Map> tableCFs) - throws ReplicationException; + public void removePeer(String peerId) { + peerCache.remove(peerId); + } /** - * Returns the ReplicationPeer for the specified connected peer. This ReplicationPeer will - * continue to track changes to the Peer's state and config. This method returns null if no - * peer has been connected with the given peerId. + * Returns the ReplicationPeerImpl for the specified cached peer. This ReplicationPeer will + * continue to track changes to the Peer's state and config. This method returns null if no peer + * has been cached with the given peerId. * @param peerId id for the peer * @return ReplicationPeer object */ - ReplicationPeer getConnectedPeer(String peerId); + public ReplicationPeerImpl getPeer(String peerId) { + return peerCache.get(peerId); + } /** * Returns the set of peerIds of the clusters that have been connected and have an underlying * ReplicationPeer. * @return a Set of Strings for peerIds */ - public Set getConnectedPeerIds(); - - /** - * Get the replication status for the specified connected remote slave cluster. - * The value might be read from cache, so it is recommended to - * use {@link #getStatusOfPeerFromBackingStore(String)} - * if reading the state after enabling or disabling it. - * @param peerId a short that identifies the cluster - * @return true if replication is enabled, false otherwise. - */ - boolean getStatusOfPeer(String peerId); - - /** - * Get the replication status for the specified remote slave cluster, which doesn't - * have to be connected. The state is read directly from the backing store. - * @param peerId a short that identifies the cluster - * @return true if replication is enabled, false otherwise. - * @throws ReplicationException thrown if there's an error contacting the store - */ - boolean getStatusOfPeerFromBackingStore(String peerId) throws ReplicationException; - - /** - * List the cluster replication configs of all remote slave clusters (whether they are - * enabled/disabled or connected/disconnected). - * @return A map of peer ids to peer cluster keys - */ - Map getAllPeerConfigs(); - - /** - * List the peer ids of all remote slave clusters (whether they are enabled/disabled or - * connected/disconnected). - * @return A list of peer ids - */ - List getAllPeerIds(); + public Set getAllPeerIds() { + return Collections.unmodifiableSet(peerCache.keySet()); + } - /** - * Returns the configured ReplicationPeerConfig for this peerId - * @param peerId a short name that identifies the cluster - * @return ReplicationPeerConfig for the peer - */ - ReplicationPeerConfig getReplicationPeerConfig(String peerId) throws ReplicationException; + public PeerState refreshPeerState(String peerId) throws ReplicationException { + ReplicationPeerImpl peer = peerCache.get(peerId); + if (peer == null) { + throw new ReplicationException("Peer with id=" + peerId + " is not cached."); + } + peer.setPeerState(peerStorage.isPeerEnabled(peerId)); + return peer.getPeerState(); + } - /** - * Returns the configuration needed to talk to the remote slave cluster. - * @param peerId a short that identifies the cluster - * @return the configuration for the peer cluster, null if it was unable to get the configuration - */ - Pair getPeerConf(String peerId) throws ReplicationException; + public ReplicationPeerConfig refreshPeerConfig(String peerId) throws ReplicationException { + ReplicationPeerImpl peer = peerCache.get(peerId); + if (peer == null) { + throw new ReplicationException("Peer with id=" + peerId + " is not cached."); + } + peer.setPeerConfig(peerStorage.getPeerConfig(peerId)); + return peer.getPeerConfig(); + } /** - * Update the peerConfig for the a given peer cluster - * @param id a short that identifies the cluster - * @param peerConfig new config for the peer cluster - * @throws ReplicationException if updating the peer configuration fails + * Helper method to connect to a peer + * @param peerId peer's identifier + * @return object representing the peer */ - void updatePeerConfig(String id, ReplicationPeerConfig peerConfig) throws ReplicationException; + private ReplicationPeerImpl createPeer(String peerId) throws ReplicationException { + ReplicationPeerConfig peerConfig = peerStorage.getPeerConfig(peerId); + boolean enabled = peerStorage.isPeerEnabled(peerId); + return new ReplicationPeerImpl(ReplicationUtils.getPeerClusterConfiguration(peerConfig, conf), + peerId, enabled, peerConfig); + } } diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java deleted file mode 100644 index 358721d..0000000 --- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java +++ /dev/null @@ -1,553 +0,0 @@ -/* - * - * 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; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collection; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.TreeMap; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.Abortable; -import org.apache.hadoop.hbase.CompoundConfiguration; -import org.apache.hadoop.hbase.HBaseConfiguration; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil; -import org.apache.hadoop.hbase.exceptions.DeserializationException; -import org.apache.hadoop.hbase.replication.ReplicationPeer.PeerState; -import org.apache.hadoop.hbase.util.Pair; -import org.apache.hadoop.hbase.zookeeper.ZKConfig; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; -import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp; -import org.apache.hadoop.hbase.zookeeper.ZKWatcher; -import org.apache.hadoop.hbase.zookeeper.ZNodePaths; -import org.apache.yetus.audience.InterfaceAudience; -import org.apache.zookeeper.KeeperException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos; - -/** - * This class provides an implementation of the ReplicationPeers interface using ZooKeeper. The - * peers znode contains a list of all peer replication clusters and the current replication state of - * those clusters. It has one child peer znode for each peer cluster. The peer znode is named with - * the cluster id provided by the user in the HBase shell. The value of the peer znode contains the - * peers cluster key provided by the user in the HBase Shell. The cluster key contains a list of - * zookeeper quorum peers, the client port for the zookeeper quorum, and the base znode for HBase. - * For example: - * - * /hbase/replication/peers/1 [Value: zk1.host.com,zk2.host.com,zk3.host.com:2181:/hbase] - * /hbase/replication/peers/2 [Value: zk5.host.com,zk6.host.com,zk7.host.com:2181:/hbase] - * - * Each of these peer znodes has a child znode that indicates whether or not replication is enabled - * on that peer cluster. These peer-state znodes do not have child znodes and simply contain a - * boolean value (i.e. ENABLED or DISABLED). This value is read/maintained by the - * ReplicationPeer.PeerStateTracker class. For example: - * - * /hbase/replication/peers/1/peer-state [Value: ENABLED] - * - * Each of these peer znodes has a child znode that indicates which data will be replicated - * to the peer cluster. These peer-tableCFs znodes do not have child znodes and only have a - * table/cf list config. This value is read/maintained by the ReplicationPeer.TableCFsTracker - * class. For example: - * - * /hbase/replication/peers/1/tableCFs [Value: "table1; table2:cf1,cf3; table3:cfx,cfy"] - */ -@InterfaceAudience.Private -public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements ReplicationPeers { - - // Map of peer clusters keyed by their id - private Map peerClusters; - private final ReplicationQueuesClient queuesClient; - private Abortable abortable; - - private static final Logger LOG = LoggerFactory.getLogger(ReplicationPeersZKImpl.class); - - public ReplicationPeersZKImpl(final ZKWatcher zk, final Configuration conf, - final ReplicationQueuesClient queuesClient, Abortable abortable) { - super(zk, conf, abortable); - this.abortable = abortable; - this.peerClusters = new ConcurrentHashMap<>(); - this.queuesClient = queuesClient; - } - - @Override - public void init() throws ReplicationException { - try { - if (ZKUtil.checkExists(this.zookeeper, this.peersZNode) < 0) { - ZKUtil.createWithParents(this.zookeeper, this.peersZNode); - } - } catch (KeeperException e) { - throw new ReplicationException("Could not initialize replication peers", e); - } - addExistingPeers(); - } - - @Override - public void registerPeer(String id, ReplicationPeerConfig peerConfig, boolean enabled) - throws ReplicationException { - try { - if (peerExists(id)) { - throw new IllegalArgumentException("Cannot add a peer with id=" + id - + " because that id already exists."); - } - - if(id.contains("-")){ - throw new IllegalArgumentException("Found invalid peer name:" + id); - } - - if (peerConfig.getClusterKey() != null) { - try { - ZKConfig.validateClusterKey(peerConfig.getClusterKey()); - } catch (IOException ioe) { - throw new IllegalArgumentException(ioe.getMessage()); - } - } - - checkQueuesDeleted(id); - - ZKUtil.createWithParents(this.zookeeper, this.peersZNode); - - List listOfOps = new ArrayList<>(2); - ZKUtilOp op1 = - ZKUtilOp.createAndFailSilent(getPeerNode(id), - ReplicationPeerConfigUtil.toByteArray(peerConfig)); - ZKUtilOp op2 = - ZKUtilOp.createAndFailSilent(getPeerStateNode(id), enabled ? ENABLED_ZNODE_BYTES - : DISABLED_ZNODE_BYTES); - listOfOps.add(op1); - listOfOps.add(op2); - ZKUtil.multiOrSequential(this.zookeeper, listOfOps, false); - } catch (KeeperException e) { - throw new ReplicationException("Could not add peer with id=" + id + ", peerConfif=>" - + peerConfig + ", state=" + (enabled ? "ENABLED" : "DISABLED"), e); - } - } - - @Override - public void unregisterPeer(String id) throws ReplicationException { - try { - if (!peerExists(id)) { - throw new IllegalArgumentException("Cannot remove peer with id=" + id - + " because that id does not exist."); - } - ZKUtil.deleteNodeRecursively(this.zookeeper, ZNodePaths.joinZNode(this.peersZNode, id)); - } catch (KeeperException e) { - throw new ReplicationException("Could not remove peer with id=" + id, e); - } - } - - @Override - public void enablePeer(String id) throws ReplicationException { - changePeerState(id, ReplicationProtos.ReplicationState.State.ENABLED); - LOG.info("peer " + id + " is enabled"); - } - - @Override - public void disablePeer(String id) throws ReplicationException { - changePeerState(id, ReplicationProtos.ReplicationState.State.DISABLED); - LOG.info("peer " + id + " is disabled"); - } - - @Override - public Map> getPeerTableCFsConfig(String id) throws ReplicationException { - try { - if (!peerExists(id)) { - throw new IllegalArgumentException("peer " + id + " doesn't exist"); - } - try { - ReplicationPeerConfig rpc = getReplicationPeerConfig(id); - if (rpc == null) { - throw new ReplicationException("Unable to get tableCFs of the peer with id=" + id); - } - return rpc.getTableCFsMap(); - } catch (Exception e) { - throw new ReplicationException(e); - } - } catch (KeeperException e) { - throw new ReplicationException("Unable to get tableCFs of the peer with id=" + id, e); - } - } - - @Override - public void setPeerTableCFsConfig(String id, - Map> tableCFs) - throws ReplicationException { - try { - if (!peerExists(id)) { - throw new IllegalArgumentException("Cannot set peer tableCFs because id=" + id - + " does not exist."); - } - ReplicationPeerConfig rpc = getReplicationPeerConfig(id); - if (rpc == null) { - throw new ReplicationException("Unable to get tableCFs of the peer with id=" + id); - } - rpc.setTableCFsMap(tableCFs); - ZKUtil.setData(this.zookeeper, getPeerNode(id), - ReplicationPeerConfigUtil.toByteArray(rpc)); - LOG.info("Peer tableCFs with id= " + id + " is now " + - ReplicationPeerConfigUtil.convertToString(tableCFs)); - } catch (KeeperException e) { - throw new ReplicationException("Unable to change tableCFs of the peer with id=" + id, e); - } - } - - @Override - public boolean getStatusOfPeer(String id) { - ReplicationPeer replicationPeer = this.peerClusters.get(id); - if (replicationPeer == null) { - throw new IllegalArgumentException("Peer with id= " + id + " is not cached"); - } - return replicationPeer.getPeerState() == PeerState.ENABLED; - } - - @Override - public boolean getStatusOfPeerFromBackingStore(String id) throws ReplicationException { - try { - if (!peerExists(id)) { - throw new IllegalArgumentException("peer " + id + " doesn't exist"); - } - String peerStateZNode = getPeerStateNode(id); - try { - return ReplicationPeerZKImpl.isStateEnabled(ZKUtil.getData(this.zookeeper, peerStateZNode)); - } catch (KeeperException e) { - throw new ReplicationException(e); - } catch (DeserializationException e) { - throw new ReplicationException(e); - } - } catch (KeeperException e) { - throw new ReplicationException("Unable to get status of the peer with id=" + id + - " from backing store", e); - } catch (InterruptedException e) { - throw new ReplicationException(e); - } - } - - @Override - public Map getAllPeerConfigs() { - Map peers = new TreeMap<>(); - List ids = null; - try { - ids = ZKUtil.listChildrenNoWatch(this.zookeeper, this.peersZNode); - for (String id : ids) { - ReplicationPeerConfig peerConfig = getReplicationPeerConfig(id); - if (peerConfig == null) { - LOG.warn("Failed to get replication peer configuration of clusterid=" + id - + " znode content, continuing."); - continue; - } - peers.put(id, peerConfig); - } - } catch (KeeperException e) { - this.abortable.abort("Cannot get the list of peers ", e); - } catch (ReplicationException e) { - this.abortable.abort("Cannot get the list of peers ", e); - } - return peers; - } - - @Override - public ReplicationPeer getConnectedPeer(String peerId) { - return peerClusters.get(peerId); - } - - @Override - public Set getConnectedPeerIds() { - return peerClusters.keySet(); // this is not thread-safe - } - - /** - * Returns a ReplicationPeerConfig from the znode or null for the given peerId. - */ - @Override - public ReplicationPeerConfig getReplicationPeerConfig(String peerId) - throws ReplicationException { - String znode = getPeerNode(peerId); - byte[] data = null; - try { - data = ZKUtil.getData(this.zookeeper, znode); - } catch (InterruptedException e) { - LOG.warn("Could not get configuration for peer because the thread " + - "was interrupted. peerId=" + peerId); - Thread.currentThread().interrupt(); - return null; - } catch (KeeperException e) { - throw new ReplicationException("Error getting configuration for peer with id=" - + peerId, e); - } - if (data == null) { - LOG.error("Could not get configuration for peer because it doesn't exist. peerId=" + peerId); - return null; - } - - try { - return ReplicationPeerConfigUtil.parsePeerFrom(data); - } catch (DeserializationException e) { - LOG.warn("Failed to parse cluster key from peerId=" + peerId - + ", specifically the content from the following znode: " + znode); - return null; - } - } - - @Override - public Pair getPeerConf(String peerId) - throws ReplicationException { - ReplicationPeerConfig peerConfig = getReplicationPeerConfig(peerId); - - if (peerConfig == null) { - return null; - } - - Configuration otherConf; - try { - otherConf = HBaseConfiguration.createClusterConf(this.conf, peerConfig.getClusterKey()); - } catch (IOException e) { - LOG.error("Can't get peer configuration for peerId=" + peerId + " because:", e); - return null; - } - - if (!peerConfig.getConfiguration().isEmpty()) { - CompoundConfiguration compound = new CompoundConfiguration(); - compound.add(otherConf); - compound.addStringMap(peerConfig.getConfiguration()); - return new Pair<>(peerConfig, compound); - } - - return new Pair<>(peerConfig, otherConf); - } - - @Override - public void updatePeerConfig(String id, ReplicationPeerConfig newConfig) - throws ReplicationException { - ReplicationPeer peer = getConnectedPeer(id); - if (peer == null){ - throw new ReplicationException("Could not find peer Id " + id + " in connected peers"); - } - ReplicationPeerConfig existingConfig = peer.getPeerConfig(); - if (newConfig.getClusterKey() != null && !newConfig.getClusterKey().isEmpty() && - !newConfig.getClusterKey().equals(existingConfig.getClusterKey())){ - throw new ReplicationException("Changing the cluster key on an existing peer is not allowed." - + " Existing key '" + existingConfig.getClusterKey() + "' does not match new key '" - + newConfig.getClusterKey() + - "'"); - } - String existingEndpointImpl = existingConfig.getReplicationEndpointImpl(); - if (newConfig.getReplicationEndpointImpl() != null && - !newConfig.getReplicationEndpointImpl().isEmpty() && - !newConfig.getReplicationEndpointImpl().equals(existingEndpointImpl)){ - throw new ReplicationException("Changing the replication endpoint implementation class " + - "on an existing peer is not allowed. Existing class '" - + existingConfig.getReplicationEndpointImpl() - + "' does not match new class '" + newConfig.getReplicationEndpointImpl() + "'"); - } - // Update existingConfig's peer config and peer data with the new values, but don't touch config - // or data that weren't explicitly changed - ReplicationPeerConfigBuilder builder = ReplicationPeerConfig.newBuilder(existingConfig); - builder.putAllConfiguration(newConfig.getConfiguration()) - .putAllPeerData(newConfig.getPeerData()) - .setReplicateAllUserTables(newConfig.replicateAllUserTables()) - .setNamespaces(newConfig.getNamespaces()).setTableCFsMap(newConfig.getTableCFsMap()) - .setExcludeNamespaces(newConfig.getExcludeNamespaces()) - .setExcludeTableCFsMap(newConfig.getExcludeTableCFsMap()) - .setBandwidth(newConfig.getBandwidth()); - - try { - ZKUtil.setData(this.zookeeper, getPeerNode(id), - ReplicationPeerConfigUtil.toByteArray(builder.build())); - } - catch(KeeperException ke){ - throw new ReplicationException("There was a problem trying to save changes to the " + - "replication peer " + id, ke); - } - } - - /** - * List all registered peer clusters and set a watch on their znodes. - */ - @Override - public List getAllPeerIds() { - List ids = null; - try { - ids = ZKUtil.listChildrenAndWatchThem(this.zookeeper, this.peersZNode); - } catch (KeeperException e) { - this.abortable.abort("Cannot get the list of peers ", e); - } - return ids; - } - - /** - * A private method used during initialization. This method attempts to add all registered - * peer clusters. This method does not set a watch on the peer cluster znodes. - */ - private void addExistingPeers() throws ReplicationException { - List znodes = null; - try { - znodes = ZKUtil.listChildrenNoWatch(this.zookeeper, this.peersZNode); - } catch (KeeperException e) { - throw new ReplicationException("Error getting the list of peer clusters.", e); - } - if (znodes != null) { - for (String z : znodes) { - createAndAddPeer(z); - } - } - } - - @Override - public boolean peerConnected(String peerId) throws ReplicationException { - return createAndAddPeer(peerId); - } - - @Override - public void peerDisconnected(String peerId) { - ReplicationPeer rp = this.peerClusters.get(peerId); - if (rp != null) { - ((ConcurrentMap) peerClusters).remove(peerId, rp); - } - } - - /** - * Attempt to connect to a new remote slave cluster. - * @param peerId a short that identifies the cluster - * @return true if a new connection was made, false if no new connection was made. - */ - public boolean createAndAddPeer(String peerId) throws ReplicationException { - if (peerClusters == null) { - return false; - } - if (this.peerClusters.containsKey(peerId)) { - return false; - } - - ReplicationPeerZKImpl peer = null; - try { - peer = createPeer(peerId); - } catch (Exception e) { - throw new ReplicationException("Error adding peer with id=" + peerId, e); - } - if (peer == null) { - return false; - } - ReplicationPeerZKImpl previous = - ((ConcurrentMap) peerClusters).putIfAbsent(peerId, peer); - if (previous == null) { - LOG.info("Added new peer cluster=" + peer.getPeerConfig().getClusterKey()); - } else { - LOG.info("Peer already present, " + previous.getPeerConfig().getClusterKey() + - ", new cluster=" + peer.getPeerConfig().getClusterKey()); - } - return true; - } - - /** - * Update the state znode of a peer cluster. - * @param id the id of the peer - * @param state the state to update to - */ - private void changePeerState(String id, ReplicationProtos.ReplicationState.State state) - throws ReplicationException { - try { - if (!peerExists(id)) { - throw new IllegalArgumentException("Cannot enable/disable peer because id=" + id - + " does not exist."); - } - String peerStateZNode = getPeerStateNode(id); - byte[] stateBytes = - (state == ReplicationProtos.ReplicationState.State.ENABLED) ? ENABLED_ZNODE_BYTES - : DISABLED_ZNODE_BYTES; - if (ZKUtil.checkExists(this.zookeeper, peerStateZNode) != -1) { - ZKUtil.setData(this.zookeeper, peerStateZNode, stateBytes); - } else { - ZKUtil.createAndWatch(this.zookeeper, peerStateZNode, stateBytes); - } - LOG.info("Peer with id= " + id + " is now " + state.name()); - } catch (KeeperException e) { - throw new ReplicationException("Unable to change state of the peer with id=" + id, e); - } - } - - /** - * Helper method to connect to a peer - * @param peerId peer's identifier - * @return object representing the peer - * @throws ReplicationException if creating the peer fails - */ - private ReplicationPeerZKImpl createPeer(String peerId) throws ReplicationException { - Pair pair = getPeerConf(peerId); - if (pair == null) { - return null; - } - Configuration peerConf = pair.getSecond(); - - ReplicationPeerZKImpl peer = new ReplicationPeerZKImpl(zookeeper, - peerConf, peerId, pair.getFirst(), abortable); - try { - peer.startStateTracker(this.getPeerStateNode(peerId)); - } catch (KeeperException e) { - throw new ReplicationException("Error starting the peer state tracker for peerId=" + - peerId, e); - } - - try { - peer.startPeerConfigTracker(this.getPeerNode(peerId)); - } catch (KeeperException e) { - throw new ReplicationException("Error starting the peer tableCFs tracker for peerId=" + - peerId, e); - } - - return peer; - } - - private void checkQueuesDeleted(String peerId) throws ReplicationException { - if (queuesClient == null) { - return; - } - - try { - List replicators = queuesClient.getListOfReplicators(); - if (replicators == null || replicators.isEmpty()) { - return; - } - for (String replicator : replicators) { - List queueIds = queuesClient.getAllQueues(replicator); - for (String queueId : queueIds) { - ReplicationQueueInfo queueInfo = new ReplicationQueueInfo(queueId); - if (queueInfo.getPeerId().equals(peerId)) { - throw new ReplicationException("undeleted queue for peerId: " + peerId - + ", replicator: " + replicator + ", queueId: " + queueId); - } - } - } - // Check for hfile-refs queue - if (-1 != ZKUtil.checkExists(zookeeper, hfileRefsZNode) - && queuesClient.getAllPeersFromHFileRefsQueue().contains(peerId)) { - throw new ReplicationException("Undeleted queue for peerId: " + peerId - + ", found in hfile-refs node path " + hfileRefsZNode); - } - } catch (KeeperException e) { - throw new ReplicationException("Could not check queues deleted with id=" + peerId, e); - } - } -} diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueInfo.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueInfo.java index 8ef1939..d39a37e 100644 --- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueInfo.java +++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueInfo.java @@ -28,7 +28,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** - * This class is responsible for the parsing logic for a znode representing a queue. + * This class is responsible for the parsing logic for a queue id representing a queue. * It will extract the peerId if it's recovered as well as the dead region servers * that were part of the queue's history. */ @@ -37,21 +37,20 @@ public class ReplicationQueueInfo { private static final Logger LOG = LoggerFactory.getLogger(ReplicationQueueInfo.class); private final String peerId; - private final String peerClusterZnode; + private final String queueId; private boolean queueRecovered; // List of all the dead region servers that had this queue (if recovered) private List deadRegionServers = new ArrayList<>(); /** - * The passed znode will be either the id of the peer cluster or - * the handling story of that queue in the form of id-servername-* + * The passed queueId will be either the id of the peer or the handling story of that queue + * in the form of id-servername-* */ - public ReplicationQueueInfo(String znode) { - this.peerClusterZnode = znode; - String[] parts = znode.split("-", 2); + public ReplicationQueueInfo(String queueId) { + this.queueId = queueId; + String[] parts = queueId.split("-", 2); this.queueRecovered = parts.length != 1; - this.peerId = this.queueRecovered ? - parts[0] : peerClusterZnode; + this.peerId = this.queueRecovered ? parts[0] : queueId; if (parts.length >= 2) { // extract dead servers extractDeadServersFromZNodeString(parts[1], this.deadRegionServers); @@ -59,7 +58,7 @@ public class ReplicationQueueInfo { } /** - * Parse dead server names from znode string servername can contain "-" such as + * Parse dead server names from queue id. servername can contain "-" such as * "ip-10-46-221-101.ec2.internal", so we need skip some "-" during parsing for the following * cases: 2-ip-10-46-221-101.ec2.internal,52170,1364333181125-<server name>-... */ @@ -119,8 +118,8 @@ public class ReplicationQueueInfo { return this.peerId; } - public String getPeerClusterZnode() { - return this.peerClusterZnode; + public String getQueueId() { + return this.queueId; } public boolean isQueueRecovered() { diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueStorage.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueStorage.java new file mode 100644 index 0000000..e774148 --- /dev/null +++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueStorage.java @@ -0,0 +1,172 @@ +/** + * 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; + +import java.util.List; +import java.util.Set; +import java.util.SortedSet; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.util.Pair; +import org.apache.yetus.audience.InterfaceAudience; + +/** + * Perform read/write to the replication queue storage. + */ +@InterfaceAudience.Private +public interface ReplicationQueueStorage { + + /** + * Remove a replication queue for a given regionserver. + * @param serverName the name of the regionserver + * @param queueId a String that identifies the queue. + */ + void removeQueue(ServerName serverName, String queueId) throws ReplicationException; + + /** + * Add a new WAL file to the given queue for a given regionserver. If the queue does not exist it + * is created. + * @param serverName the name of the regionserver + * @param queueId a String that identifies the queue. + * @param fileName name of the WAL + */ + void addWAL(ServerName serverName, String queueId, String fileName) throws ReplicationException; + + /** + * Remove an WAL file from the given queue for a given regionserver. + * @param serverName the name of the regionserver + * @param queueId a String that identifies the queue. + * @param fileName name of the WAL + */ + void removeWAL(ServerName serverName, String queueId, String fileName) + throws ReplicationException; + + /** + * Set the current position for a specific WAL in a given queue for a given regionserver. + * @param serverName the name of the regionserver + * @param queueId a String that identifies the queue + * @param fileName name of the WAL + * @param position the current position in the file + */ + void setWALPosition(ServerName serverName, String queueId, String fileName, long position) + throws ReplicationException; + + /** + * Get the current position for a specific WAL in a given queue for a given regionserver. + * @param serverName the name of the regionserver + * @param queueId a String that identifies the queue + * @param fileName name of the WAL + * @return the current position in the file + */ + long getWALPosition(ServerName serverName, String queueId, String fileName) + throws ReplicationException; + + /** + * Get a list of all WALs in the given queue on the given region server. + * @param serverName the server name of the region server that owns the queue + * @param queueId a String that identifies the queue + * @return a list of WALs + */ + List getWALsInQueue(ServerName serverName, String queueId) throws ReplicationException; + + /** + * Get a list of all queues for the specified region server. + * @param serverName the server name of the region server that owns the set of queues + * @return a list of queueIds + */ + List getAllQueues(ServerName serverName) throws ReplicationException; + + /** + * Change ownership for the queue identified by queueId and belongs to a dead region server. + * @param sourceServerName the name of the dead region server + * @param destServerName the name of the target region server + * @param queueId the id of the queue + * @return the new PeerId and A SortedSet of WALs in its queue + */ + Pair> claimQueue(ServerName sourceServerName, String queueId, + ServerName destServerName) throws ReplicationException; + + /** + * Remove the record of region server if the queue is empty. + */ + void removeReplicatorIfQueueIsEmpty(ServerName serverName) throws ReplicationException; + + /** + * Get a list of all region servers that have outstanding replication queues. These servers could + * be alive, dead or from a previous run of the cluster. + * @return a list of server names + */ + List getListOfReplicators() throws ReplicationException; + + /** + * Load all wals in all replication queues. This method guarantees to return a snapshot which + * contains all WALs at the start of this call even there is concurrent queue failover. However, + * some newly created WALs during the call may not be included. + */ + Set getAllWALs() throws ReplicationException; + + /** + * Add a peer to hfile reference queue if peer does not exist. + * @param peerId peer cluster id to be added + * @throws ReplicationException if fails to add a peer id to hfile reference queue + */ + void addPeerToHFileRefs(String peerId) throws ReplicationException; + + /** + * Remove a peer from hfile reference queue. + * @param peerId peer cluster id to be removed + */ + void removePeerFromHFileRefs(String peerId) throws ReplicationException; + + /** + * Add new hfile references to the queue. + * @param peerId peer cluster id to which the hfiles need to be replicated + * @param pairs list of pairs of { HFile location in staging dir, HFile path in region dir which + * will be added in the queue } + * @throws ReplicationException if fails to add a hfile reference + */ + void addHFileRefs(String peerId, List> pairs) throws ReplicationException; + + /** + * Remove hfile references from the queue. + * @param peerId peer cluster id from which this hfile references needs to be removed + * @param files list of hfile references to be removed + */ + void removeHFileRefs(String peerId, List files) throws ReplicationException; + + /** + * Get list of all peers from hfile reference queue. + * @return a list of peer ids + */ + List getAllPeersFromHFileRefsQueue() throws ReplicationException; + + /** + * Get a list of all hfile references in the given peer. + * @param peerId a String that identifies the peer + * @return a list of hfile references + */ + List getReplicableHFiles(String peerId) throws ReplicationException; + + /** + * Load all hfile references in all replication queues. This method guarantees to return a + * snapshot which contains all hfile references at the start of this call. However, some newly + * created hfile references during the call may not be included. + */ + Set getAllHFileRefs() throws ReplicationException; +} diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java deleted file mode 100644 index 81e94c9..0000000 --- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java +++ /dev/null @@ -1,161 +0,0 @@ -/* - * - * 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; - -import java.util.List; -import java.util.SortedSet; - -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.util.Pair; -import org.apache.yetus.audience.InterfaceAudience; - -/** - * This provides an interface for maintaining a region server's replication queues. These queues - * keep track of the WALs and HFile references (if hbase.replication.bulkload.enabled is enabled) - * that still need to be replicated to remote clusters. - */ -@InterfaceAudience.Private -public interface ReplicationQueues { - - /** - * Initialize the region server replication queue interface. - * @param serverName The server name of the region server that owns the replication queues this - * interface manages. - */ - void init(String serverName) throws ReplicationException; - - /** - * Remove a replication queue. - * @param queueId a String that identifies the queue. - */ - void removeQueue(String queueId); - - /** - * Add a new WAL file to the given queue. If the queue does not exist it is created. - * @param queueId a String that identifies the queue. - * @param filename name of the WAL - */ - void addLog(String queueId, String filename) throws ReplicationException; - - /** - * Remove an WAL file from the given queue. - * @param queueId a String that identifies the queue. - * @param filename name of the WAL - */ - void removeLog(String queueId, String filename); - - /** - * Set the current position for a specific WAL in a given queue. - * @param queueId a String that identifies the queue - * @param filename name of the WAL - * @param position the current position in the file - */ - void setLogPosition(String queueId, String filename, long position); - - /** - * Get the current position for a specific WAL in a given queue. - * @param queueId a String that identifies the queue - * @param filename name of the WAL - * @return the current position in the file - */ - long getLogPosition(String queueId, String filename) throws ReplicationException; - - /** - * Remove all replication queues for this region server. - */ - void removeAllQueues(); - - /** - * Get a list of all WALs in the given queue. - * @param queueId a String that identifies the queue - * @return a list of WALs, null if no such queue exists for this server - */ - List getLogsInQueue(String queueId); - - /** - * Get a list of all queues for this region server. - * @return a list of queueIds, an empty list if this region server is dead and has no outstanding - * queues - */ - List getAllQueues(); - - /** - * Get queueIds from a dead region server, whose queues has not been claimed by other region - * servers. - * @return empty if the queue exists but no children, null if the queue does not exist. - */ - List getUnClaimedQueueIds(String regionserver); - - /** - * Take ownership for the queue identified by queueId and belongs to a dead region server. - * @param regionserver the id of the dead region server - * @param queueId the id of the queue - * @return the new PeerId and A SortedSet of WALs in its queue, and null if no unclaimed queue. - */ - Pair> claimQueue(String regionserver, String queueId); - - /** - * Remove the znode of region server if the queue is empty. - * @param regionserver the id of the region server - */ - void removeReplicatorIfQueueIsEmpty(String regionserver); - - /** - * Get a list of all region servers that have outstanding replication queues. These servers could - * be alive, dead or from a previous run of the cluster. - * @return a list of server names - */ - List getListOfReplicators(); - - /** - * Checks if the provided znode is the same as this region server's - * @param regionserver the id of the region server - * @return if this is this rs's znode - */ - boolean isThisOurRegionServer(String regionserver); - - /** - * Add a peer to hfile reference queue if peer does not exist. - * @param peerId peer cluster id to be added - * @throws ReplicationException if fails to add a peer id to hfile reference queue - */ - void addPeerToHFileRefs(String peerId) throws ReplicationException; - - /** - * Remove a peer from hfile reference queue. - * @param peerId peer cluster id to be removed - */ - void removePeerFromHFileRefs(String peerId); - - /** - * Add new hfile references to the queue. - * @param peerId peer cluster id to which the hfiles need to be replicated - * @param pairs list of pairs of { HFile location in staging dir, HFile path in region dir which - * will be added in the queue } - * @throws ReplicationException if fails to add a hfile reference - */ - void addHFileRefs(String peerId, List> pairs) throws ReplicationException; - - /** - * Remove hfile references from the queue. - * @param peerId peer cluster id from which this hfile references needs to be removed - * @param files list of hfile references to be removed - */ - void removeHFileRefs(String peerId, List files); -} diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesArguments.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesArguments.java deleted file mode 100644 index c2a5df3..0000000 --- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesArguments.java +++ /dev/null @@ -1,70 +0,0 @@ -/* - * - * 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; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.Abortable; -import org.apache.hadoop.hbase.zookeeper.ZKWatcher; -import org.apache.yetus.audience.InterfaceAudience; - -/** - * Wrapper around common arguments used to construct ReplicationQueues. Used to construct various - * ReplicationQueues Implementations with different constructor arguments by reflection. - */ -@InterfaceAudience.Private -public class ReplicationQueuesArguments { - - private ZKWatcher zk; - private Configuration conf; - private Abortable abort; - - public ReplicationQueuesArguments(Configuration conf, Abortable abort) { - this.conf = conf; - this.abort = abort; - } - - public ReplicationQueuesArguments(Configuration conf, Abortable abort, ZKWatcher zk) { - this(conf, abort); - setZk(zk); - } - - public ZKWatcher getZk() { - return zk; - } - - public void setZk(ZKWatcher zk) { - this.zk = zk; - } - - public Configuration getConf() { - return conf; - } - - public void setConf(Configuration conf) { - this.conf = conf; - } - - public Abortable getAbortable() { - return abort; - } - - public void setAbortable(Abortable abort) { - this.abort = abort; - } -} diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClient.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClient.java deleted file mode 100644 index 7ef4004..0000000 --- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClient.java +++ /dev/null @@ -1,93 +0,0 @@ -/* - * - * 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; - -import java.util.List; -import java.util.Set; - -import org.apache.yetus.audience.InterfaceAudience; -import org.apache.zookeeper.KeeperException; - -/** - * This provides an interface for clients of replication to view replication queues. These queues - * keep track of the sources(WALs/HFile references) that still need to be replicated to remote - * clusters. - */ -@InterfaceAudience.Private -public interface ReplicationQueuesClient { - - /** - * Initialize the replication queue client interface. - */ - public void init() throws ReplicationException; - - /** - * Get a list of all region servers that have outstanding replication queues. These servers could - * be alive, dead or from a previous run of the cluster. - * @return a list of server names - * @throws KeeperException zookeeper exception - */ - List getListOfReplicators() throws KeeperException; - - /** - * Get a list of all WALs in the given queue on the given region server. - * @param serverName the server name of the region server that owns the queue - * @param queueId a String that identifies the queue - * @return a list of WALs, null if this region server is dead and has no outstanding queues - * @throws KeeperException zookeeper exception - */ - List getLogsInQueue(String serverName, String queueId) throws KeeperException; - - /** - * Get a list of all queues for the specified region server. - * @param serverName the server name of the region server that owns the set of queues - * @return a list of queueIds, null if this region server is not a replicator. - */ - List getAllQueues(String serverName) throws KeeperException; - - /** - * Load all wals in all replication queues from ZK. This method guarantees to return a - * snapshot which contains all WALs in the zookeeper at the start of this call even there - * is concurrent queue failover. However, some newly created WALs during the call may - * not be included. - */ - Set getAllWALs() throws KeeperException; - - /** - * Get the change version number of replication hfile references node. This can be used as - * optimistic locking to get a consistent snapshot of the replication queues of hfile references. - * @return change version number of hfile references node - */ - int getHFileRefsNodeChangeVersion() throws KeeperException; - - /** - * Get list of all peers from hfile reference queue. - * @return a list of peer ids - * @throws KeeperException zookeeper exception - */ - List getAllPeersFromHFileRefsQueue() throws KeeperException; - - /** - * Get a list of all hfile references in the given peer. - * @param peerId a String that identifies the peer - * @return a list of hfile references, null if not found any - * @throws KeeperException zookeeper exception - */ - List getReplicableHFiles(String peerId) throws KeeperException; -} diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientArguments.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientArguments.java deleted file mode 100644 index 9b79294..0000000 --- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientArguments.java +++ /dev/null @@ -1,40 +0,0 @@ -/* - * - * 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; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.Abortable; -import org.apache.hadoop.hbase.zookeeper.ZKWatcher; -import org.apache.yetus.audience.InterfaceAudience; - -/** - * Wrapper around common arguments used to construct ReplicationQueuesClient. Used to construct - * various ReplicationQueuesClient Implementations with different constructor arguments by - * reflection. - */ -@InterfaceAudience.Private -public class ReplicationQueuesClientArguments extends ReplicationQueuesArguments { - public ReplicationQueuesClientArguments(Configuration conf, Abortable abort, - ZKWatcher zk) { - super(conf, abort, zk); - } - public ReplicationQueuesClientArguments(Configuration conf, Abortable abort) { - super(conf, abort); - } -} diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java deleted file mode 100644 index 4dccf7f..0000000 --- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java +++ /dev/null @@ -1,177 +0,0 @@ -/* - * - * 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; - -import java.util.List; -import java.util.Set; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.Abortable; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; -import org.apache.hadoop.hbase.zookeeper.ZKWatcher; -import org.apache.hadoop.hbase.zookeeper.ZNodePaths; -import org.apache.yetus.audience.InterfaceAudience; -import org.apache.zookeeper.KeeperException; -import org.apache.zookeeper.data.Stat; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableSet; -import org.apache.hbase.thirdparty.com.google.common.collect.Sets; - -@InterfaceAudience.Private -public class ReplicationQueuesClientZKImpl extends ReplicationStateZKBase implements - ReplicationQueuesClient { - - Logger LOG = LoggerFactory.getLogger(ReplicationQueuesClientZKImpl.class); - - public ReplicationQueuesClientZKImpl(ReplicationQueuesClientArguments args) { - this(args.getZk(), args.getConf(), args.getAbortable()); - } - - public ReplicationQueuesClientZKImpl(final ZKWatcher zk, Configuration conf, - Abortable abortable) { - super(zk, conf, abortable); - } - - @Override - public void init() throws ReplicationException { - try { - if (ZKUtil.checkExists(this.zookeeper, this.queuesZNode) < 0) { - ZKUtil.createWithParents(this.zookeeper, this.queuesZNode); - } - } catch (KeeperException e) { - throw new ReplicationException("Internal error while initializing a queues client", e); - } - } - - @Override - public List getLogsInQueue(String serverName, String queueId) throws KeeperException { - String znode = ZNodePaths.joinZNode(this.queuesZNode, serverName); - znode = ZNodePaths.joinZNode(znode, queueId); - List result = null; - try { - result = ZKUtil.listChildrenNoWatch(this.zookeeper, znode); - } catch (KeeperException e) { - this.abortable.abort("Failed to get list of wals for queueId=" + queueId - + " and serverName=" + serverName, e); - throw e; - } - return result; - } - - @Override - public List getAllQueues(String serverName) throws KeeperException { - String znode = ZNodePaths.joinZNode(this.queuesZNode, serverName); - List result = null; - try { - result = ZKUtil.listChildrenNoWatch(this.zookeeper, znode); - } catch (KeeperException e) { - this.abortable.abort("Failed to get list of queues for serverName=" + serverName, e); - throw e; - } - return result; - } - - @Override - public Set getAllWALs() throws KeeperException { - /** - * Load all wals in all replication queues from ZK. This method guarantees to return a - * snapshot which contains all WALs in the zookeeper at the start of this call even there - * is concurrent queue failover. However, some newly created WALs during the call may - * not be included. - */ - for (int retry = 0; ; retry++) { - int v0 = getQueuesZNodeCversion(); - List rss = getListOfReplicators(); - if (rss == null || rss.isEmpty()) { - LOG.debug("Didn't find any region server that replicates, won't prevent any deletions."); - return ImmutableSet.of(); - } - Set wals = Sets.newHashSet(); - for (String rs : rss) { - List listOfPeers = getAllQueues(rs); - // if rs just died, this will be null - if (listOfPeers == null) { - continue; - } - for (String id : listOfPeers) { - List peersWals = getLogsInQueue(rs, id); - if (peersWals != null) { - wals.addAll(peersWals); - } - } - } - int v1 = getQueuesZNodeCversion(); - if (v0 == v1) { - return wals; - } - LOG.info(String.format("Replication queue node cversion changed from %d to %d, retry = %d", - v0, v1, retry)); - } - } - - public int getQueuesZNodeCversion() throws KeeperException { - try { - Stat stat = new Stat(); - ZKUtil.getDataNoWatch(this.zookeeper, this.queuesZNode, stat); - return stat.getCversion(); - } catch (KeeperException e) { - this.abortable.abort("Failed to get stat of replication rs node", e); - throw e; - } - } - - @Override - public int getHFileRefsNodeChangeVersion() throws KeeperException { - Stat stat = new Stat(); - try { - ZKUtil.getDataNoWatch(this.zookeeper, this.hfileRefsZNode, stat); - } catch (KeeperException e) { - this.abortable.abort("Failed to get stat of replication hfile references node.", e); - throw e; - } - return stat.getCversion(); - } - - @Override - public List getAllPeersFromHFileRefsQueue() throws KeeperException { - List result = null; - try { - result = ZKUtil.listChildrenNoWatch(this.zookeeper, this.hfileRefsZNode); - } catch (KeeperException e) { - this.abortable.abort("Failed to get list of all peers in hfile references node.", e); - throw e; - } - return result; - } - - @Override - public List getReplicableHFiles(String peerId) throws KeeperException { - String znode = ZNodePaths.joinZNode(this.hfileRefsZNode, peerId); - List result = null; - try { - result = ZKUtil.listChildrenNoWatch(this.zookeeper, znode); - } catch (KeeperException e) { - this.abortable.abort("Failed to get list of hfile references for peerId=" + peerId, e); - throw e; - } - return result; - } -} diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java deleted file mode 100644 index 56abed2..0000000 --- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java +++ /dev/null @@ -1,408 +0,0 @@ -/* - * - * 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; - -import java.util.ArrayList; -import java.util.List; -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.Abortable; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.exceptions.DeserializationException; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.Pair; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; -import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp; -import org.apache.hadoop.hbase.zookeeper.ZKWatcher; -import org.apache.hadoop.hbase.zookeeper.ZNodePaths; -import org.apache.yetus.audience.InterfaceAudience; -import org.apache.zookeeper.KeeperException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * This class provides an implementation of the - * interface using ZooKeeper. The - * base znode that this class works at is the myQueuesZnode. The myQueuesZnode contains a list of - * all outstanding WAL files on this region server that need to be replicated. The myQueuesZnode is - * the regionserver name (a concatenation of the region server’s hostname, client port and start - * code). For example: - * - * /hbase/replication/rs/hostname.example.org,6020,1234 - * - * Within this znode, the region server maintains a set of WAL replication queues. These queues are - * represented by child znodes named using there give queue id. For example: - * - * /hbase/replication/rs/hostname.example.org,6020,1234/1 - * /hbase/replication/rs/hostname.example.org,6020,1234/2 - * - * Each queue has one child znode for every WAL that still needs to be replicated. The value of - * these WAL child znodes is the latest position that has been replicated. This position is updated - * every time a WAL entry is replicated. For example: - * - * /hbase/replication/rs/hostname.example.org,6020,1234/1/23522342.23422 [VALUE: 254] - */ -@InterfaceAudience.Private -public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements ReplicationQueues { - - /** Znode containing all replication queues for this region server. */ - private String myQueuesZnode; - - private static final Logger LOG = LoggerFactory.getLogger(ReplicationQueuesZKImpl.class); - - public ReplicationQueuesZKImpl(ReplicationQueuesArguments args) { - this(args.getZk(), args.getConf(), args.getAbortable()); - } - - public ReplicationQueuesZKImpl(final ZKWatcher zk, Configuration conf, - Abortable abortable) { - super(zk, conf, abortable); - } - - @Override - public void init(String serverName) throws ReplicationException { - this.myQueuesZnode = ZNodePaths.joinZNode(this.queuesZNode, serverName); - try { - if (ZKUtil.checkExists(this.zookeeper, this.myQueuesZnode) < 0) { - ZKUtil.createWithParents(this.zookeeper, this.myQueuesZnode); - } - } catch (KeeperException e) { - throw new ReplicationException("Could not initialize replication queues.", e); - } - if (conf.getBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY, - HConstants.REPLICATION_BULKLOAD_ENABLE_DEFAULT)) { - try { - if (ZKUtil.checkExists(this.zookeeper, this.hfileRefsZNode) < 0) { - ZKUtil.createWithParents(this.zookeeper, this.hfileRefsZNode); - } - } catch (KeeperException e) { - throw new ReplicationException("Could not initialize hfile references replication queue.", - e); - } - } - } - - @Override - public void removeQueue(String queueId) { - try { - ZKUtil.deleteNodeRecursively(this.zookeeper, - ZNodePaths.joinZNode(this.myQueuesZnode, queueId)); - } catch (KeeperException e) { - this.abortable.abort("Failed to delete queue (queueId=" + queueId + ")", e); - } - } - - @Override - public void addLog(String queueId, String filename) throws ReplicationException { - String znode = ZNodePaths.joinZNode(this.myQueuesZnode, queueId); - znode = ZNodePaths.joinZNode(znode, filename); - try { - ZKUtil.createWithParents(this.zookeeper, znode); - } catch (KeeperException e) { - throw new ReplicationException( - "Could not add log because znode could not be created. queueId=" + queueId - + ", filename=" + filename); - } - } - - @Override - public void removeLog(String queueId, String filename) { - try { - String znode = ZNodePaths.joinZNode(this.myQueuesZnode, queueId); - znode = ZNodePaths.joinZNode(znode, filename); - ZKUtil.deleteNode(this.zookeeper, znode); - } catch (KeeperException e) { - this.abortable.abort("Failed to remove wal from queue (queueId=" + queueId + ", filename=" - + filename + ")", e); - } - } - - @Override - public void setLogPosition(String queueId, String filename, long position) { - try { - String znode = ZNodePaths.joinZNode(this.myQueuesZnode, queueId); - znode = ZNodePaths.joinZNode(znode, filename); - // Why serialize String of Long and not Long as bytes? - ZKUtil.setData(this.zookeeper, znode, ZKUtil.positionToByteArray(position)); - } catch (KeeperException e) { - this.abortable.abort("Failed to write replication wal position (filename=" + filename - + ", position=" + position + ")", e); - } - } - - @Override - public long getLogPosition(String queueId, String filename) throws ReplicationException { - String clusterZnode = ZNodePaths.joinZNode(this.myQueuesZnode, queueId); - String znode = ZNodePaths.joinZNode(clusterZnode, filename); - byte[] bytes = null; - try { - bytes = ZKUtil.getData(this.zookeeper, znode); - } catch (KeeperException e) { - throw new ReplicationException("Internal Error: could not get position in log for queueId=" - + queueId + ", filename=" + filename, e); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - return 0; - } - try { - return ZKUtil.parseWALPositionFrom(bytes); - } catch (DeserializationException de) { - LOG.warn("Failed to parse WALPosition for queueId=" + queueId + " and wal=" + filename - + " znode content, continuing."); - } - // if we can not parse the position, start at the beginning of the wal file - // again - return 0; - } - - @Override - public boolean isThisOurRegionServer(String regionserver) { - return ZNodePaths.joinZNode(this.queuesZNode, regionserver).equals(this.myQueuesZnode); - } - - @Override - public List getUnClaimedQueueIds(String regionserver) { - if (isThisOurRegionServer(regionserver)) { - return null; - } - String rsZnodePath = ZNodePaths.joinZNode(this.queuesZNode, regionserver); - List queues = null; - try { - queues = ZKUtil.listChildrenNoWatch(this.zookeeper, rsZnodePath); - } catch (KeeperException e) { - this.abortable.abort("Failed to getUnClaimedQueueIds for RS" + regionserver, e); - } - return queues; - } - - @Override - public Pair> claimQueue(String regionserver, String queueId) { - LOG.info("Atomically moving " + regionserver + "/" + queueId + "'s WALs to my queue"); - return moveQueueUsingMulti(regionserver, queueId); - } - - @Override - public void removeReplicatorIfQueueIsEmpty(String regionserver) { - String rsPath = ZNodePaths.joinZNode(this.queuesZNode, regionserver); - try { - List list = ZKUtil.listChildrenNoWatch(this.zookeeper, rsPath); - if (list != null && list.isEmpty()){ - ZKUtil.deleteNode(this.zookeeper, rsPath); - } - } catch (KeeperException e) { - LOG.warn("Got error while removing replicator", e); - } - } - - @Override - public void removeAllQueues() { - try { - ZKUtil.deleteNodeRecursively(this.zookeeper, this.myQueuesZnode); - } catch (KeeperException e) { - // if the znode is already expired, don't bother going further - if (e instanceof KeeperException.SessionExpiredException) { - return; - } - this.abortable.abort("Failed to delete replication queues for region server: " - + this.myQueuesZnode, e); - } - } - - @Override - public List getLogsInQueue(String queueId) { - String znode = ZNodePaths.joinZNode(this.myQueuesZnode, queueId); - List result = null; - try { - result = ZKUtil.listChildrenNoWatch(this.zookeeper, znode); - } catch (KeeperException e) { - this.abortable.abort("Failed to get list of wals for queueId=" + queueId, e); - } - return result; - } - - @Override - public List getAllQueues() { - List listOfQueues = null; - try { - listOfQueues = ZKUtil.listChildrenNoWatch(this.zookeeper, this.myQueuesZnode); - } catch (KeeperException e) { - this.abortable.abort("Failed to get a list of queues for region server: " - + this.myQueuesZnode, e); - } - return listOfQueues == null ? new ArrayList<>() : listOfQueues; - } - - /** - * It "atomically" copies one peer's wals queue from another dead region server and returns them - * all sorted. The new peer id is equal to the old peer id appended with the dead server's znode. - * @param znode pertaining to the region server to copy the queues from - * @peerId peerId pertaining to the queue need to be copied - */ - private Pair> moveQueueUsingMulti(String znode, String peerId) { - try { - // hbase/replication/rs/deadrs - String deadRSZnodePath = ZNodePaths.joinZNode(this.queuesZNode, znode); - List listOfOps = new ArrayList<>(); - ReplicationQueueInfo replicationQueueInfo = new ReplicationQueueInfo(peerId); - - String newPeerId = peerId + "-" + znode; - String newPeerZnode = ZNodePaths.joinZNode(this.myQueuesZnode, newPeerId); - // check the logs queue for the old peer cluster - String oldClusterZnode = ZNodePaths.joinZNode(deadRSZnodePath, peerId); - List wals = ZKUtil.listChildrenNoWatch(this.zookeeper, oldClusterZnode); - - if (!peerExists(replicationQueueInfo.getPeerId())) { - LOG.warn("Peer " + replicationQueueInfo.getPeerId() + - " didn't exist, will move its queue to avoid the failure of multi op"); - for (String wal : wals) { - String oldWalZnode = ZNodePaths.joinZNode(oldClusterZnode, wal); - listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldWalZnode)); - } - listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldClusterZnode)); - ZKUtil.multiOrSequential(this.zookeeper, listOfOps, false); - return null; - } - - SortedSet logQueue = new TreeSet<>(); - if (wals == null || wals.isEmpty()) { - listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldClusterZnode)); - } else { - // create the new cluster znode - ZKUtilOp op = ZKUtilOp.createAndFailSilent(newPeerZnode, HConstants.EMPTY_BYTE_ARRAY); - listOfOps.add(op); - // get the offset of the logs and set it to new znodes - for (String wal : wals) { - String oldWalZnode = ZNodePaths.joinZNode(oldClusterZnode, wal); - byte[] logOffset = ZKUtil.getData(this.zookeeper, oldWalZnode); - LOG.debug("Creating " + wal + " with data " + Bytes.toString(logOffset)); - String newLogZnode = ZNodePaths.joinZNode(newPeerZnode, wal); - listOfOps.add(ZKUtilOp.createAndFailSilent(newLogZnode, logOffset)); - listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldWalZnode)); - logQueue.add(wal); - } - // add delete op for peer - listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldClusterZnode)); - - if (LOG.isTraceEnabled()) { - LOG.trace(" The multi list size is: " + listOfOps.size()); - } - } - ZKUtil.multiOrSequential(this.zookeeper, listOfOps, false); - - LOG.info("Atomically moved " + znode + "/" + peerId + "'s WALs to my queue"); - return new Pair<>(newPeerId, logQueue); - } catch (KeeperException e) { - // Multi call failed; it looks like some other regionserver took away the logs. - LOG.warn("Got exception in copyQueuesFromRSUsingMulti: ", e); - } catch (InterruptedException e) { - LOG.warn("Got exception in copyQueuesFromRSUsingMulti: ", e); - Thread.currentThread().interrupt(); - } - return null; - } - - @Override - public void addHFileRefs(String peerId, List> pairs) - throws ReplicationException { - String peerZnode = ZNodePaths.joinZNode(this.hfileRefsZNode, peerId); - boolean debugEnabled = LOG.isDebugEnabled(); - if (debugEnabled) { - LOG.debug("Adding hfile references " + pairs + " in queue " + peerZnode); - } - - int size = pairs.size(); - List listOfOps = new ArrayList<>(size); - - for (int i = 0; i < size; i++) { - listOfOps.add(ZKUtilOp.createAndFailSilent( - ZNodePaths.joinZNode(peerZnode, pairs.get(i).getSecond().getName()), - HConstants.EMPTY_BYTE_ARRAY)); - } - if (debugEnabled) { - LOG.debug(" The multi list size for adding hfile references in zk for node " + peerZnode - + " is " + listOfOps.size()); - } - try { - ZKUtil.multiOrSequential(this.zookeeper, listOfOps, true); - } catch (KeeperException e) { - throw new ReplicationException("Failed to create hfile reference znode=" + e.getPath(), e); - } - } - - @Override - public void removeHFileRefs(String peerId, List files) { - String peerZnode = ZNodePaths.joinZNode(this.hfileRefsZNode, peerId); - boolean debugEnabled = LOG.isDebugEnabled(); - if (debugEnabled) { - LOG.debug("Removing hfile references " + files + " from queue " + peerZnode); - } - - int size = files.size(); - List listOfOps = new ArrayList<>(size); - - for (int i = 0; i < size; i++) { - listOfOps.add(ZKUtilOp.deleteNodeFailSilent(ZNodePaths.joinZNode(peerZnode, files.get(i)))); - } - if (debugEnabled) { - LOG.debug(" The multi list size for removing hfile references in zk for node " + peerZnode - + " is " + listOfOps.size()); - } - try { - ZKUtil.multiOrSequential(this.zookeeper, listOfOps, true); - } catch (KeeperException e) { - LOG.error("Failed to remove hfile reference znode=" + e.getPath(), e); - } - } - - @Override - public void addPeerToHFileRefs(String peerId) throws ReplicationException { - String peerZnode = ZNodePaths.joinZNode(this.hfileRefsZNode, peerId); - try { - if (ZKUtil.checkExists(this.zookeeper, peerZnode) == -1) { - LOG.info("Adding peer " + peerId + " to hfile reference queue."); - ZKUtil.createWithParents(this.zookeeper, peerZnode); - } - } catch (KeeperException e) { - throw new ReplicationException("Failed to add peer " + peerId + " to hfile reference queue.", - e); - } - } - - @Override - public void removePeerFromHFileRefs(String peerId) { - final String peerZnode = ZNodePaths.joinZNode(this.hfileRefsZNode, peerId); - try { - if (ZKUtil.checkExists(this.zookeeper, peerZnode) == -1) { - if (LOG.isDebugEnabled()) { - LOG.debug("Peer " + peerZnode + " not found in hfile reference queue."); - } - return; - } else { - LOG.info("Removing peer " + peerZnode + " from hfile reference queue."); - ZKUtil.deleteNodeRecursively(this.zookeeper, peerZnode); - } - } catch (KeeperException e) { - LOG.error("Ignoring the exception to remove peer " + peerId + " from hfile reference queue.", - e); - } - } -} diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java deleted file mode 100644 index f28a0de..0000000 --- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java +++ /dev/null @@ -1,156 +0,0 @@ -/* - * - * 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; - -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.util.List; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.Abortable; -import org.apache.hadoop.hbase.zookeeper.ZKConfig; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; -import org.apache.hadoop.hbase.zookeeper.ZKWatcher; -import org.apache.hadoop.hbase.zookeeper.ZNodePaths; -import org.apache.yetus.audience.InterfaceAudience; -import org.apache.zookeeper.KeeperException; - -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; -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; - -/** - * This is a base class for maintaining replication state in zookeeper. - */ -@InterfaceAudience.Private -public abstract class ReplicationStateZKBase { - - /** - * The name of the znode that contains the replication status of a remote slave (i.e. peer) - * cluster. - */ - protected final String peerStateNodeName; - /** The name of the base znode that contains all replication state. */ - protected final String replicationZNode; - /** The name of the znode that contains a list of all remote slave (i.e. peer) clusters. */ - protected final String peersZNode; - /** The name of the znode that contains all replication queues */ - protected final String queuesZNode; - /** The name of the znode that contains queues of hfile references to be replicated */ - protected final String hfileRefsZNode; - /** The cluster key of the local cluster */ - protected final String ourClusterKey; - /** The name of the znode that contains tableCFs */ - protected final String tableCFsNodeName; - - protected final ZKWatcher zookeeper; - protected final Configuration conf; - protected final Abortable abortable; - - // Public for testing - public static final byte[] ENABLED_ZNODE_BYTES = - toByteArray(ReplicationProtos.ReplicationState.State.ENABLED); - public static final byte[] DISABLED_ZNODE_BYTES = - toByteArray(ReplicationProtos.ReplicationState.State.DISABLED); - public static final String ZOOKEEPER_ZNODE_REPLICATION_HFILE_REFS_KEY = - "zookeeper.znode.replication.hfile.refs"; - public static final String ZOOKEEPER_ZNODE_REPLICATION_HFILE_REFS_DEFAULT = "hfile-refs"; - - public ReplicationStateZKBase(ZKWatcher zookeeper, Configuration conf, - Abortable abortable) { - this.zookeeper = zookeeper; - this.conf = conf; - this.abortable = abortable; - - String replicationZNodeName = conf.get("zookeeper.znode.replication", "replication"); - String peersZNodeName = conf.get("zookeeper.znode.replication.peers", "peers"); - String queuesZNodeName = conf.get("zookeeper.znode.replication.rs", "rs"); - String hfileRefsZNodeName = conf.get(ZOOKEEPER_ZNODE_REPLICATION_HFILE_REFS_KEY, - ZOOKEEPER_ZNODE_REPLICATION_HFILE_REFS_DEFAULT); - this.peerStateNodeName = conf.get("zookeeper.znode.replication.peers.state", "peer-state"); - this.tableCFsNodeName = conf.get("zookeeper.znode.replication.peers.tableCFs", "tableCFs"); - this.ourClusterKey = ZKConfig.getZooKeeperClusterKey(this.conf); - this.replicationZNode = ZNodePaths.joinZNode(this.zookeeper.znodePaths.baseZNode, - replicationZNodeName); - this.peersZNode = ZNodePaths.joinZNode(replicationZNode, peersZNodeName); - this.queuesZNode = ZNodePaths.joinZNode(replicationZNode, queuesZNodeName); - this.hfileRefsZNode = ZNodePaths.joinZNode(replicationZNode, hfileRefsZNodeName); - } - - public List getListOfReplicators() { - List result = null; - try { - result = ZKUtil.listChildrenNoWatch(this.zookeeper, this.queuesZNode); - } catch (KeeperException e) { - this.abortable.abort("Failed to get list of replicators", e); - } - return result; - } - - /** - * @param state the state to convert into a byte array - * @return Serialized protobuf of state with pb magic prefix prepended suitable for - * use as content of a peer-state znode under a peer cluster id as in - * /hbase/replication/peers/PEER_ID/peer-state. - */ - protected static byte[] toByteArray(final ReplicationProtos.ReplicationState.State state) { - ReplicationProtos.ReplicationState msg = - ReplicationProtos.ReplicationState.newBuilder().setState(state).build(); - // There is no toByteArray on this pb Message? - // 32 bytes is default which seems fair enough here. - try (ByteArrayOutputStream baos = new ByteArrayOutputStream()) { - CodedOutputStream cos = CodedOutputStream.newInstance(baos, 16); - msg.writeTo(cos); - cos.flush(); - baos.flush(); - return ProtobufUtil.prependPBMagic(baos.toByteArray()); - } catch (IOException e) { - throw new RuntimeException(e); - } - } - - protected boolean peerExists(String id) throws KeeperException { - return ZKUtil.checkExists(this.zookeeper, ZNodePaths.joinZNode(this.peersZNode, id)) >= 0; - } - - /** - * Determine if a ZK path points to a peer node. - * @param path path to be checked - * @return true if the path points to a peer node, otherwise false - */ - protected boolean isPeerPath(String path) { - return path.split("/").length == peersZNode.split("/").length + 1; - } - - @VisibleForTesting - protected String getTableCFsNode(String id) { - return ZNodePaths.joinZNode(this.peersZNode, ZNodePaths.joinZNode(id, this.tableCFsNodeName)); - } - - @VisibleForTesting - protected String getPeerStateNode(String id) { - return ZNodePaths.joinZNode(this.peersZNode, ZNodePaths.joinZNode(id, this.peerStateNodeName)); - } - @VisibleForTesting - protected String getPeerNode(String id) { - return ZNodePaths.joinZNode(this.peersZNode, id); - } -} diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStorageFactory.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStorageFactory.java new file mode 100644 index 0000000..462cfed --- /dev/null +++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStorageFactory.java @@ -0,0 +1,49 @@ +/** + * 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; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.zookeeper.ZKWatcher; +import org.apache.yetus.audience.InterfaceAudience; + +/** + * Used to create replication storage(peer, queue) classes. + *

+ * For now we only have zk based implementation. + */ +@InterfaceAudience.Private +public final class ReplicationStorageFactory { + + private ReplicationStorageFactory() { + } + + /** + * Create a new {@link ReplicationPeerStorage}. + */ + public static ReplicationPeerStorage getReplicationPeerStorage(ZKWatcher zk, Configuration conf) { + return new ZKReplicationPeerStorage(zk, conf); + } + + /** + * Create a new {@link ReplicationQueueStorage}. + */ + public static ReplicationQueueStorage getReplicationQueueStorage(ZKWatcher zk, + Configuration conf) { + return new ZKReplicationQueueStorage(zk, conf); + } +} diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationTableBase.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationTableBase.java deleted file mode 100644 index 6aae08d..0000000 --- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationTableBase.java +++ /dev/null @@ -1,441 +0,0 @@ -/* -* -* 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; - -import java.io.IOException; -import java.io.InterruptedIOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.Executor; -import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.ThreadPoolExecutor; -import java.util.concurrent.TimeUnit; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.Abortable; -import org.apache.hadoop.hbase.CompareOperator; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.NamespaceDescriptor; -import org.apache.hadoop.hbase.TableExistsException; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.client.Admin; -import org.apache.hadoop.hbase.client.Connection; -import org.apache.hadoop.hbase.client.ConnectionFactory; -import org.apache.hadoop.hbase.client.Get; -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.client.Table; -import org.apache.hadoop.hbase.filter.SingleColumnValueFilter; -import org.apache.hadoop.hbase.regionserver.BloomType; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.RetryCounter; -import org.apache.hadoop.hbase.util.RetryCounterFactory; -import org.apache.yetus.audience.InterfaceAudience; - -import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; - -/* - * Abstract class that provides an interface to the Replication Table. Which is currently - * being used for WAL offset tracking. - * The basic schema of this table will store each individual queue as a - * seperate row. The row key will be a unique identifier of the creating server's name and the - * queueId. Each queue must have the following two columns: - * COL_QUEUE_OWNER: tracks which server is currently responsible for tracking the queue - * COL_QUEUE_OWNER_HISTORY: a "|" delimited list of the previous server's that have owned this - * queue. The most recent previous owner is the leftmost entry. - * They will also have columns mapping [WAL filename : offset] - * The most flexible method of interacting with the Replication Table is by calling - * getOrBlockOnReplicationTable() which will return a new copy of the Replication Table. It is up - * to the caller to close the returned table. - */ -@InterfaceAudience.Private -abstract class ReplicationTableBase { - - /** Name of the HBase Table used for tracking replication*/ - public static final TableName REPLICATION_TABLE_NAME = - TableName.valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "replication"); - - // Column family and column names for Queues in the Replication Table - public static final byte[] CF_QUEUE = Bytes.toBytes("q"); - public static final byte[] COL_QUEUE_OWNER = Bytes.toBytes("o"); - public static final byte[] COL_QUEUE_OWNER_HISTORY = Bytes.toBytes("h"); - - // Column Descriptor for the Replication Table - private static final HColumnDescriptor REPLICATION_COL_DESCRIPTOR = - new HColumnDescriptor(CF_QUEUE).setMaxVersions(1) - .setInMemory(true) - .setScope(HConstants.REPLICATION_SCOPE_LOCAL) - // TODO: Figure out which bloom filter to use - .setBloomFilterType(BloomType.NONE); - - // The value used to delimit the queueId and server name inside of a queue's row key. Currently a - // hyphen, because it is guaranteed that queueId (which is a cluster id) cannot contain hyphens. - // See HBASE-11394. - public static final String ROW_KEY_DELIMITER = "-"; - - // The value used to delimit server names in the queue history list - public static final String QUEUE_HISTORY_DELIMITER = "|"; - - /* - * Make sure that HBase table operations for replication have a high number of retries. This is - * because the server is aborted if any HBase table operation fails. Each RPC will be attempted - * 3600 times before exiting. This provides each operation with 2 hours of retries - * before the server is aborted. - */ - private static final int CLIENT_RETRIES = 3600; - private static final int RPC_TIMEOUT = 2000; - private static final int OPERATION_TIMEOUT = CLIENT_RETRIES * RPC_TIMEOUT; - - // We only need a single thread to initialize the Replication Table - private static final int NUM_INITIALIZE_WORKERS = 1; - - protected final Configuration conf; - protected final Abortable abortable; - private final Connection connection; - private final Executor executor; - private volatile CountDownLatch replicationTableInitialized; - - public ReplicationTableBase(Configuration conf, Abortable abort) throws IOException { - this.conf = new Configuration(conf); - this.abortable = abort; - decorateConf(); - this.connection = ConnectionFactory.createConnection(this.conf); - this.executor = setUpExecutor(); - this.replicationTableInitialized = new CountDownLatch(1); - createReplicationTableInBackground(); - } - - /** - * Modify the connection's config so that operations run on the Replication Table have longer and - * a larger number of retries - */ - private void decorateConf() { - this.conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, CLIENT_RETRIES); - } - - /** - * Sets up the thread pool executor used to build the Replication Table in the background - * @return the configured executor - */ - private Executor setUpExecutor() { - ThreadPoolExecutor tempExecutor = new ThreadPoolExecutor(NUM_INITIALIZE_WORKERS, - NUM_INITIALIZE_WORKERS, 100, TimeUnit.MILLISECONDS, new LinkedBlockingQueue<>()); - ThreadFactoryBuilder tfb = new ThreadFactoryBuilder(); - tfb.setNameFormat("ReplicationTableExecutor-%d"); - tfb.setDaemon(true); - tempExecutor.setThreadFactory(tfb.build()); - return tempExecutor; - } - - /** - * Get whether the Replication Table has been successfully initialized yet - * @return whether the Replication Table is initialized - */ - public boolean getInitializationStatus() { - return replicationTableInitialized.getCount() == 0; - } - - /** - * Increases the RPC and operations timeouts for the Replication Table - */ - private Table setReplicationTableTimeOuts(Table replicationTable) { - replicationTable.setRpcTimeout(RPC_TIMEOUT); - replicationTable.setOperationTimeout(OPERATION_TIMEOUT); - return replicationTable; - } - - /** - * Build the row key for the given queueId. This will uniquely identify it from all other queues - * in the cluster. - * @param serverName The owner of the queue - * @param queueId String identifier of the queue - * @return String representation of the queue's row key - */ - protected String buildQueueRowKey(String serverName, String queueId) { - return queueId + ROW_KEY_DELIMITER + serverName; - } - - /** - * Parse the original queueId from a row key - * @param rowKey String representation of a queue's row key - * @return the original queueId - */ - protected String getRawQueueIdFromRowKey(String rowKey) { - return rowKey.split(ROW_KEY_DELIMITER)[0]; - } - - /** - * Returns a queue's row key given either its raw or reclaimed queueId - * - * @param queueId queueId of the queue - * @return byte representation of the queue's row key - */ - protected byte[] queueIdToRowKey(String serverName, String queueId) { - // Cluster id's are guaranteed to have no hyphens, so if the passed in queueId has no hyphen - // then this is not a reclaimed queue. - if (!queueId.contains(ROW_KEY_DELIMITER)) { - return Bytes.toBytes(buildQueueRowKey(serverName, queueId)); - // If the queueId contained some hyphen it was reclaimed. In this case, the queueId is the - // queue's row key - } else { - return Bytes.toBytes(queueId); - } - } - - /** - * Creates a "|" delimited record of the queue's past region server owners. - * - * @param originalHistory the queue's original owner history - * @param oldServer the name of the server that used to own the queue - * @return the queue's new owner history - */ - protected String buildClaimedQueueHistory(String originalHistory, String oldServer) { - return oldServer + QUEUE_HISTORY_DELIMITER + originalHistory; - } - - /** - * Get a list of all region servers that have outstanding replication queues. These servers could - * be alive, dead or from a previous run of the cluster. - * @return a list of server names - */ - protected List getListOfReplicators() { - // scan all of the queues and return a list of all unique OWNER values - Set peerServers = new HashSet<>(); - ResultScanner allQueuesInCluster = null; - try (Table replicationTable = getOrBlockOnReplicationTable()){ - Scan scan = new Scan(); - scan.addColumn(CF_QUEUE, COL_QUEUE_OWNER); - allQueuesInCluster = replicationTable.getScanner(scan); - for (Result queue : allQueuesInCluster) { - peerServers.add(Bytes.toString(queue.getValue(CF_QUEUE, COL_QUEUE_OWNER))); - } - } catch (IOException e) { - String errMsg = "Failed getting list of replicators"; - abortable.abort(errMsg, e); - } finally { - if (allQueuesInCluster != null) { - allQueuesInCluster.close(); - } - } - return new ArrayList<>(peerServers); - } - - protected List getAllQueues(String serverName) { - List allQueues = new ArrayList<>(); - ResultScanner queueScanner = null; - try { - queueScanner = getQueuesBelongingToServer(serverName); - for (Result queue : queueScanner) { - String rowKey = Bytes.toString(queue.getRow()); - // If the queue does not have a Owner History, then we must be its original owner. So we - // want to return its queueId in raw form - if (Bytes.toString(queue.getValue(CF_QUEUE, COL_QUEUE_OWNER_HISTORY)).length() == 0) { - allQueues.add(getRawQueueIdFromRowKey(rowKey)); - } else { - allQueues.add(rowKey); - } - } - return allQueues; - } catch (IOException e) { - String errMsg = "Failed getting list of all replication queues for serverName=" + serverName; - abortable.abort(errMsg, e); - return null; - } finally { - if (queueScanner != null) { - queueScanner.close(); - } - } - } - - protected List getLogsInQueue(String serverName, String queueId) { - String rowKey = queueId; - if (!queueId.contains(ROW_KEY_DELIMITER)) { - rowKey = buildQueueRowKey(serverName, queueId); - } - return getLogsInQueue(Bytes.toBytes(rowKey)); - } - - protected List getLogsInQueue(byte[] rowKey) { - String errMsg = "Failed getting logs in queue queueId=" + Bytes.toString(rowKey); - try (Table replicationTable = getOrBlockOnReplicationTable()) { - Get getQueue = new Get(rowKey); - Result queue = replicationTable.get(getQueue); - if (queue == null || queue.isEmpty()) { - abortable.abort(errMsg, new ReplicationException(errMsg)); - return null; - } - return readWALsFromResult(queue); - } catch (IOException e) { - abortable.abort(errMsg, e); - return null; - } - } - - /** - * Read all of the WAL's from a queue into a list - * - * @param queue HBase query result containing the queue - * @return a list of all the WAL filenames - */ - protected List readWALsFromResult(Result queue) { - List wals = new ArrayList<>(); - Map familyMap = queue.getFamilyMap(CF_QUEUE); - for (byte[] cQualifier : familyMap.keySet()) { - // Ignore the meta data fields of the queue - if (Arrays.equals(cQualifier, COL_QUEUE_OWNER) || Arrays.equals(cQualifier, - COL_QUEUE_OWNER_HISTORY)) { - continue; - } - wals.add(Bytes.toString(cQualifier)); - } - return wals; - } - - /** - * Get the queue id's and meta data (Owner and History) for the queues belonging to the named - * server - * - * @param server name of the server - * @return a ResultScanner over the QueueIds belonging to the server - * @throws IOException if getting the table or the scanner fails - */ - protected ResultScanner getQueuesBelongingToServer(String server) throws IOException { - Scan scan = new Scan(); - SingleColumnValueFilter filterMyQueues = new SingleColumnValueFilter(CF_QUEUE, COL_QUEUE_OWNER, - CompareOperator.EQUAL, Bytes.toBytes(server)); - scan.setFilter(filterMyQueues); - scan.addColumn(CF_QUEUE, COL_QUEUE_OWNER); - scan.addColumn(CF_QUEUE, COL_QUEUE_OWNER_HISTORY); - try (Table replicationTable = getOrBlockOnReplicationTable()) { - ResultScanner results = replicationTable.getScanner(scan); - return results; - } - } - - /** - * Attempts to acquire the Replication Table. This operation will block until it is assigned by - * the CreateReplicationWorker thread. It is up to the caller of this method to close the - * returned Table - * @return the Replication Table when it is created - * @throws IOException if getting the table or the scanner fails - */ - protected Table getOrBlockOnReplicationTable() throws IOException { - // Sleep until the Replication Table becomes available - try { - replicationTableInitialized.await(); - } catch (InterruptedException e) { - String errMsg = "Unable to acquire the Replication Table due to InterruptedException: " + - e.getMessage(); - throw new InterruptedIOException(errMsg); - } - return getAndSetUpReplicationTable(); - } - - /** - * Creates a new copy of the Replication Table and sets up the proper Table time outs for it - * - * @return the Replication Table - * @throws IOException if getting the table fails - */ - private Table getAndSetUpReplicationTable() throws IOException { - Table replicationTable = connection.getTable(REPLICATION_TABLE_NAME); - setReplicationTableTimeOuts(replicationTable); - return replicationTable; - } - - /** - * Builds the Replication Table in a background thread. Any method accessing the Replication Table - * should do so through getOrBlockOnReplicationTable() - * - * @return the Replication Table - * @throws IOException if the Replication Table takes too long to build - */ - private void createReplicationTableInBackground() throws IOException { - executor.execute(new CreateReplicationTableWorker()); - } - - /** - * Attempts to build the Replication Table. Will continue blocking until we have a valid - * Table for the Replication Table. - */ - private class CreateReplicationTableWorker implements Runnable { - - private Admin admin; - - @Override - public void run() { - try { - admin = connection.getAdmin(); - if (!replicationTableExists()) { - createReplicationTable(); - } - int maxRetries = conf.getInt("hbase.replication.queues.createtable.retries.number", - CLIENT_RETRIES); - RetryCounterFactory counterFactory = new RetryCounterFactory(maxRetries, RPC_TIMEOUT); - RetryCounter retryCounter = counterFactory.create(); - while (!replicationTableExists()) { - retryCounter.sleepUntilNextRetry(); - if (!retryCounter.shouldRetry()) { - throw new IOException("Unable to acquire the Replication Table"); - } - } - replicationTableInitialized.countDown(); - } catch (IOException | InterruptedException e) { - abortable.abort("Failed building Replication Table", e); - } - } - - /** - * Create the replication table with the provided HColumnDescriptor REPLICATION_COL_DESCRIPTOR - * in TableBasedReplicationQueuesImpl - * - * @throws IOException if creating the table fails - */ - private void createReplicationTable() throws IOException { - HTableDescriptor replicationTableDescriptor = new HTableDescriptor(REPLICATION_TABLE_NAME); - replicationTableDescriptor.addFamily(REPLICATION_COL_DESCRIPTOR); - try { - admin.createTable(replicationTableDescriptor); - } catch (TableExistsException e) { - // In this case we can just continue as normal - } - } - - /** - * Checks whether the Replication Table exists yet - * - * @return whether the Replication Table exists - */ - private boolean replicationTableExists() { - try { - return admin.tableExists(REPLICATION_TABLE_NAME); - } catch (IOException e) { - return false; - } - } - } -} diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationTrackerZKImpl.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationTrackerZKImpl.java index 848052c..16a1668 100644 --- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationTrackerZKImpl.java +++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationTrackerZKImpl.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -21,8 +20,6 @@ package org.apache.hadoop.hbase.replication; import java.util.ArrayList; import java.util.List; import java.util.concurrent.CopyOnWriteArrayList; - -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Abortable; import org.apache.hadoop.hbase.Stoppable; import org.apache.hadoop.hbase.zookeeper.ZKListener; @@ -39,25 +36,28 @@ import org.slf4j.LoggerFactory; * interface. */ @InterfaceAudience.Private -public class ReplicationTrackerZKImpl extends ReplicationStateZKBase implements ReplicationTracker { +public class ReplicationTrackerZKImpl implements ReplicationTracker { private static final Logger LOG = LoggerFactory.getLogger(ReplicationTrackerZKImpl.class); + + // Zookeeper + private final ZKWatcher zookeeper; + // Server to abort. + private final Abortable abortable; // All about stopping private final Stoppable stopper; // listeners to be notified private final List listeners = new CopyOnWriteArrayList<>(); // List of all the other region servers in this cluster private final ArrayList otherRegionServers = new ArrayList<>(); - private final ReplicationPeers replicationPeers; - public ReplicationTrackerZKImpl(ZKWatcher zookeeper, - final ReplicationPeers replicationPeers, Configuration conf, Abortable abortable, - Stoppable stopper) { - super(zookeeper, conf, abortable); - this.replicationPeers = replicationPeers; + public ReplicationTrackerZKImpl(ZKWatcher zookeeper, Abortable abortable, Stoppable stopper) { + this.zookeeper = zookeeper; + this.abortable = abortable; this.stopper = stopper; this.zookeeper.registerListener(new OtherRegionServerWatcher(this.zookeeper)); - this.zookeeper.registerListener(new PeersWatcher(this.zookeeper)); + // watch the changes + refreshOtherRegionServersList(true); } @Override @@ -75,7 +75,7 @@ public class ReplicationTrackerZKImpl extends ReplicationStateZKBase implements */ @Override public List getListOfRegionServers() { - refreshOtherRegionServersList(); + refreshOtherRegionServersList(false); List list = null; synchronized (otherRegionServers) { @@ -141,76 +141,11 @@ public class ReplicationTrackerZKImpl extends ReplicationStateZKBase implements if (!path.startsWith(this.watcher.znodePaths.rsZNode)) { return false; } - return refreshOtherRegionServersList(); + return refreshOtherRegionServersList(true); } } /** - * Watcher used to follow the creation and deletion of peer clusters. - */ - public class PeersWatcher extends ZKListener { - - /** - * Construct a ZooKeeper event listener. - */ - public PeersWatcher(ZKWatcher watcher) { - super(watcher); - } - - /** - * Called when a node has been deleted - * @param path full path of the deleted node - */ - @Override - public void nodeDeleted(String path) { - List peers = refreshPeersList(path); - if (peers == null) { - return; - } - if (isPeerPath(path)) { - String id = getZNodeName(path); - LOG.info(path + " znode expired, triggering peerRemoved event"); - for (ReplicationListener rl : listeners) { - rl.peerRemoved(id); - } - } - } - - /** - * Called when an existing node has a child node added or removed. - * @param path full path of the node whose children have changed - */ - @Override - public void nodeChildrenChanged(String path) { - List peers = refreshPeersList(path); - if (peers == null) { - return; - } - LOG.info(path + " znode expired, triggering peerListChanged event"); - for (ReplicationListener rl : listeners) { - rl.peerListChanged(peers); - } - } - } - - /** - * Verify if this event is meant for us, and if so then get the latest peers' list from ZK. Also - * reset the watches. - * @param path path to check against - * @return A list of peers' identifiers if the event concerns this watcher, else null. - */ - private List refreshPeersList(String path) { - if (!path.startsWith(getPeersZNode())) { - return null; - } - return this.replicationPeers.getAllPeerIds(); - } - - private String getPeersZNode() { - return this.peersZNode; - } - - /** * Extracts the znode name of a peer cluster from a ZK path * @param fullPath Path to extract the id from * @return the id or an empty string if path is invalid @@ -226,8 +161,8 @@ public class ReplicationTrackerZKImpl extends ReplicationStateZKBase implements * @return true if the local list of the other region servers was updated with the ZK data (even * if it was empty), false if the data was missing in ZK */ - private boolean refreshOtherRegionServersList() { - List newRsList = getRegisteredRegionServers(); + private boolean refreshOtherRegionServersList(boolean watch) { + List newRsList = getRegisteredRegionServers(watch); if (newRsList == null) { return false; } else { @@ -243,10 +178,14 @@ public class ReplicationTrackerZKImpl extends ReplicationStateZKBase implements * Get a list of all the other region servers in this cluster and set a watch * @return a list of server nanes */ - private List getRegisteredRegionServers() { + private List getRegisteredRegionServers(boolean watch) { List result = null; try { - result = ZKUtil.listChildrenAndWatchThem(this.zookeeper, this.zookeeper.znodePaths.rsZNode); + if (watch) { + result = ZKUtil.listChildrenAndWatchThem(this.zookeeper, this.zookeeper.znodePaths.rsZNode); + } else { + result = ZKUtil.listChildrenNoWatch(this.zookeeper, this.zookeeper.znodePaths.rsZNode); + } } catch (KeeperException e) { this.abortable.abort("Get list of registered region servers", e); } diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationUtils.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationUtils.java new file mode 100644 index 0000000..ebe68a7 --- /dev/null +++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationUtils.java @@ -0,0 +1,125 @@ +/** + * 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; + +import java.io.IOException; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.Set; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.CompoundConfiguration; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.TableName; +import org.apache.yetus.audience.InterfaceAudience; + +/** + * Helper class for replication. + */ +@InterfaceAudience.Private +public final class ReplicationUtils { + + private ReplicationUtils() { + } + + public static Configuration getPeerClusterConfiguration(ReplicationPeerConfig peerConfig, + Configuration baseConf) throws ReplicationException { + Configuration otherConf; + try { + otherConf = HBaseConfiguration.createClusterConf(baseConf, peerConfig.getClusterKey()); + } catch (IOException e) { + throw new ReplicationException("Can't get peer configuration for peer " + peerConfig, e); + } + + if (!peerConfig.getConfiguration().isEmpty()) { + CompoundConfiguration compound = new CompoundConfiguration(); + compound.add(otherConf); + compound.addStringMap(peerConfig.getConfiguration()); + return compound; + } + + return otherConf; + } + + public static void removeAllQueues(ReplicationQueueStorage queueStorage, String peerId) + throws ReplicationException { + for (ServerName replicator : queueStorage.getListOfReplicators()) { + List 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); + } + } + + private static boolean isCollectionEqual(Collection c1, Collection c2) { + if (c1 == null) { + return c2 == null; + } + if (c2 == null) { + return false; + } + return c1.size() == c2.size() && c1.containsAll(c2); + } + + private static boolean isNamespacesEqual(Set ns1, Set ns2) { + return isCollectionEqual(ns1, ns2); + } + + private static boolean isTableCFsEqual(Map> tableCFs1, + Map> tableCFs2) { + if (tableCFs1 == null) { + return tableCFs2 == null; + } + if (tableCFs2 == null) { + return false; + } + if (tableCFs1.size() != tableCFs2.size()) { + return false; + } + for (Map.Entry> entry1 : tableCFs1.entrySet()) { + TableName table = entry1.getKey(); + if (!tableCFs2.containsKey(table)) { + return false; + } + List cfs1 = entry1.getValue(); + List cfs2 = tableCFs2.get(table); + if (!isCollectionEqual(cfs1, cfs2)) { + return false; + } + } + return true; + } + + public static boolean isKeyConfigEqual(ReplicationPeerConfig rpc1, ReplicationPeerConfig rpc2) { + if (rpc1.replicateAllUserTables() != rpc2.replicateAllUserTables()) { + return false; + } + if (rpc1.replicateAllUserTables()) { + return isNamespacesEqual(rpc1.getExcludeNamespaces(), rpc2.getExcludeNamespaces()) && + isTableCFsEqual(rpc1.getExcludeTableCFsMap(), rpc2.getExcludeTableCFsMap()); + } else { + return isNamespacesEqual(rpc1.getNamespaces(), rpc2.getNamespaces()) && + isTableCFsEqual(rpc1.getTableCFsMap(), rpc2.getTableCFsMap()); + } + } +} diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationPeerStorage.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationPeerStorage.java new file mode 100644 index 0000000..a53500a --- /dev/null +++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationPeerStorage.java @@ -0,0 +1,169 @@ +/** + * 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; + +import java.util.Arrays; +import java.util.List; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil; +import org.apache.hadoop.hbase.exceptions.DeserializationException; +import org.apache.hadoop.hbase.util.CollectionUtils; +import org.apache.hadoop.hbase.zookeeper.ZKUtil; +import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp; +import org.apache.hadoop.hbase.zookeeper.ZKWatcher; +import org.apache.hadoop.hbase.zookeeper.ZNodePaths; +import org.apache.yetus.audience.InterfaceAudience; +import org.apache.zookeeper.KeeperException; +import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; + +import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos; + +/** + * ZK based replication peer storage. + */ +@InterfaceAudience.Private +public class ZKReplicationPeerStorage extends ZKReplicationStorageBase + implements ReplicationPeerStorage { + + public static final String PEERS_ZNODE = "zookeeper.znode.replication.peers"; + public static final String PEERS_ZNODE_DEFAULT = "peers"; + + public static final String PEERS_STATE_ZNODE = "zookeeper.znode.replication.peers.state"; + public static final String PEERS_STATE_ZNODE_DEFAULT = "peer-state"; + + public static final byte[] ENABLED_ZNODE_BYTES = + toByteArray(ReplicationProtos.ReplicationState.State.ENABLED); + public static final byte[] DISABLED_ZNODE_BYTES = + toByteArray(ReplicationProtos.ReplicationState.State.DISABLED); + + /** + * The name of the znode that contains the replication status of a remote slave (i.e. peer) + * cluster. + */ + private final String peerStateNodeName; + + /** + * The name of the znode that contains a list of all remote slave (i.e. peer) clusters. + */ + private final String peersZNode; + + public ZKReplicationPeerStorage(ZKWatcher zookeeper, Configuration conf) { + super(zookeeper, conf); + this.peerStateNodeName = conf.get(PEERS_STATE_ZNODE, PEERS_STATE_ZNODE_DEFAULT); + String peersZNodeName = conf.get(PEERS_ZNODE, PEERS_ZNODE_DEFAULT); + this.peersZNode = ZNodePaths.joinZNode(replicationZNode, peersZNodeName); + } + + @VisibleForTesting + public String getPeerStateNode(String peerId) { + return ZNodePaths.joinZNode(getPeerNode(peerId), peerStateNodeName); + } + + @VisibleForTesting + public String getPeerNode(String peerId) { + return ZNodePaths.joinZNode(peersZNode, peerId); + } + + @Override + public void addPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled) + throws ReplicationException { + try { + ZKUtil.createWithParents(zookeeper, peersZNode); + ZKUtil.multiOrSequential(zookeeper, + Arrays.asList( + ZKUtilOp.createAndFailSilent(getPeerNode(peerId), + ReplicationPeerConfigUtil.toByteArray(peerConfig)), + ZKUtilOp.createAndFailSilent(getPeerStateNode(peerId), + enabled ? ENABLED_ZNODE_BYTES : DISABLED_ZNODE_BYTES)), + false); + } catch (KeeperException e) { + throw new ReplicationException("Could not add peer with id=" + peerId + ", peerConfif=>" + + peerConfig + ", state=" + (enabled ? "ENABLED" : "DISABLED"), e); + } + } + + @Override + public void removePeer(String peerId) throws ReplicationException { + try { + ZKUtil.deleteNodeRecursively(zookeeper, getPeerNode(peerId)); + } catch (KeeperException e) { + throw new ReplicationException("Could not remove peer with id=" + peerId, e); + } + } + + @Override + public void setPeerState(String peerId, boolean enabled) throws ReplicationException { + byte[] stateBytes = enabled ? ENABLED_ZNODE_BYTES : DISABLED_ZNODE_BYTES; + try { + ZKUtil.setData(zookeeper, getPeerStateNode(peerId), stateBytes); + } catch (KeeperException e) { + throw new ReplicationException("Unable to change state of the peer with id=" + peerId, e); + } + } + + @Override + public void updatePeerConfig(String peerId, ReplicationPeerConfig peerConfig) + throws ReplicationException { + try { + ZKUtil.setData(this.zookeeper, getPeerNode(peerId), + ReplicationPeerConfigUtil.toByteArray(peerConfig)); + } catch (KeeperException e) { + throw new ReplicationException( + "There was a problem trying to save changes to the " + "replication peer " + peerId, e); + } + } + + @Override + public List listPeerIds() throws ReplicationException { + try { + return CollectionUtils.nullToEmpty(ZKUtil.listChildrenNoWatch(zookeeper, peersZNode)); + } catch (KeeperException e) { + throw new ReplicationException("Cannot get the list of peers", e); + } + } + + @Override + public boolean isPeerEnabled(String peerId) throws ReplicationException { + try { + return Arrays.equals(ENABLED_ZNODE_BYTES, + ZKUtil.getData(zookeeper, getPeerStateNode(peerId))); + } catch (KeeperException | InterruptedException e) { + throw new ReplicationException("Unable to get status of the peer with id=" + peerId, e); + } + } + + @Override + public ReplicationPeerConfig getPeerConfig(String peerId) throws ReplicationException { + byte[] data; + try { + data = ZKUtil.getData(zookeeper, getPeerNode(peerId)); + } catch (KeeperException | InterruptedException e) { + throw new ReplicationException("Error getting configuration for peer with id=" + peerId, e); + } + if (data == null || data.length == 0) { + throw new ReplicationException( + "Replication peer config data shouldn't be empty, peerId=" + peerId); + } + try { + return ReplicationPeerConfigUtil.parsePeerFrom(data); + } catch (DeserializationException e) { + throw new ReplicationException( + "Failed to parse replication peer config for peer with id=" + peerId, e); + } + } +} diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java new file mode 100644 index 0000000..ee237f2 --- /dev/null +++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java @@ -0,0 +1,496 @@ +/** + * 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; + +import static java.util.stream.Collectors.toList; +import static org.apache.hadoop.hbase.util.CollectionUtils.nullToEmpty; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashSet; +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; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.exceptions.DeserializationException; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.CollectionUtils; +import org.apache.hadoop.hbase.util.Pair; +import org.apache.hadoop.hbase.zookeeper.ZKUtil; +import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp; +import org.apache.hadoop.hbase.zookeeper.ZKWatcher; +import org.apache.hadoop.hbase.zookeeper.ZNodePaths; +import org.apache.yetus.audience.InterfaceAudience; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.KeeperException.BadVersionException; +import org.apache.zookeeper.KeeperException.NoNodeException; +import org.apache.zookeeper.KeeperException.NodeExistsException; +import org.apache.zookeeper.KeeperException.NotEmptyException; +import org.apache.zookeeper.data.Stat; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; + +/** + * ZK based replication queue storage. + *

+ * The base znode for each regionserver is the regionserver name. For example: + * + *

+ * /hbase/replication/rs/hostname.example.org,6020,1234
+ * 
+ * + * Within this znode, the region server maintains a set of WAL replication queues. These queues are + * represented by child znodes named using there give queue id. For example: + * + *
+ * /hbase/replication/rs/hostname.example.org,6020,1234/1
+ * /hbase/replication/rs/hostname.example.org,6020,1234/2
+ * 
+ * + * Each queue has one child znode for every WAL that still needs to be replicated. The value of + * these WAL child znodes is the latest position that has been replicated. This position is updated + * every time a WAL entry is replicated. For example: + * + *
+ * /hbase/replication/rs/hostname.example.org,6020,1234/1/23522342.23422 [VALUE: 254]
+ * 
+ */ +@InterfaceAudience.Private +class ZKReplicationQueueStorage extends ZKReplicationStorageBase + implements ReplicationQueueStorage { + + private static final Logger LOG = LoggerFactory.getLogger(ZKReplicationQueueStorage.class); + + public static final String ZOOKEEPER_ZNODE_REPLICATION_HFILE_REFS_KEY = + "zookeeper.znode.replication.hfile.refs"; + public static final String ZOOKEEPER_ZNODE_REPLICATION_HFILE_REFS_DEFAULT = "hfile-refs"; + + /** + * The name of the znode that contains all replication queues + */ + private final String queuesZNode; + + /** + * The name of the znode that contains queues of hfile references to be replicated + */ + private final String hfileRefsZNode; + + public ZKReplicationQueueStorage(ZKWatcher zookeeper, Configuration conf) { + super(zookeeper, conf); + + String queuesZNodeName = conf.get("zookeeper.znode.replication.rs", "rs"); + String hfileRefsZNodeName = conf.get(ZOOKEEPER_ZNODE_REPLICATION_HFILE_REFS_KEY, + ZOOKEEPER_ZNODE_REPLICATION_HFILE_REFS_DEFAULT); + this.queuesZNode = ZNodePaths.joinZNode(replicationZNode, queuesZNodeName); + this.hfileRefsZNode = ZNodePaths.joinZNode(replicationZNode, hfileRefsZNodeName); + } + + private String getRsNode(ServerName serverName) { + return ZNodePaths.joinZNode(queuesZNode, serverName.getServerName()); + } + + private String getQueueNode(ServerName serverName, String queueId) { + return ZNodePaths.joinZNode(getRsNode(serverName), queueId); + } + + private String getFileNode(String queueNode, String fileName) { + return ZNodePaths.joinZNode(queueNode, fileName); + } + + private String getFileNode(ServerName serverName, String queueId, String fileName) { + return getFileNode(getQueueNode(serverName, queueId), fileName); + } + + @Override + public void removeQueue(ServerName serverName, String queueId) throws ReplicationException { + try { + ZKUtil.deleteNodeRecursively(zookeeper, getQueueNode(serverName, queueId)); + } catch (KeeperException e) { + throw new ReplicationException( + "Failed to delete queue (serverName=" + serverName + ", queueId=" + queueId + ")", e); + } + } + + @Override + public void addWAL(ServerName serverName, String queueId, String fileName) + throws ReplicationException { + try { + ZKUtil.createWithParents(zookeeper, getFileNode(serverName, queueId, fileName)); + } catch (KeeperException e) { + throw new ReplicationException("Failed to add wal to queue (serverName=" + serverName + + ", queueId=" + queueId + ", fileName=" + fileName + ")", e); + } + } + + @Override + public void removeWAL(ServerName serverName, String queueId, String fileName) + throws ReplicationException { + String fileNode = getFileNode(serverName, queueId, fileName); + try { + ZKUtil.deleteNode(zookeeper, fileNode); + } catch (NoNodeException e) { + LOG.warn(fileNode + " has already been deleted when removing log"); + } catch (KeeperException e) { + throw new ReplicationException("Failed to remove wal from queue (serverName=" + serverName + + ", queueId=" + queueId + ", fileName=" + fileName + ")", e); + } + } + + @Override + public void setWALPosition(ServerName serverName, String queueId, String fileName, long position) + throws ReplicationException { + try { + ZKUtil.setData(zookeeper, getFileNode(serverName, queueId, fileName), + ZKUtil.positionToByteArray(position)); + } catch (KeeperException e) { + throw new ReplicationException("Failed to set log position (serverName=" + serverName + + ", queueId=" + queueId + ", fileName=" + fileName + ", position=" + position + ")", e); + } + } + + @Override + public long getWALPosition(ServerName serverName, String queueId, String fileName) + throws ReplicationException { + byte[] bytes; + try { + bytes = ZKUtil.getData(zookeeper, getFileNode(serverName, queueId, fileName)); + } catch (KeeperException | InterruptedException e) { + throw new ReplicationException("Failed to get log position (serverName=" + serverName + + ", queueId=" + queueId + ", fileName=" + fileName + ")", e); + } + try { + return ZKUtil.parseWALPositionFrom(bytes); + } catch (DeserializationException de) { + LOG.warn("Failed to parse log position (serverName=" + serverName + ", queueId=" + queueId + + ", fileName=" + fileName + ")"); + } + // if we can not parse the position, start at the beginning of the wal file again + return 0; + } + + @Override + public Pair> claimQueue(ServerName sourceServerName, String queueId, + ServerName destServerName) throws ReplicationException { + LOG.info( + "Atomically moving " + sourceServerName + "/" + queueId + "'s WALs to " + destServerName); + try { + ZKUtil.createWithParents(zookeeper, getRsNode(destServerName)); + } catch (KeeperException e) { + throw new ReplicationException( + "Claim queue queueId=" + queueId + " from " + sourceServerName + " to " + destServerName + + " failed when creating the node for " + destServerName, + e); + } + try { + String oldQueueNode = getQueueNode(sourceServerName, queueId); + List wals = ZKUtil.listChildrenNoWatch(zookeeper, oldQueueNode); + String newQueueId = queueId + "-" + sourceServerName; + if (CollectionUtils.isEmpty(wals)) { + ZKUtil.deleteNodeFailSilent(zookeeper, oldQueueNode); + LOG.info("Removed " + sourceServerName + "/" + queueId + " since it's empty"); + return new Pair<>(newQueueId, Collections.emptySortedSet()); + } + String newQueueNode = getQueueNode(destServerName, newQueueId); + List listOfOps = new ArrayList<>(); + SortedSet logQueue = new TreeSet<>(); + // create the new cluster znode + listOfOps.add(ZKUtilOp.createAndFailSilent(newQueueNode, HConstants.EMPTY_BYTE_ARRAY)); + // get the offset of the logs and set it to new znodes + for (String wal : wals) { + String oldWalNode = getFileNode(oldQueueNode, wal); + byte[] logOffset = ZKUtil.getData(this.zookeeper, oldWalNode); + if (LOG.isDebugEnabled()) { + LOG.debug("Creating " + wal + " with data " + Bytes.toStringBinary(logOffset)); + } + String newWalNode = getFileNode(newQueueNode, wal); + listOfOps.add(ZKUtilOp.createAndFailSilent(newWalNode, logOffset)); + listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldWalNode)); + logQueue.add(wal); + } + // add delete op for peer + listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldQueueNode)); + + if (LOG.isTraceEnabled()) { + LOG.trace("The multi list size is: " + listOfOps.size()); + } + ZKUtil.multiOrSequential(zookeeper, listOfOps, false); + + LOG.info( + "Atomically moved " + sourceServerName + "/" + queueId + "'s WALs to " + destServerName); + return new Pair<>(newQueueId, logQueue); + } catch (NoNodeException | NodeExistsException | NotEmptyException | BadVersionException e) { + // Multi call failed; it looks like some other regionserver took away the logs. + // These exceptions mean that zk tells us the request can not be execute so it is safe to just + // return a null. For other types of exception should be thrown out to notify the upper layer. + LOG.info( + "Claim queue queueId=" + queueId + " from " + sourceServerName + " to " + destServerName + + " failed with " + e.toString() + ", maybe someone else has already took away the logs"); + return null; + } catch (KeeperException | InterruptedException e) { + throw new ReplicationException("Claim queue queueId=" + queueId + " from " + + sourceServerName + " to " + destServerName + " failed", e); + } + } + + @Override + public void removeReplicatorIfQueueIsEmpty(ServerName serverName) throws ReplicationException { + try { + ZKUtil.deleteNodeFailSilent(zookeeper, getRsNode(serverName)); + } catch (NotEmptyException e) { + // keep silence to avoid logging too much. + } catch (KeeperException e) { + throw new ReplicationException("Failed to remove replicator for " + serverName, e); + } + } + + private List getListOfReplicators0() throws KeeperException { + return nullToEmpty(ZKUtil.listChildrenNoWatch(zookeeper, queuesZNode)).stream() + .map(ServerName::parseServerName).collect(toList()); + } + + @Override + public List getListOfReplicators() throws ReplicationException { + try { + return getListOfReplicators0(); + } catch (KeeperException e) { + throw new ReplicationException("Failed to get list of replicators", e); + } + } + + private List getWALsInQueue0(ServerName serverName, String queueId) + throws KeeperException { + return nullToEmpty(ZKUtil.listChildrenNoWatch(zookeeper, getQueueNode(serverName, queueId))); + } + + @Override + public List getWALsInQueue(ServerName serverName, String queueId) + throws ReplicationException { + try { + return getWALsInQueue0(serverName, queueId); + } catch (KeeperException e) { + throw new ReplicationException( + "Failed to get wals in queue (serverName=" + serverName + ", queueId=" + queueId + ")", + e); + } + } + + private List getAllQueues0(ServerName serverName) throws KeeperException { + return nullToEmpty(ZKUtil.listChildrenNoWatch(zookeeper, getRsNode(serverName))); + } + + @Override + public List getAllQueues(ServerName serverName) throws ReplicationException { + try { + return getAllQueues0(serverName); + } catch (KeeperException e) { + throw new ReplicationException("Failed to get all queues (serverName=" + serverName + ")", e); + } + } + + // will be overridden in UTs + @VisibleForTesting + protected int getQueuesZNodeCversion() throws KeeperException { + Stat stat = new Stat(); + ZKUtil.getDataNoWatch(this.zookeeper, this.queuesZNode, stat); + return stat.getCversion(); + } + + @Override + public Set getAllWALs() throws ReplicationException { + try { + for (int retry = 0;; retry++) { + int v0 = getQueuesZNodeCversion(); + List rss = getListOfReplicators0(); + if (rss.isEmpty()) { + LOG.debug("Didn't find any region server that replicates, won't prevent any deletions."); + return Collections.emptySet(); + } + Set wals = new HashSet<>(); + for (ServerName rs : rss) { + for (String queueId : getAllQueues0(rs)) { + wals.addAll(getWALsInQueue0(rs, queueId)); + } + } + int v1 = getQueuesZNodeCversion(); + if (v0 == v1) { + return wals; + } + LOG.info(String.format("Replication queue node cversion changed from %d to %d, retry = %d", + v0, v1, retry)); + } + } catch (KeeperException e) { + throw new ReplicationException("Failed to get all wals", e); + } + } + + private String getHFileRefsPeerNode(String peerId) { + return ZNodePaths.joinZNode(hfileRefsZNode, peerId); + } + + private String getHFileNode(String peerNode, String fileName) { + return ZNodePaths.joinZNode(peerNode, fileName); + } + + @Override + public void addPeerToHFileRefs(String peerId) throws ReplicationException { + String peerNode = getHFileRefsPeerNode(peerId); + try { + if (ZKUtil.checkExists(zookeeper, peerNode) == -1) { + LOG.info("Adding peer " + peerId + " to hfile reference queue."); + ZKUtil.createWithParents(zookeeper, peerNode); + } + } catch (KeeperException e) { + throw new ReplicationException("Failed to add peer " + peerId + " to hfile reference queue.", + e); + } + } + + @Override + public void removePeerFromHFileRefs(String peerId) throws ReplicationException { + String peerNode = getHFileRefsPeerNode(peerId); + try { + if (ZKUtil.checkExists(zookeeper, peerNode) == -1) { + if (LOG.isDebugEnabled()) { + LOG.debug("Peer " + peerNode + " not found in hfile reference queue."); + } + } else { + LOG.info("Removing peer " + peerNode + " from hfile reference queue."); + ZKUtil.deleteNodeRecursively(zookeeper, peerNode); + } + } catch (KeeperException e) { + throw new ReplicationException( + "Failed to remove peer " + peerId + " from hfile reference queue.", e); + } + } + + @Override + public void addHFileRefs(String peerId, List> pairs) + throws ReplicationException { + String peerNode = getHFileRefsPeerNode(peerId); + boolean debugEnabled = LOG.isDebugEnabled(); + if (debugEnabled) { + LOG.debug("Adding hfile references " + pairs + " in queue " + peerNode); + } + List listOfOps = pairs.stream().map(p -> p.getSecond().getName()) + .map(n -> getHFileNode(peerNode, n)) + .map(f -> ZKUtilOp.createAndFailSilent(f, HConstants.EMPTY_BYTE_ARRAY)).collect(toList()); + if (debugEnabled) { + LOG.debug("The multi list size for adding hfile references in zk for node " + peerNode + + " is " + listOfOps.size()); + } + try { + ZKUtil.multiOrSequential(this.zookeeper, listOfOps, true); + } catch (KeeperException e) { + throw new ReplicationException("Failed to add hfile reference to peer " + peerId, e); + } + } + + @Override + public void removeHFileRefs(String peerId, List files) throws ReplicationException { + String peerNode = getHFileRefsPeerNode(peerId); + boolean debugEnabled = LOG.isDebugEnabled(); + if (debugEnabled) { + LOG.debug("Removing hfile references " + files + " from queue " + peerNode); + } + + List listOfOps = files.stream().map(n -> getHFileNode(peerNode, n)) + .map(ZKUtilOp::deleteNodeFailSilent).collect(toList()); + if (debugEnabled) { + LOG.debug("The multi list size for removing hfile references in zk for node " + peerNode + + " is " + listOfOps.size()); + } + try { + ZKUtil.multiOrSequential(this.zookeeper, listOfOps, true); + } catch (KeeperException e) { + throw new ReplicationException("Failed to remove hfile reference from peer " + peerId, e); + } + } + + private List getAllPeersFromHFileRefsQueue0() throws KeeperException { + return nullToEmpty(ZKUtil.listChildrenNoWatch(zookeeper, hfileRefsZNode)); + } + + @Override + public List getAllPeersFromHFileRefsQueue() throws ReplicationException { + try { + return getAllPeersFromHFileRefsQueue0(); + } catch (KeeperException e) { + throw new ReplicationException("Failed to get list of all peers in hfile references node.", + e); + } + } + + private List getReplicableHFiles0(String peerId) throws KeeperException { + return nullToEmpty(ZKUtil.listChildrenNoWatch(this.zookeeper, getHFileRefsPeerNode(peerId))); + } + + @Override + public List getReplicableHFiles(String peerId) throws ReplicationException { + try { + return getReplicableHFiles0(peerId); + } catch (KeeperException e) { + throw new ReplicationException("Failed to get list of hfile references for peer " + peerId, + e); + } + } + + // will be overridden in UTs + @VisibleForTesting + protected int getHFileRefsZNodeCversion() throws ReplicationException { + Stat stat = new Stat(); + try { + ZKUtil.getDataNoWatch(zookeeper, hfileRefsZNode, stat); + } catch (KeeperException e) { + throw new ReplicationException("Failed to get stat of replication hfile references node.", e); + } + return stat.getCversion(); + } + + @Override + public Set getAllHFileRefs() throws ReplicationException { + try { + for (int retry = 0;; retry++) { + int v0 = getHFileRefsZNodeCversion(); + List peers = getAllPeersFromHFileRefsQueue(); + if (peers.isEmpty()) { + LOG.debug("Didn't find any peers with hfile references, won't prevent any deletions."); + return Collections.emptySet(); + } + Set hfileRefs = new HashSet<>(); + for (String peer : peers) { + hfileRefs.addAll(getReplicableHFiles0(peer)); + } + int v1 = getHFileRefsZNodeCversion(); + if (v0 == v1) { + return hfileRefs; + } + LOG.debug(String.format( + "Replication hfile references node cversion changed from " + "%d to %d, retry = %d", v0, + v1, retry)); + } + } catch (KeeperException e) { + throw new ReplicationException("Failed to get all hfile refs", e); + } + } +} diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationStorageBase.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationStorageBase.java new file mode 100644 index 0000000..7190aeb --- /dev/null +++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationStorageBase.java @@ -0,0 +1,77 @@ +/** + * 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; + +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.hbase.thirdparty.com.google.protobuf.CodedOutputStream; + +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos; + +/** + * This is a base class for maintaining replication related data,for example, peer, queue, etc, in + * zookeeper. + */ +@InterfaceAudience.Private +public class ZKReplicationStorageBase { + + public static final String REPLICATION_ZNODE = "zookeeper.znode.replication"; + public static final String REPLICATION_ZNODE_DEFAULT = "replication"; + + /** The name of the base znode that contains all replication state. */ + protected final String replicationZNode; + + protected final ZKWatcher zookeeper; + protected final Configuration conf; + + protected ZKReplicationStorageBase(ZKWatcher zookeeper, Configuration conf) { + this.zookeeper = zookeeper; + this.conf = conf; + + this.replicationZNode = ZNodePaths.joinZNode(this.zookeeper.znodePaths.baseZNode, + conf.get(REPLICATION_ZNODE, REPLICATION_ZNODE_DEFAULT)); + } + + /** + * Serialized protobuf of state with pb magic prefix prepended suitable for use as + * content of a peer-state znode under a peer cluster id as in + * /hbase/replication/peers/PEER_ID/peer-state. + */ + protected static byte[] toByteArray(final ReplicationProtos.ReplicationState.State state) { + ReplicationProtos.ReplicationState msg = + ReplicationProtos.ReplicationState.newBuilder().setState(state).build(); + // There is no toByteArray on this pb Message? + // 32 bytes is default which seems fair enough here. + try (ByteArrayOutputStream baos = new ByteArrayOutputStream()) { + CodedOutputStream cos = CodedOutputStream.newInstance(baos, 16); + msg.writeTo(cos); + cos.flush(); + baos.flush(); + return ProtobufUtil.prependPBMagic(baos.toByteArray()); + } catch (IOException e) { + throw new RuntimeException(e); + } + } +} diff --git a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java new file mode 100644 index 0000000..fccffb5 --- /dev/null +++ b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java @@ -0,0 +1,317 @@ +/** + * 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; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.replication.ReplicationPeer.PeerState; +import org.apache.hadoop.hbase.util.Pair; +import org.apache.hadoop.hbase.zookeeper.ZKConfig; +import org.apache.zookeeper.KeeperException; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * White box testing for replication state interfaces. Implementations should extend this class, and + * initialize the interfaces properly. + */ +public abstract class TestReplicationStateBasic { + + private static final Logger LOG = LoggerFactory.getLogger(TestReplicationStateBasic.class); + + protected ReplicationQueueStorage rqs; + protected ServerName server1 = ServerName.valueOf("hostname1.example.org", 1234, 12345); + protected ServerName server2 = ServerName.valueOf("hostname2.example.org", 1234, 12345); + protected ServerName server3 = ServerName.valueOf("hostname3.example.org", 1234, 12345); + protected ReplicationPeers rp; + protected static final String ID_ONE = "1"; + protected static final String ID_TWO = "2"; + protected static String KEY_ONE; + protected static String KEY_TWO; + + // For testing when we try to replicate to ourself + protected String OUR_KEY; + + protected static int zkTimeoutCount; + protected static final int ZK_MAX_COUNT = 300; + protected static final int ZK_SLEEP_INTERVAL = 100; // millis + + @Test + public void testReplicationQueueStorage() throws ReplicationException { + // Test methods with empty state + assertEquals(0, rqs.getListOfReplicators().size()); + assertTrue(rqs.getWALsInQueue(server1, "qId1").isEmpty()); + assertTrue(rqs.getAllQueues(server1).isEmpty()); + + /* + * Set up data Two replicators: -- server1: three queues with 0, 1 and 2 log files each -- + * server2: zero queues + */ + rqs.addWAL(server1, "qId1", "trash"); + rqs.removeWAL(server1, "qId1", "trash"); + rqs.addWAL(server1,"qId2", "filename1"); + rqs.addWAL(server1,"qId3", "filename2"); + rqs.addWAL(server1,"qId3", "filename3"); + rqs.addWAL(server2,"trash", "trash"); + rqs.removeQueue(server2,"trash"); + + List reps = rqs.getListOfReplicators(); + assertEquals(2, reps.size()); + assertTrue(server1.getServerName(), reps.contains(server1)); + assertTrue(server2.getServerName(), reps.contains(server2)); + + assertTrue(rqs.getWALsInQueue(ServerName.valueOf("bogus", 12345, 12345), "bogus").isEmpty()); + assertTrue(rqs.getWALsInQueue(server1, "bogus").isEmpty()); + assertEquals(0, rqs.getWALsInQueue(server1, "qId1").size()); + assertEquals(1, rqs.getWALsInQueue(server1, "qId2").size()); + assertEquals("filename1", rqs.getWALsInQueue(server1, "qId2").get(0)); + + assertTrue(rqs.getAllQueues(ServerName.valueOf("bogus", 12345, -1L)).isEmpty()); + assertEquals(0, rqs.getAllQueues(server2).size()); + List list = rqs.getAllQueues(server1); + assertEquals(3, list.size()); + assertTrue(list.contains("qId2")); + assertTrue(list.contains("qId3")); + } + + private void removeAllQueues(ServerName serverName) throws ReplicationException { + for (String queue: rqs.getAllQueues(serverName)) { + rqs.removeQueue(serverName, queue); + } + } + @Test + public void testReplicationQueues() throws ReplicationException { + // Initialize ReplicationPeer so we can add peers (we don't transfer lone queues) + rp.init(); + + rqs.removeQueue(server1, "bogus"); + rqs.removeWAL(server1, "bogus", "bogus"); + removeAllQueues(server1); + assertEquals(0, rqs.getAllQueues(server1).size()); + assertEquals(0, rqs.getWALPosition(server1, "bogus", "bogus")); + assertTrue(rqs.getWALsInQueue(server1, "bogus").isEmpty()); + assertTrue(rqs.getAllQueues(ServerName.valueOf("bogus", 1234, 12345)).isEmpty()); + + populateQueues(); + + assertEquals(3, rqs.getListOfReplicators().size()); + assertEquals(0, rqs.getWALsInQueue(server2, "qId1").size()); + assertEquals(5, rqs.getWALsInQueue(server3, "qId5").size()); + assertEquals(0, rqs.getWALPosition(server3, "qId1", "filename0")); + rqs.setWALPosition(server3, "qId5", "filename4", 354L); + assertEquals(354L, rqs.getWALPosition(server3, "qId5", "filename4")); + + assertEquals(5, rqs.getWALsInQueue(server3, "qId5").size()); + assertEquals(0, rqs.getWALsInQueue(server2, "qId1").size()); + assertEquals(0, rqs.getAllQueues(server1).size()); + assertEquals(1, rqs.getAllQueues(server2).size()); + assertEquals(5, rqs.getAllQueues(server3).size()); + + assertEquals(0, rqs.getAllQueues(server1).size()); + rqs.removeReplicatorIfQueueIsEmpty(server1); + assertEquals(2, rqs.getListOfReplicators().size()); + + List queues = rqs.getAllQueues(server3); + assertEquals(5, queues.size()); + for (String queue : queues) { + rqs.claimQueue(server3, queue, server2); + } + rqs.removeReplicatorIfQueueIsEmpty(server3); + assertEquals(1, rqs.getListOfReplicators().size()); + + assertEquals(6, rqs.getAllQueues(server2).size()); + removeAllQueues(server2); + rqs.removeReplicatorIfQueueIsEmpty(server2); + assertEquals(0, rqs.getListOfReplicators().size()); + } + + @Test + public void testHfileRefsReplicationQueues() throws ReplicationException, KeeperException { + rp.init(); + + List> files1 = new ArrayList<>(3); + files1.add(new Pair<>(null, new Path("file_1"))); + files1.add(new Pair<>(null, new Path("file_2"))); + files1.add(new Pair<>(null, new Path("file_3"))); + assertTrue(rqs.getReplicableHFiles(ID_ONE).isEmpty()); + assertEquals(0, rqs.getAllPeersFromHFileRefsQueue().size()); + rp.getPeerStorage().addPeer(ID_ONE, + ReplicationPeerConfig.newBuilder().setClusterKey(KEY_ONE).build(), true); + rqs.addPeerToHFileRefs(ID_ONE); + rqs.addHFileRefs(ID_ONE, files1); + assertEquals(1, rqs.getAllPeersFromHFileRefsQueue().size()); + assertEquals(3, rqs.getReplicableHFiles(ID_ONE).size()); + List hfiles2 = new ArrayList<>(files1.size()); + for (Pair p : files1) { + hfiles2.add(p.getSecond().getName()); + } + String removedString = hfiles2.remove(0); + rqs.removeHFileRefs(ID_ONE, hfiles2); + assertEquals(1, rqs.getReplicableHFiles(ID_ONE).size()); + hfiles2 = new ArrayList<>(1); + hfiles2.add(removedString); + rqs.removeHFileRefs(ID_ONE, hfiles2); + assertEquals(0, rqs.getReplicableHFiles(ID_ONE).size()); + rp.getPeerStorage().removePeer(ID_ONE); + } + + @Test + public void testRemovePeerForHFileRefs() throws ReplicationException, KeeperException { + rp.init(); + rp.getPeerStorage().addPeer(ID_ONE, + ReplicationPeerConfig.newBuilder().setClusterKey(KEY_ONE).build(), true); + rqs.addPeerToHFileRefs(ID_ONE); + rp.getPeerStorage().addPeer(ID_TWO, + ReplicationPeerConfig.newBuilder().setClusterKey(KEY_TWO).build(), true); + rqs.addPeerToHFileRefs(ID_TWO); + + List> files1 = new ArrayList<>(3); + files1.add(new Pair<>(null, new Path("file_1"))); + files1.add(new Pair<>(null, new Path("file_2"))); + files1.add(new Pair<>(null, new Path("file_3"))); + rqs.addHFileRefs(ID_ONE, files1); + rqs.addHFileRefs(ID_TWO, files1); + assertEquals(2, rqs.getAllPeersFromHFileRefsQueue().size()); + assertEquals(3, rqs.getReplicableHFiles(ID_ONE).size()); + assertEquals(3, rqs.getReplicableHFiles(ID_TWO).size()); + + rp.getPeerStorage().removePeer(ID_ONE); + rqs.removePeerFromHFileRefs(ID_ONE); + assertEquals(1, rqs.getAllPeersFromHFileRefsQueue().size()); + assertTrue(rqs.getReplicableHFiles(ID_ONE).isEmpty()); + assertEquals(3, rqs.getReplicableHFiles(ID_TWO).size()); + + rp.getPeerStorage().removePeer(ID_TWO); + rqs.removePeerFromHFileRefs(ID_TWO); + assertEquals(0, rqs.getAllPeersFromHFileRefsQueue().size()); + assertTrue(rqs.getReplicableHFiles(ID_TWO).isEmpty()); + } + + @Test + public void testReplicationPeers() throws Exception { + rp.init(); + + try { + rp.getPeerStorage().setPeerState("bogus", true); + fail("Should have thrown an IllegalArgumentException when passed a bogus peerId"); + } catch (ReplicationException e) { + } + try { + rp.getPeerStorage().setPeerState("bogus", false); + fail("Should have thrown an IllegalArgumentException when passed a bogus peerId"); + } catch (ReplicationException e) { + } + + try { + assertFalse(rp.addPeer("bogus")); + fail("Should have thrown an ReplicationException when passed a bogus peerId"); + } catch (ReplicationException e) { + } + + assertNumberOfPeers(0); + + // Add some peers + rp.getPeerStorage().addPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE), true); + assertNumberOfPeers(1); + rp.getPeerStorage().addPeer(ID_TWO, new ReplicationPeerConfig().setClusterKey(KEY_TWO), true); + assertNumberOfPeers(2); + + assertEquals(KEY_ONE, ZKConfig.getZooKeeperClusterKey(ReplicationUtils + .getPeerClusterConfiguration(rp.getPeerStorage().getPeerConfig(ID_ONE), rp.getConf()))); + rp.getPeerStorage().removePeer(ID_ONE); + rp.removePeer(ID_ONE); + assertNumberOfPeers(1); + + // Add one peer + rp.getPeerStorage().addPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE), true); + rp.addPeer(ID_ONE); + assertNumberOfPeers(2); + assertTrue(rp.getPeer(ID_ONE).isPeerEnabled()); + rp.getPeerStorage().setPeerState(ID_ONE, false); + // now we do not rely on zk watcher to trigger the state change so we need to trigger it + // manually... + ReplicationPeerImpl peer = rp.getPeer(ID_ONE); + rp.refreshPeerState(peer.getId()); + assertEquals(PeerState.DISABLED, peer.getPeerState()); + assertConnectedPeerStatus(false, ID_ONE); + rp.getPeerStorage().setPeerState(ID_ONE, true); + // now we do not rely on zk watcher to trigger the state change so we need to trigger it + // manually... + rp.refreshPeerState(peer.getId()); + assertEquals(PeerState.ENABLED, peer.getPeerState()); + assertConnectedPeerStatus(true, ID_ONE); + + // Disconnect peer + rp.removePeer(ID_ONE); + assertNumberOfPeers(2); + } + + protected void assertConnectedPeerStatus(boolean status, String peerId) throws Exception { + // we can first check if the value was changed in the store, if it wasn't then fail right away + if (status != rp.getPeerStorage().isPeerEnabled(peerId)) { + fail("ConnectedPeerStatus was " + !status + " but expected " + status + " in ZK"); + } + while (true) { + if (status == rp.getPeer(peerId).isPeerEnabled()) { + return; + } + if (zkTimeoutCount < ZK_MAX_COUNT) { + LOG.debug("ConnectedPeerStatus was " + !status + " but expected " + status + + ", sleeping and trying again."); + Thread.sleep(ZK_SLEEP_INTERVAL); + } else { + fail("Timed out waiting for ConnectedPeerStatus to be " + status); + } + } + } + + protected void assertNumberOfPeers(int total) throws ReplicationException { + assertEquals(total, rp.getPeerStorage().listPeerIds().size()); + } + + /* + * three replicators: rq1 has 0 queues, rq2 has 1 queue with no logs, rq3 has 5 queues with 1, 2, + * 3, 4, 5 log files respectively + */ + protected void populateQueues() throws ReplicationException { + rqs.addWAL(server1, "trash", "trash"); + rqs.removeQueue(server1, "trash"); + + rqs.addWAL(server2, "qId1", "trash"); + rqs.removeWAL(server2, "qId1", "trash"); + + for (int i = 1; i < 6; i++) { + for (int j = 0; j < i; j++) { + rqs.addWAL(server3, "qId" + i, "filename" + j); + } + // Add peers for the corresponding queues so they are not orphans + rp.getPeerStorage().addPeer("qId" + i, + ReplicationPeerConfig.newBuilder().setClusterKey("localhost:2818:/bogus" + i).build(), + true); + } + } +} diff --git a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java new file mode 100644 index 0000000..2790bd0 --- /dev/null +++ b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java @@ -0,0 +1,89 @@ +/** + * 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; + +import java.io.IOException; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.ClusterId; +import org.apache.hadoop.hbase.HBaseZKTestingUtility; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.testclassification.ReplicationTests; +import org.apache.hadoop.hbase.zookeeper.ZKClusterId; +import org.apache.hadoop.hbase.zookeeper.ZKConfig; +import org.apache.hadoop.hbase.zookeeper.ZKUtil; +import org.apache.hadoop.hbase.zookeeper.ZKWatcher; +import org.apache.hadoop.hbase.zookeeper.ZNodePaths; +import org.apache.zookeeper.KeeperException; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.experimental.categories.Category; + +@Category({ ReplicationTests.class, MediumTests.class }) +public class TestReplicationStateZKImpl extends TestReplicationStateBasic { + + private static Configuration conf; + private static HBaseZKTestingUtility utility; + private static ZKWatcher zkw; + private static String replicationZNode; + + @BeforeClass + public static void setUpBeforeClass() throws Exception { + utility = new HBaseZKTestingUtility(); + utility.startMiniZKCluster(); + conf = utility.getConfiguration(); + conf.setBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY, true); + zkw = utility.getZooKeeperWatcher(); + String replicationZNodeName = conf.get("zookeeper.znode.replication", "replication"); + replicationZNode = ZNodePaths.joinZNode(zkw.znodePaths.baseZNode, replicationZNodeName); + KEY_ONE = initPeerClusterState("/hbase1"); + KEY_TWO = initPeerClusterState("/hbase2"); + } + + private static String initPeerClusterState(String baseZKNode) + throws IOException, KeeperException { + // Add a dummy region server and set up the cluster id + Configuration testConf = new Configuration(conf); + testConf.set(HConstants.ZOOKEEPER_ZNODE_PARENT, baseZKNode); + ZKWatcher zkw1 = new ZKWatcher(testConf, "test1", null); + String fakeRs = ZNodePaths.joinZNode(zkw1.znodePaths.rsZNode, "hostname1.example.org:1234"); + ZKUtil.createWithParents(zkw1, fakeRs); + ZKClusterId.setClusterId(zkw1, new ClusterId()); + return ZKConfig.getZooKeeperClusterKey(testConf); + } + + @Before + public void setUp() { + zkTimeoutCount = 0; + rqs = ReplicationStorageFactory.getReplicationQueueStorage(zkw, conf); + rp = ReplicationFactory.getReplicationPeers(zkw, conf); + OUR_KEY = ZKConfig.getZooKeeperClusterKey(conf); + } + + @After + public void tearDown() throws KeeperException, IOException { + ZKUtil.deleteNodeRecursively(zkw, replicationZNode); + } + + @AfterClass + public static void tearDownAfterClass() throws Exception { + utility.shutdownMiniZKCluster(); + } +} diff --git a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationPeerStorage.java b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationPeerStorage.java new file mode 100644 index 0000000..3eb11da --- /dev/null +++ b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationPeerStorage.java @@ -0,0 +1,173 @@ +/** + * 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; + +import static java.util.stream.Collectors.toList; +import static java.util.stream.Collectors.toSet; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.Set; +import java.util.stream.Stream; + +import org.apache.hadoop.hbase.HBaseZKTestingUtility; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.testclassification.ReplicationTests; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category({ ReplicationTests.class, MediumTests.class }) +public class TestZKReplicationPeerStorage { + + private static final HBaseZKTestingUtility UTIL = new HBaseZKTestingUtility(); + + private static ZKReplicationPeerStorage STORAGE; + + @BeforeClass + public static void setUp() throws Exception { + UTIL.startMiniZKCluster(); + STORAGE = new ZKReplicationPeerStorage(UTIL.getZooKeeperWatcher(), UTIL.getConfiguration()); + } + + @AfterClass + public static void tearDown() throws IOException { + UTIL.shutdownMiniZKCluster(); + } + + private Set randNamespaces(Random rand) { + return Stream.generate(() -> Long.toHexString(rand.nextLong())).limit(rand.nextInt(5)) + .collect(toSet()); + } + + private Map> randTableCFs(Random rand) { + int size = rand.nextInt(5); + Map> map = new HashMap<>(); + for (int i = 0; i < size; i++) { + TableName tn = TableName.valueOf(Long.toHexString(rand.nextLong())); + List cfs = Stream.generate(() -> Long.toHexString(rand.nextLong())) + .limit(rand.nextInt(5)).collect(toList()); + map.put(tn, cfs); + } + return map; + } + + private ReplicationPeerConfig getConfig(int seed) { + Random rand = new Random(seed); + return ReplicationPeerConfig.newBuilder().setClusterKey(Long.toHexString(rand.nextLong())) + .setReplicationEndpointImpl(Long.toHexString(rand.nextLong())) + .setNamespaces(randNamespaces(rand)).setExcludeNamespaces(randNamespaces(rand)) + .setTableCFsMap(randTableCFs(rand)).setReplicateAllUserTables(rand.nextBoolean()) + .setBandwidth(rand.nextInt(1000)).build(); + } + + private void assertSetEquals(Set expected, Set actual) { + if (expected == null || expected.size() == 0) { + assertTrue(actual == null || actual.size() == 0); + return; + } + assertEquals(expected.size(), actual.size()); + expected.forEach(s -> assertTrue(actual.contains(s))); + } + + private void assertMapEquals(Map> expected, + Map> actual) { + if (expected == null || expected.size() == 0) { + assertTrue(actual == null || actual.size() == 0); + return; + } + assertEquals(expected.size(), actual.size()); + expected.forEach((expectedTn, expectedCFs) -> { + List actualCFs = actual.get(expectedTn); + if (expectedCFs == null || expectedCFs.size() == 0) { + assertTrue(actual.containsKey(expectedTn)); + assertTrue(actualCFs == null || actualCFs.size() == 0); + } else { + assertNotNull(actualCFs); + assertEquals(expectedCFs.size(), actualCFs.size()); + for (Iterator expectedIt = expectedCFs.iterator(), actualIt = actualCFs.iterator(); + expectedIt.hasNext();) { + assertEquals(expectedIt.next(), actualIt.next()); + } + } + }); + } + + private void assertConfigEquals(ReplicationPeerConfig expected, ReplicationPeerConfig actual) { + assertEquals(expected.getClusterKey(), actual.getClusterKey()); + assertEquals(expected.getReplicationEndpointImpl(), actual.getReplicationEndpointImpl()); + assertSetEquals(expected.getNamespaces(), actual.getNamespaces()); + assertSetEquals(expected.getExcludeNamespaces(), actual.getExcludeNamespaces()); + assertMapEquals(expected.getTableCFsMap(), actual.getTableCFsMap()); + assertMapEquals(expected.getExcludeTableCFsMap(), actual.getExcludeTableCFsMap()); + assertEquals(expected.replicateAllUserTables(), actual.replicateAllUserTables()); + assertEquals(expected.getBandwidth(), actual.getBandwidth()); + } + + @Test + public void test() throws ReplicationException { + int peerCount = 10; + for (int i = 0; i < peerCount; i++) { + STORAGE.addPeer(Integer.toString(i), getConfig(i), i % 2 == 0); + } + List peerIds = STORAGE.listPeerIds(); + assertEquals(peerCount, peerIds.size()); + for (String peerId : peerIds) { + int seed = Integer.parseInt(peerId); + assertConfigEquals(getConfig(seed), STORAGE.getPeerConfig(peerId)); + } + for (int i = 0; i < peerCount; i++) { + STORAGE.updatePeerConfig(Integer.toString(i), getConfig(i + 1)); + } + for (String peerId : peerIds) { + int seed = Integer.parseInt(peerId); + assertConfigEquals(getConfig(seed + 1), STORAGE.getPeerConfig(peerId)); + } + for (int i = 0; i < peerCount; i++) { + assertEquals(i % 2 == 0, STORAGE.isPeerEnabled(Integer.toString(i))); + } + for (int i = 0; i < peerCount; i++) { + STORAGE.setPeerState(Integer.toString(i), i % 2 != 0); + } + for (int i = 0; i < peerCount; i++) { + assertEquals(i % 2 != 0, STORAGE.isPeerEnabled(Integer.toString(i))); + } + String toRemove = Integer.toString(peerCount / 2); + STORAGE.removePeer(toRemove); + peerIds = STORAGE.listPeerIds(); + assertEquals(peerCount - 1, peerIds.size()); + assertFalse(peerIds.contains(toRemove)); + + try { + STORAGE.getPeerConfig(toRemove); + fail("Should throw a ReplicationException when get peer config of a peerId"); + } catch (ReplicationException e) { + } + } +} diff --git a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationQueueStorage.java b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationQueueStorage.java new file mode 100644 index 0000000..786730f --- /dev/null +++ b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationQueueStorage.java @@ -0,0 +1,245 @@ +/** + * 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; + +import static org.hamcrest.CoreMatchers.hasItems; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.Set; +import java.util.SortedSet; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseZKTestingUtility; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.testclassification.ReplicationTests; +import org.apache.hadoop.hbase.util.Pair; +import org.apache.zookeeper.KeeperException; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category({ ReplicationTests.class, MediumTests.class }) +public class TestZKReplicationQueueStorage { + private static final HBaseZKTestingUtility UTIL = new HBaseZKTestingUtility(); + + private static ZKReplicationQueueStorage STORAGE; + + @BeforeClass + public static void setUp() throws Exception { + UTIL.startMiniZKCluster(); + STORAGE = new ZKReplicationQueueStorage(UTIL.getZooKeeperWatcher(), UTIL.getConfiguration()); + } + + @AfterClass + public static void tearDown() throws IOException { + UTIL.shutdownMiniZKCluster(); + } + + @After + public void tearDownAfterTest() throws ReplicationException { + for (ServerName serverName : STORAGE.getListOfReplicators()) { + for (String queue : STORAGE.getAllQueues(serverName)) { + STORAGE.removeQueue(serverName, queue); + } + STORAGE.removeReplicatorIfQueueIsEmpty(serverName); + } + for (String peerId : STORAGE.getAllPeersFromHFileRefsQueue()) { + STORAGE.removePeerFromHFileRefs(peerId); + } + } + + private ServerName getServerName(int i) { + return ServerName.valueOf("127.0.0.1", 8000 + i, 10000 + i); + } + + @Test + public void testReplicator() throws ReplicationException { + assertTrue(STORAGE.getListOfReplicators().isEmpty()); + String queueId = "1"; + for (int i = 0; i < 10; i++) { + STORAGE.addWAL(getServerName(i), queueId, "file" + i); + } + List replicators = STORAGE.getListOfReplicators(); + assertEquals(10, replicators.size()); + for (int i = 0; i < 10; i++) { + assertThat(replicators, hasItems(getServerName(i))); + } + for (int i = 0; i < 5; i++) { + STORAGE.removeQueue(getServerName(i), queueId); + } + for (int i = 0; i < 10; i++) { + STORAGE.removeReplicatorIfQueueIsEmpty(getServerName(i)); + } + replicators = STORAGE.getListOfReplicators(); + assertEquals(5, replicators.size()); + for (int i = 5; i < 10; i++) { + assertThat(replicators, hasItems(getServerName(i))); + } + } + + private String getFileName(String base, int i) { + return String.format(base + "-%04d", i); + } + + @Test + public void testAddRemoveLog() throws ReplicationException { + ServerName serverName1 = ServerName.valueOf("127.0.0.1", 8000, 10000); + assertTrue(STORAGE.getAllQueues(serverName1).isEmpty()); + String queue1 = "1"; + String queue2 = "2"; + for (int i = 0; i < 10; i++) { + STORAGE.addWAL(serverName1, queue1, getFileName("file1", i)); + STORAGE.addWAL(serverName1, queue2, getFileName("file2", i)); + } + List queueIds = STORAGE.getAllQueues(serverName1); + assertEquals(2, queueIds.size()); + assertThat(queueIds, hasItems("1", "2")); + + List wals1 = STORAGE.getWALsInQueue(serverName1, queue1); + List wals2 = STORAGE.getWALsInQueue(serverName1, queue2); + assertEquals(10, wals1.size()); + assertEquals(10, wals1.size()); + for (int i = 0; i < 10; i++) { + assertThat(wals1, hasItems(getFileName("file1", i))); + assertThat(wals2, hasItems(getFileName("file2", i))); + } + + for (int i = 0; i < 10; i++) { + assertEquals(0, STORAGE.getWALPosition(serverName1, queue1, getFileName("file1", i))); + assertEquals(0, STORAGE.getWALPosition(serverName1, queue2, getFileName("file2", i))); + STORAGE.setWALPosition(serverName1, queue1, getFileName("file1", i), (i + 1) * 100); + STORAGE.setWALPosition(serverName1, queue2, getFileName("file2", i), (i + 1) * 100 + 10); + } + + for (int i = 0; i < 10; i++) { + assertEquals((i + 1) * 100, + STORAGE.getWALPosition(serverName1, queue1, getFileName("file1", i))); + assertEquals((i + 1) * 100 + 10, + STORAGE.getWALPosition(serverName1, queue2, getFileName("file2", i))); + } + + for (int i = 0; i < 10; i++) { + if (i % 2 == 0) { + STORAGE.removeWAL(serverName1, queue1, getFileName("file1", i)); + } else { + STORAGE.removeWAL(serverName1, queue2, getFileName("file2", i)); + } + } + + queueIds = STORAGE.getAllQueues(serverName1); + assertEquals(2, queueIds.size()); + assertThat(queueIds, hasItems("1", "2")); + + ServerName serverName2 = ServerName.valueOf("127.0.0.1", 8001, 10001); + Pair> peer1 = STORAGE.claimQueue(serverName1, "1", serverName2); + + assertEquals("1-" + serverName1.getServerName(), peer1.getFirst()); + assertEquals(5, peer1.getSecond().size()); + int i = 1; + for (String wal : peer1.getSecond()) { + assertEquals(getFileName("file1", i), wal); + assertEquals((i + 1) * 100, + STORAGE.getWALPosition(serverName2, peer1.getFirst(), getFileName("file1", i))); + i += 2; + } + + queueIds = STORAGE.getAllQueues(serverName1); + assertEquals(1, queueIds.size()); + assertThat(queueIds, hasItems("2")); + wals2 = STORAGE.getWALsInQueue(serverName1, queue2); + assertEquals(5, wals2.size()); + for (i = 0; i < 10; i += 2) { + assertThat(wals2, hasItems(getFileName("file2", i))); + } + + queueIds = STORAGE.getAllQueues(serverName2); + assertEquals(1, queueIds.size()); + assertThat(queueIds, hasItems(peer1.getFirst())); + wals1 = STORAGE.getWALsInQueue(serverName2, peer1.getFirst()); + assertEquals(5, wals1.size()); + for (i = 1; i < 10; i += 2) { + assertThat(wals1, hasItems(getFileName("file1", i))); + } + + Set allWals = STORAGE.getAllWALs(); + assertEquals(10, allWals.size()); + for (i = 0; i < 10; i++) { + assertThat(allWals, hasItems(i % 2 == 0 ? getFileName("file2", i) : getFileName("file1", i))); + } + } + + // For HBASE-12865 + @Test + public void testClaimQueueChangeCversion() throws ReplicationException, KeeperException { + ServerName serverName1 = ServerName.valueOf("127.0.0.1", 8000, 10000); + STORAGE.addWAL(serverName1, "1", "file"); + + int v0 = STORAGE.getQueuesZNodeCversion(); + ServerName serverName2 = ServerName.valueOf("127.0.0.1", 8001, 10001); + STORAGE.claimQueue(serverName1, "1", serverName2); + int v1 = STORAGE.getQueuesZNodeCversion(); + // cversion should increase by 1 since a child node is deleted + assertEquals(1, v1 - v0); + } + + private ZKReplicationQueueStorage createWithUnstableCversion() throws IOException { + return new ZKReplicationQueueStorage(UTIL.getZooKeeperWatcher(), UTIL.getConfiguration()) { + + private int called = 0; + + @Override + protected int getQueuesZNodeCversion() throws KeeperException { + if (called < 4) { + called++; + } + return called; + } + }; + } + + @Test + public void testGetAllWALsCversionChange() throws IOException, ReplicationException { + ZKReplicationQueueStorage storage = createWithUnstableCversion(); + storage.addWAL(getServerName(0), "1", "file"); + // This should return eventually when cversion stabilizes + Set allWals = storage.getAllWALs(); + assertEquals(1, allWals.size()); + assertThat(allWals, hasItems("file")); + } + + // For HBASE-14621 + @Test + public void testGetAllHFileRefsCversionChange() throws IOException, ReplicationException { + ZKReplicationQueueStorage storage = createWithUnstableCversion(); + storage.addPeerToHFileRefs("1"); + Path p = new Path("/test"); + storage.addHFileRefs("1", Arrays.asList(Pair.newPair(p, p))); + // This should return eventually when cversion stabilizes + Set allHFileRefs = storage.getAllHFileRefs(); + assertEquals(1, allHFileRefs.size()); + assertThat(allHFileRefs, hasItems("test")); + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionServerObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionServerObserver.java index c1af3fb..5b751df 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionServerObserver.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionServerObserver.java @@ -126,4 +126,18 @@ public interface RegionServerObserver { default void postClearCompactionQueues( final ObserverContext ctx) throws IOException {} + + /** + * This will be called before executing procedures + * @param ctx the environment to interact with the framework and region server. + */ + default void preExecuteProcedures(ObserverContext ctx) + throws IOException {} + + /** + * This will be called after executing procedures + * @param ctx the environment to interact with the framework and region server. + */ + default void postExecuteProcedures(ObserverContext ctx) + throws IOException {} } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/EventType.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/EventType.java index 26fb63a..922deb8 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/EventType.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/EventType.java @@ -20,15 +20,14 @@ package org.apache.hadoop.hbase.executor; import org.apache.yetus.audience.InterfaceAudience; /** - * List of all HBase event handler types. Event types are named by a - * convention: event type names specify the component from which the event - * originated and then where its destined -- e.g. RS2ZK_ prefix means the - * event came from a regionserver destined for zookeeper -- and then what - * the even is; e.g. REGION_OPENING. - * - *

We give the enums indices so we can add types later and keep them - * grouped together rather than have to add them always to the end as we - * would have to if we used raw enum ordinals. + * List of all HBase event handler types. + *

+ * Event types are named by a convention: event type names specify the component from which the + * event originated and then where its destined -- e.g. RS_ZK_ prefix means the event came from a + * regionserver destined for zookeeper -- and then what the even is; e.g. REGION_OPENING. + *

+ * We give the enums indices so we can add types later and keep them grouped together rather than + * have to add them always to the end as we would have to if we used raw enum ordinals. */ @InterfaceAudience.Private public enum EventType { @@ -275,7 +274,14 @@ public enum EventType { * * RS_COMPACTED_FILES_DISCHARGER */ - RS_COMPACTED_FILES_DISCHARGER (83, ExecutorType.RS_COMPACTED_FILES_DISCHARGER); + RS_COMPACTED_FILES_DISCHARGER (83, ExecutorType.RS_COMPACTED_FILES_DISCHARGER), + + /** + * RS refresh peer.
+ * + * RS_REFRESH_PEER + */ + RS_REFRESH_PEER (84, ExecutorType.RS_REFRESH_PEER); private final int code; private final ExecutorType executor; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/ExecutorType.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/ExecutorType.java index c75a0a9..7f130d1 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/ExecutorType.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/ExecutorType.java @@ -46,7 +46,8 @@ public enum ExecutorType { RS_LOG_REPLAY_OPS (27), RS_REGION_REPLICA_FLUSH_OPS (28), RS_COMPACTED_FILES_DISCHARGER (29), - RS_OPEN_PRIORITY_REGION (30); + RS_OPEN_PRIORITY_REGION (30), + RS_REFRESH_PEER (31); ExecutorType(int value) { } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java index 945f54d..b3e8250 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -39,6 +38,7 @@ import java.util.List; import java.util.Map; import java.util.Map.Entry; import java.util.Objects; +import java.util.Optional; import java.util.Set; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; @@ -108,8 +108,6 @@ import org.apache.hadoop.hbase.master.balancer.LoadBalancerFactory; import org.apache.hadoop.hbase.master.cleaner.HFileCleaner; import org.apache.hadoop.hbase.master.cleaner.LogCleaner; import org.apache.hadoop.hbase.master.cleaner.ReplicationMetaCleaner; -import org.apache.hadoop.hbase.master.cleaner.ReplicationZKNodeCleaner; -import org.apache.hadoop.hbase.master.cleaner.ReplicationZKNodeCleanerChore; import org.apache.hadoop.hbase.master.locking.LockManager; import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan; import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan.PlanType; @@ -128,7 +126,13 @@ import org.apache.hadoop.hbase.master.procedure.ModifyTableProcedure; import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch; import org.apache.hadoop.hbase.master.procedure.RecoverMetaProcedure; import org.apache.hadoop.hbase.master.procedure.TruncateTableProcedure; -import org.apache.hadoop.hbase.master.replication.ReplicationManager; +import org.apache.hadoop.hbase.master.replication.AddPeerProcedure; +import org.apache.hadoop.hbase.master.replication.DisablePeerProcedure; +import org.apache.hadoop.hbase.master.replication.EnablePeerProcedure; +import org.apache.hadoop.hbase.master.replication.ModifyPeerProcedure; +import org.apache.hadoop.hbase.master.replication.RemovePeerProcedure; +import org.apache.hadoop.hbase.master.replication.ReplicationPeerManager; +import org.apache.hadoop.hbase.master.replication.UpdatePeerConfigProcedure; import org.apache.hadoop.hbase.master.snapshot.SnapshotManager; import org.apache.hadoop.hbase.mob.MobConstants; import org.apache.hadoop.hbase.monitoring.MemoryBoundedLogMessageBuffer; @@ -140,6 +144,8 @@ import org.apache.hadoop.hbase.procedure2.LockedResource; import org.apache.hadoop.hbase.procedure2.Procedure; import org.apache.hadoop.hbase.procedure2.ProcedureEvent; import org.apache.hadoop.hbase.procedure2.ProcedureExecutor; +import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteProcedure; +import org.apache.hadoop.hbase.procedure2.RemoteProcedureException; import org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore; import org.apache.hadoop.hbase.quotas.MasterQuotaManager; import org.apache.hadoop.hbase.quotas.MasterSpaceQuotaObserver; @@ -168,7 +174,6 @@ import org.apache.hadoop.hbase.util.Addressing; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.CompressionTest; import org.apache.hadoop.hbase.util.EncryptionTest; -import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.HFileArchiveUtil; import org.apache.hadoop.hbase.util.HasThread; import org.apache.hadoop.hbase.util.IdLock; @@ -197,6 +202,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; @@ -318,7 +324,7 @@ public class HMaster extends HRegionServer implements MasterServices { private AssignmentManager assignmentManager; // manager of replication - private ReplicationManager replicationManager; + private ReplicationPeerManager replicationPeerManager; // buffer for "fatal error" notices from region servers // in the cluster. This is only used for assisting @@ -328,17 +334,16 @@ public class HMaster extends HRegionServer implements MasterServices { // flag set after we become the active master (used for testing) private volatile boolean activeMaster = false; - // flag set after we complete initialization once active, - // it is not private since it's used in unit tests - private final ProcedureEvent initialized = new ProcedureEvent("master initialized"); + // flag set after we complete initialization once active + private final ProcedureEvent initialized = new ProcedureEvent<>("master initialized"); // flag set after master services are started, // initialization may have not completed yet. volatile boolean serviceStarted = false; // flag set after we complete assignMeta. - private final ProcedureEvent serverCrashProcessingEnabled = - new ProcedureEvent("server crash processing"); + private final ProcedureEvent serverCrashProcessingEnabled = + new ProcedureEvent<>("server crash processing"); // Maximum time we should run balancer for private final int maxBlancingTime; @@ -356,7 +361,6 @@ public class HMaster extends HRegionServer implements MasterServices { CatalogJanitor catalogJanitorChore; private ReplicationMetaCleaner replicationMetaCleaner; - private ReplicationZKNodeCleanerChore replicationZKNodeCleanerChore; private LogCleaner logCleaner; private HFileCleaner hfileCleaner; private ExpiredMobFileCleanerChore expiredMobFileCleanerChore; @@ -690,8 +694,8 @@ public class HMaster extends HRegionServer implements MasterServices { /** * Initialize all ZK based system trackers. */ - void initializeZKBasedSystemTrackers() throws IOException, - InterruptedException, KeeperException, CoordinatedStateException { + void initializeZKBasedSystemTrackers() throws IOException, InterruptedException, KeeperException, + CoordinatedStateException, ReplicationException { this.balancer = LoadBalancerFactory.getLoadBalancer(conf); this.normalizer = RegionNormalizerFactory.getRegionNormalizer(conf); this.normalizer.setMasterServices(this); @@ -709,7 +713,7 @@ public class HMaster extends HRegionServer implements MasterServices { this.assignmentManager = new AssignmentManager(this); this.assignmentManager.start(); - this.replicationManager = new ReplicationManager(conf, zooKeeper, this); + this.replicationPeerManager = ReplicationPeerManager.create(zooKeeper, conf); this.regionServerTracker = new RegionServerTracker(zooKeeper, this, this.serverManager); this.regionServerTracker.start(); @@ -754,9 +758,8 @@ public class HMaster extends HRegionServer implements MasterServices { *

  • Handle either fresh cluster start or master failover
  • * */ - private void finishActiveMasterInitialization(MonitoredTask status) - throws IOException, InterruptedException, KeeperException, CoordinatedStateException { - + private void finishActiveMasterInitialization(MonitoredTask status) throws IOException, + InterruptedException, KeeperException, CoordinatedStateException, ReplicationException { activeMaster = true; Thread zombieDetector = new Thread(new InitializationMonitor(this), "ActiveMasterInitializationMonitor-" + System.currentTimeMillis()); @@ -798,8 +801,8 @@ public class HMaster extends HRegionServer implements MasterServices { // This is for backwards compatibility // See HBASE-11393 status.setStatus("Update TableCFs node in ZNode"); - ReplicationPeerConfigUpgrader tableCFsUpdater = new ReplicationPeerConfigUpgrader(zooKeeper, - conf, this.clusterConnection); + ReplicationPeerConfigUpgrader tableCFsUpdater = + new ReplicationPeerConfigUpgrader(zooKeeper, conf); tableCFsUpdater.copyTableCFs(); // Add the Observer to delete space quotas on table deletion before starting all CPs by @@ -1144,15 +1147,6 @@ public class HMaster extends HRegionServer implements MasterServices { if (LOG.isTraceEnabled()) { LOG.trace("Started service threads"); } - - // Start replication zk node cleaner - try { - replicationZKNodeCleanerChore = new ReplicationZKNodeCleanerChore(this, cleanerInterval, - new ReplicationZKNodeCleaner(this.conf, this.getZooKeeper(), this)); - getChoreService().scheduleChore(replicationZKNodeCleanerChore); - } catch (Exception e) { - LOG.error("start replicationZKNodeCleanerChore failed", e); - } replicationMetaCleaner = new ReplicationMetaCleaner(this, this, cleanerInterval); getChoreService().scheduleChore(replicationMetaCleaner); } @@ -1177,7 +1171,6 @@ public class HMaster extends HRegionServer implements MasterServices { // Clean up and close up shop if (this.logCleaner != null) this.logCleaner.cancel(true); if (this.hfileCleaner != null) this.hfileCleaner.cancel(true); - if (this.replicationZKNodeCleanerChore != null) this.replicationZKNodeCleanerChore.cancel(true); if (this.replicationMetaCleaner != null) this.replicationMetaCleaner.cancel(true); if (this.quotaManager != null) this.quotaManager.stop(); @@ -1194,7 +1187,6 @@ public class HMaster extends HRegionServer implements MasterServices { private void startProcedureExecutor() throws IOException { final MasterProcedureEnv procEnv = new MasterProcedureEnv(this); - final Path rootDir = FSUtils.getRootDir(conf); procedureStore = new WALProcedureStore(conf, new MasterProcedureEnv.WALStoreLeaseRecovery(this)); @@ -2314,11 +2306,8 @@ public class HMaster extends HRegionServer implements MasterServices { return true; } Pair pair = - new Pair(MetaTableAccessor.getRegionInfo(data), + new Pair<>(MetaTableAccessor.getRegionInfo(data), MetaTableAccessor.getServerName(data,0)); - if (pair == null) { - return false; - } if (!pair.getFirst().getTable().equals(tableName)) { return false; } @@ -2744,7 +2733,7 @@ public class HMaster extends HRegionServer implements MasterServices { } @Override - public ProcedureEvent getInitializedEvent() { + public ProcedureEvent getInitializedEvent() { return initialized; } @@ -2763,7 +2752,7 @@ public class HMaster extends HRegionServer implements MasterServices { procedureExecutor.getEnvironment().setEventReady(serverCrashProcessingEnabled, b); } - public ProcedureEvent getServerCrashProcessingEnabledEvent() { + public ProcedureEvent getServerCrashProcessingEnabledEvent() { return serverCrashProcessingEnabled; } @@ -3314,83 +3303,60 @@ public class HMaster extends HRegionServer implements MasterServices { return favoredNodesManager; } + private long executePeerProcedure(ModifyPeerProcedure procedure) throws IOException { + long procId = procedureExecutor.submitProcedure(procedure); + procedure.getLatch().await(); + return procId; + } + @Override - public void addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled) + public long addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled) throws ReplicationException, IOException { - if (cpHost != null) { - cpHost.preAddReplicationPeer(peerId, peerConfig); - } - LOG.info(getClientIdAuditPrefix() + " creating replication peer, id=" + peerId + ", config=" - + peerConfig + ", state=" + (enabled ? "ENABLED" : "DISABLED")); - this.replicationManager.addReplicationPeer(peerId, peerConfig, enabled); - if (cpHost != null) { - cpHost.postAddReplicationPeer(peerId, peerConfig); - } + LOG.info(getClientIdAuditPrefix() + " creating replication peer, id=" + peerId + ", config=" + + peerConfig + ", state=" + (enabled ? "ENABLED" : "DISABLED")); + return executePeerProcedure(new AddPeerProcedure(peerId, peerConfig, enabled)); } @Override - public void removeReplicationPeer(String peerId) throws ReplicationException, IOException { - if (cpHost != null) { - cpHost.preRemoveReplicationPeer(peerId); - } + public long removeReplicationPeer(String peerId) throws ReplicationException, IOException { LOG.info(getClientIdAuditPrefix() + " removing replication peer, id=" + peerId); - this.replicationManager.removeReplicationPeer(peerId); - if (cpHost != null) { - cpHost.postRemoveReplicationPeer(peerId); - } + return executePeerProcedure(new RemovePeerProcedure(peerId)); } @Override - public void enableReplicationPeer(String peerId) throws ReplicationException, IOException { - if (cpHost != null) { - cpHost.preEnableReplicationPeer(peerId); - } + public long enableReplicationPeer(String peerId) throws ReplicationException, IOException { LOG.info(getClientIdAuditPrefix() + " enable replication peer, id=" + peerId); - this.replicationManager.enableReplicationPeer(peerId); - if (cpHost != null) { - cpHost.postEnableReplicationPeer(peerId); - } + return executePeerProcedure(new EnablePeerProcedure(peerId)); } @Override - public void disableReplicationPeer(String peerId) throws ReplicationException, IOException { - if (cpHost != null) { - cpHost.preDisableReplicationPeer(peerId); - } + public long disableReplicationPeer(String peerId) throws ReplicationException, IOException { LOG.info(getClientIdAuditPrefix() + " disable replication peer, id=" + peerId); - this.replicationManager.disableReplicationPeer(peerId); - if (cpHost != null) { - cpHost.postDisableReplicationPeer(peerId); - } + return executePeerProcedure(new DisablePeerProcedure(peerId)); } @Override - public ReplicationPeerConfig getReplicationPeerConfig(String peerId) throws ReplicationException, - IOException { + public ReplicationPeerConfig getReplicationPeerConfig(String peerId) + throws ReplicationException, IOException { if (cpHost != null) { cpHost.preGetReplicationPeerConfig(peerId); } - final ReplicationPeerConfig peerConfig = this.replicationManager.getPeerConfig(peerId); - LOG.info(getClientIdAuditPrefix() + " get replication peer config, id=" + peerId + ", config=" - + peerConfig); + LOG.info(getClientIdAuditPrefix() + " get replication peer config, id=" + peerId); + Optional peerConfig = + this.replicationPeerManager.getPeerConfig(peerId); + if (cpHost != null) { cpHost.postGetReplicationPeerConfig(peerId); } - return peerConfig; + return peerConfig.orElse(null); } @Override - public void updateReplicationPeerConfig(String peerId, ReplicationPeerConfig peerConfig) + public long updateReplicationPeerConfig(String peerId, ReplicationPeerConfig peerConfig) throws ReplicationException, IOException { - if (cpHost != null) { - cpHost.preUpdateReplicationPeerConfig(peerId, peerConfig); - } - LOG.info(getClientIdAuditPrefix() + " update replication peer config, id=" + peerId - + ", config=" + peerConfig); - this.replicationManager.updatePeerConfig(peerId, peerConfig); - if (cpHost != null) { - cpHost.postUpdateReplicationPeerConfig(peerId, peerConfig); - } + LOG.info(getClientIdAuditPrefix() + " update replication peer config, id=" + peerId + + ", config=" + peerConfig); + return executePeerProcedure(new UpdatePeerConfigProcedure(peerId, peerConfig)); } @Override @@ -3401,7 +3367,8 @@ public class HMaster extends HRegionServer implements MasterServices { } LOG.info(getClientIdAuditPrefix() + " list replication peers, regex=" + regex); Pattern pattern = regex == null ? null : Pattern.compile(regex); - List peers = this.replicationManager.listReplicationPeers(pattern); + List peers = + this.replicationPeerManager.listPeers(pattern); if (cpHost != null) { cpHost.postListReplicationPeers(regex); } @@ -3525,4 +3492,33 @@ public class HMaster extends HRegionServer implements MasterServices { public SpaceQuotaSnapshotNotifier getSpaceQuotaSnapshotNotifier() { return this.spaceQuotaSnapshotNotifier; } + + @SuppressWarnings("unchecked") + private RemoteProcedure getRemoteProcedure(long procId) { + Procedure procedure = procedureExecutor.getProcedure(procId); + if (procedure == null) { + return null; + } + assert procedure instanceof RemoteProcedure; + return (RemoteProcedure) procedure; + } + + public void remoteProcedureCompleted(long procId) { + RemoteProcedure procedure = getRemoteProcedure(procId); + if (procedure != null) { + procedure.remoteOperationCompleted(procedureExecutor.getEnvironment()); + } + } + + public void remoteProcedureFailed(long procId, RemoteProcedureException error) { + RemoteProcedure procedure = getRemoteProcedure(procId); + if (procedure != null) { + procedure.remoteOperationFailed(procedureExecutor.getEnvironment(), error); + } + } + + @Override + public ReplicationPeerManager getReplicationPeerManager() { + return replicationPeerManager; + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java index 8f41e4f..d822aee 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java @@ -71,6 +71,7 @@ import org.apache.hadoop.hbase.procedure2.LockedResource; import org.apache.hadoop.hbase.procedure2.Procedure; import org.apache.hadoop.hbase.procedure2.ProcedureExecutor; import org.apache.hadoop.hbase.procedure2.ProcedureUtil; +import org.apache.hadoop.hbase.procedure2.RemoteProcedureException; import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.AccessControlService; import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsService; import org.apache.hadoop.hbase.quotas.MasterQuotaManager; @@ -263,6 +264,9 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProto import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse; import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RemoteProcedureResult; +import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportProcedureDoneRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportProcedureDoneResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest; @@ -1882,10 +1886,10 @@ public class MasterRpcServices extends RSRpcServices public AddReplicationPeerResponse addReplicationPeer(RpcController controller, AddReplicationPeerRequest request) throws ServiceException { try { - master.addReplicationPeer(request.getPeerId(), - ReplicationPeerConfigUtil.convert(request.getPeerConfig()), request.getPeerState() - .getState().equals(ReplicationState.State.ENABLED)); - return AddReplicationPeerResponse.newBuilder().build(); + long procId = master.addReplicationPeer(request.getPeerId(), + ReplicationPeerConfigUtil.convert(request.getPeerConfig()), + request.getPeerState().getState().equals(ReplicationState.State.ENABLED)); + return AddReplicationPeerResponse.newBuilder().setProcId(procId).build(); } catch (ReplicationException | IOException e) { throw new ServiceException(e); } @@ -1895,8 +1899,8 @@ public class MasterRpcServices extends RSRpcServices public RemoveReplicationPeerResponse removeReplicationPeer(RpcController controller, RemoveReplicationPeerRequest request) throws ServiceException { try { - master.removeReplicationPeer(request.getPeerId()); - return RemoveReplicationPeerResponse.newBuilder().build(); + long procId = master.removeReplicationPeer(request.getPeerId()); + return RemoveReplicationPeerResponse.newBuilder().setProcId(procId).build(); } catch (ReplicationException | IOException e) { throw new ServiceException(e); } @@ -1906,8 +1910,8 @@ public class MasterRpcServices extends RSRpcServices public EnableReplicationPeerResponse enableReplicationPeer(RpcController controller, EnableReplicationPeerRequest request) throws ServiceException { try { - master.enableReplicationPeer(request.getPeerId()); - return EnableReplicationPeerResponse.newBuilder().build(); + long procId = master.enableReplicationPeer(request.getPeerId()); + return EnableReplicationPeerResponse.newBuilder().setProcId(procId).build(); } catch (ReplicationException | IOException e) { throw new ServiceException(e); } @@ -1917,8 +1921,8 @@ public class MasterRpcServices extends RSRpcServices public DisableReplicationPeerResponse disableReplicationPeer(RpcController controller, DisableReplicationPeerRequest request) throws ServiceException { try { - master.disableReplicationPeer(request.getPeerId()); - return DisableReplicationPeerResponse.newBuilder().build(); + long procId = master.disableReplicationPeer(request.getPeerId()); + return DisableReplicationPeerResponse.newBuilder().setProcId(procId).build(); } catch (ReplicationException | IOException e) { throw new ServiceException(e); } @@ -1944,9 +1948,9 @@ public class MasterRpcServices extends RSRpcServices public UpdateReplicationPeerConfigResponse updateReplicationPeerConfig(RpcController controller, UpdateReplicationPeerConfigRequest request) throws ServiceException { try { - master.updateReplicationPeerConfig(request.getPeerId(), + long procId = master.updateReplicationPeerConfig(request.getPeerId(), ReplicationPeerConfigUtil.convert(request.getPeerConfig())); - return UpdateReplicationPeerConfigResponse.newBuilder().build(); + return UpdateReplicationPeerConfigResponse.newBuilder().setProcId(procId).build(); } catch (ReplicationException | IOException e) { throw new ServiceException(e); } @@ -2246,4 +2250,18 @@ public class MasterRpcServices extends RSRpcServices } return response.build(); } + + @Override + public ReportProcedureDoneResponse reportProcedureDone(RpcController controller, + ReportProcedureDoneRequest request) throws ServiceException { + request.getResultList().forEach(result -> { + if (result.getStatus() == RemoteProcedureResult.Status.SUCCESS) { + master.remoteProcedureCompleted(result.getProcId()); + } else { + master.remoteProcedureFailed(result.getProcId(), + RemoteProcedureException.fromProto(result.getError())); + } + }); + return ReportProcedureDoneResponse.getDefaultInstance(); + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java index 9786fde..9d371bd 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -18,10 +17,11 @@ */ package org.apache.hadoop.hbase.master; +import com.google.protobuf.Service; + import java.io.IOException; import java.util.List; -import org.apache.hadoop.hbase.HBaseIOException; import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableDescriptors; @@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.master.assignment.AssignmentManager; import org.apache.hadoop.hbase.master.locking.LockManager; import org.apache.hadoop.hbase.master.normalizer.RegionNormalizer; import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv; +import org.apache.hadoop.hbase.master.replication.ReplicationPeerManager; import org.apache.hadoop.hbase.master.snapshot.SnapshotManager; import org.apache.hadoop.hbase.procedure.MasterProcedureManagerHost; import org.apache.hadoop.hbase.procedure2.LockedResource; @@ -52,8 +53,6 @@ import org.apache.yetus.audience.InterfaceAudience; import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; -import com.google.protobuf.Service; - /** * A curated subset of services provided by {@link HMaster}. * For use internally only. Passed to Managers, Services and Chores so can pass less-than-a @@ -136,7 +135,7 @@ public interface MasterServices extends Server { * @return Tripped when Master has finished initialization. */ @VisibleForTesting - public ProcedureEvent getInitializedEvent(); + public ProcedureEvent getInitializedEvent(); /** * @return Master's instance of {@link MetricsMaster} @@ -430,26 +429,26 @@ public interface MasterServices extends Server { * @param peerConfig configuration for the replication slave cluster * @param enabled peer state, true if ENABLED and false if DISABLED */ - void addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled) + long addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled) throws ReplicationException, IOException; /** * Removes a peer and stops the replication * @param peerId a short name that identifies the peer */ - void removeReplicationPeer(String peerId) throws ReplicationException, IOException; + long removeReplicationPeer(String peerId) throws ReplicationException, IOException; /** * Restart the replication stream to the specified peer * @param peerId a short name that identifies the peer */ - void enableReplicationPeer(String peerId) throws ReplicationException, IOException; + long enableReplicationPeer(String peerId) throws ReplicationException, IOException; /** * Stop the replication stream to the specified peer * @param peerId a short name that identifies the peer */ - void disableReplicationPeer(String peerId) throws ReplicationException, IOException; + long disableReplicationPeer(String peerId) throws ReplicationException, IOException; /** * Returns the configured ReplicationPeerConfig for the specified peer @@ -460,11 +459,16 @@ public interface MasterServices extends Server { IOException; /** + * Returns the {@link ReplicationPeerManager}. + */ + ReplicationPeerManager getReplicationPeerManager(); + + /** * Update the peerConfig for the specified peer * @param peerId a short name that identifies the peer * @param peerConfig new config for the peer */ - void updateReplicationPeerConfig(String peerId, ReplicationPeerConfig peerConfig) + long updateReplicationPeerConfig(String peerId, ReplicationPeerConfig peerConfig) throws ReplicationException, IOException; /** diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java index 4a88e3b..8277dbe 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java @@ -16,12 +16,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.master.assignment; import java.io.IOException; import java.util.concurrent.atomic.AtomicBoolean; - import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.RegionInfo; @@ -33,13 +31,16 @@ import org.apache.hadoop.hbase.procedure2.Procedure; import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException; import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteOperation; import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteProcedure; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RegionTransitionState; -import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode; +import org.apache.hadoop.hbase.procedure2.RemoteProcedureException; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; + +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RegionTransitionState; +import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode; + /** * Base class for the Assign and Unassign Procedure. * @@ -173,12 +174,6 @@ public abstract class RegionTransitionProcedure RegionStateNode regionNode, IOException exception); @Override - public void remoteCallCompleted(final MasterProcedureEnv env, - final ServerName serverName, final RemoteOperation response) { - // Ignore the response? reportTransition() is the one that count? - } - - @Override public void remoteCallFailed(final MasterProcedureEnv env, final ServerName serverName, final IOException exception) { final RegionStateNode regionNode = getRegionState(env); @@ -413,4 +408,16 @@ public abstract class RegionTransitionProcedure * @return ServerName the Assign or Unassign is going against. */ public abstract ServerName getServer(final MasterProcedureEnv env); + + @Override + public void remoteOperationCompleted(MasterProcedureEnv env) { + // should not be called for region operation until we modified the open/close region procedure + throw new UnsupportedOperationException(); + } + + @Override + public void remoteOperationFailed(MasterProcedureEnv env, RemoteProcedureException error) { + // should not be called for region operation until we modified the open/close region procedure + throw new UnsupportedOperationException(); + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKNodeCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKNodeCleaner.java deleted file mode 100644 index 97deab5..0000000 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKNodeCleaner.java +++ /dev/null @@ -1,206 +0,0 @@ -/** - * 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.master.cleaner; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.Map.Entry; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.Abortable; -import org.apache.hadoop.hbase.zookeeper.ZKWatcher; -import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hadoop.hbase.replication.ReplicationFactory; -import org.apache.hadoop.hbase.replication.ReplicationPeers; -import org.apache.hadoop.hbase.replication.ReplicationQueueInfo; -import org.apache.hadoop.hbase.replication.ReplicationQueuesClient; -import org.apache.hadoop.hbase.replication.ReplicationQueuesClientArguments; -import org.apache.hadoop.hbase.replication.ReplicationStateZKBase; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; -import org.apache.hadoop.hbase.zookeeper.ZNodePaths; -import org.apache.zookeeper.KeeperException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Used to clean the replication queues belonging to the peer which does not exist. - */ -@InterfaceAudience.Private -public class ReplicationZKNodeCleaner { - private static final Logger LOG = LoggerFactory.getLogger(ReplicationZKNodeCleaner.class); - private final ZKWatcher zkw; - private final ReplicationQueuesClient queuesClient; - private final ReplicationPeers replicationPeers; - private final ReplicationQueueDeletor queueDeletor; - - public ReplicationZKNodeCleaner(Configuration conf, ZKWatcher zkw, Abortable abortable) - throws IOException { - try { - this.zkw = zkw; - this.queuesClient = ReplicationFactory - .getReplicationQueuesClient(new ReplicationQueuesClientArguments(conf, abortable, zkw)); - this.queuesClient.init(); - this.replicationPeers = ReplicationFactory.getReplicationPeers(zkw, conf, this.queuesClient, - abortable); - this.replicationPeers.init(); - this.queueDeletor = new ReplicationQueueDeletor(zkw, conf, abortable); - } catch (Exception e) { - throw new IOException("failed to construct ReplicationZKNodeCleaner", e); - } - } - - /** - * @return undeletedQueues replicator with its queueIds for removed peers - * @throws IOException - */ - public Map> getUnDeletedQueues() throws IOException { - Map> undeletedQueues = new HashMap<>(); - Set peerIds = new HashSet<>(this.replicationPeers.getAllPeerIds()); - try { - List replicators = this.queuesClient.getListOfReplicators(); - if (replicators == null || replicators.isEmpty()) { - return undeletedQueues; - } - for (String replicator : replicators) { - List queueIds = this.queuesClient.getAllQueues(replicator); - for (String queueId : queueIds) { - ReplicationQueueInfo queueInfo = new ReplicationQueueInfo(queueId); - if (!peerIds.contains(queueInfo.getPeerId())) { - undeletedQueues.computeIfAbsent(replicator, (key) -> new ArrayList<>()).add( - queueId); - if (LOG.isDebugEnabled()) { - LOG.debug("Undeleted replication queue for removed peer found: " - + String.format("[removedPeerId=%s, replicator=%s, queueId=%s]", - queueInfo.getPeerId(), replicator, queueId)); - } - } - } - } - } catch (KeeperException ke) { - throw new IOException("Failed to get the replication queues of all replicators", ke); - } - return undeletedQueues; - } - - /** - * @return undeletedHFileRefsQueue replicator with its undeleted queueIds for removed peers in - * hfile-refs queue - * @throws IOException - */ - public Set getUnDeletedHFileRefsQueues() throws IOException { - Set undeletedHFileRefsQueue = new HashSet<>(); - Set peerIds = new HashSet<>(this.replicationPeers.getAllPeerIds()); - String hfileRefsZNode = queueDeletor.getHfileRefsZNode(); - try { - if (-1 == ZKUtil.checkExists(zkw, hfileRefsZNode)) { - return null; - } - List listOfPeers = this.queuesClient.getAllPeersFromHFileRefsQueue(); - Set peers = new HashSet<>(listOfPeers); - peers.removeAll(peerIds); - if (!peers.isEmpty()) { - undeletedHFileRefsQueue.addAll(peers); - } - } catch (KeeperException e) { - throw new IOException("Failed to get list of all peers from hfile-refs znode " - + hfileRefsZNode, e); - } - return undeletedHFileRefsQueue; - } - - private class ReplicationQueueDeletor extends ReplicationStateZKBase { - - public ReplicationQueueDeletor(ZKWatcher zk, Configuration conf, Abortable abortable) { - super(zk, conf, abortable); - } - - /** - * @param replicator The regionserver which has undeleted queue - * @param queueId The undeleted queue id - * @throws IOException - */ - public void removeQueue(final String replicator, final String queueId) throws IOException { - String queueZnodePath = - ZNodePaths.joinZNode(ZNodePaths.joinZNode(this.queuesZNode, replicator), queueId); - try { - ReplicationQueueInfo queueInfo = new ReplicationQueueInfo(queueId); - if (!replicationPeers.getAllPeerIds().contains(queueInfo.getPeerId())) { - ZKUtil.deleteNodeRecursively(this.zookeeper, queueZnodePath); - LOG.info("Successfully removed replication queue, replicator: " + replicator - + ", queueId: " + queueId); - } - } catch (KeeperException e) { - throw new IOException("Failed to delete queue, replicator: " + replicator + ", queueId: " - + queueId); - } - } - - /** - * @param hfileRefsQueueId The undeleted hfile-refs queue id - * @throws IOException - */ - public void removeHFileRefsQueue(final String hfileRefsQueueId) throws IOException { - String node = ZNodePaths.joinZNode(this.hfileRefsZNode, hfileRefsQueueId); - try { - if (!replicationPeers.getAllPeerIds().contains(hfileRefsQueueId)) { - ZKUtil.deleteNodeRecursively(this.zookeeper, node); - LOG.info("Successfully removed hfile-refs queue " + hfileRefsQueueId + " from path " - + hfileRefsZNode); - } - } catch (KeeperException e) { - throw new IOException("Failed to delete hfile-refs queue " + hfileRefsQueueId - + " from path " + hfileRefsZNode); - } - } - - String getHfileRefsZNode() { - return this.hfileRefsZNode; - } - } - - /** - * Remove the undeleted replication queue's zk node for removed peers. - * @param undeletedQueues replicator with its queueIds for removed peers - * @throws IOException - */ - public void removeQueues(final Map> undeletedQueues) throws IOException { - for (Entry> replicatorAndQueueIds : undeletedQueues.entrySet()) { - String replicator = replicatorAndQueueIds.getKey(); - for (String queueId : replicatorAndQueueIds.getValue()) { - queueDeletor.removeQueue(replicator, queueId); - } - } - } - - /** - * Remove the undeleted hfile-refs queue's zk node for removed peers. - * @param undeletedHFileRefsQueues replicator with its undeleted queueIds for removed peers in - * hfile-refs queue - * @throws IOException - */ - public void removeHFileRefsQueues(final Set undeletedHFileRefsQueues) throws IOException { - for (String hfileRefsQueueId : undeletedHFileRefsQueues) { - queueDeletor.removeHFileRefsQueue(hfileRefsQueueId); - } - } -} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKNodeCleanerChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKNodeCleanerChore.java deleted file mode 100644 index 8d5df9b..0000000 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKNodeCleanerChore.java +++ /dev/null @@ -1,55 +0,0 @@ -/** - * 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.master.cleaner; - -import java.io.IOException; -import java.util.List; -import java.util.Map; - -import org.apache.hadoop.hbase.ScheduledChore; -import org.apache.hadoop.hbase.Stoppable; -import org.apache.yetus.audience.InterfaceAudience; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Chore that will clean the replication queues belonging to the peer which does not exist. - */ -@InterfaceAudience.Private -public class ReplicationZKNodeCleanerChore extends ScheduledChore { - private static final Logger LOG = LoggerFactory.getLogger(ReplicationZKNodeCleanerChore.class); - private final ReplicationZKNodeCleaner cleaner; - - public ReplicationZKNodeCleanerChore(Stoppable stopper, int period, - ReplicationZKNodeCleaner cleaner) { - super("ReplicationZKNodeCleanerChore", stopper, period); - this.cleaner = cleaner; - } - - @Override - protected void chore() { - try { - Map> undeletedQueues = cleaner.getUnDeletedQueues(); - cleaner.removeQueues(undeletedQueues); - } catch (IOException e) { - LOG.warn("Failed to clean replication zk node", e); - } - } - -} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureEnv.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureEnv.java index 0a4c97d..ae038a5 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureEnv.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureEnv.java @@ -24,23 +24,24 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.conf.ConfigurationObserver; -import org.apache.yetus.audience.InterfaceAudience; -import org.apache.yetus.audience.InterfaceStability; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import org.apache.hadoop.hbase.ipc.RpcServer; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.master.MasterCoprocessorHost; import org.apache.hadoop.hbase.master.MasterServices; import org.apache.hadoop.hbase.master.assignment.AssignmentManager; +import org.apache.hadoop.hbase.master.replication.ReplicationPeerManager; import org.apache.hadoop.hbase.procedure2.Procedure; import org.apache.hadoop.hbase.procedure2.ProcedureEvent; import org.apache.hadoop.hbase.procedure2.store.ProcedureStore; import org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore; -import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.security.Superusers; +import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.util.CancelableProgressable; import org.apache.hadoop.hbase.util.FSUtils; +import org.apache.yetus.audience.InterfaceAudience; +import org.apache.yetus.audience.InterfaceStability; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; @InterfaceAudience.Private @InterfaceStability.Evolving @@ -137,6 +138,10 @@ public class MasterProcedureEnv implements ConfigurationObserver { return remoteDispatcher; } + public ReplicationPeerManager getReplicationPeerManager() { + return master.getReplicationPeerManager(); + } + public boolean isRunning() { if (this.master == null || this.master.getMasterProcedureExecutor() == null) return false; return master.getMasterProcedureExecutor().isRunning(); @@ -146,22 +151,22 @@ public class MasterProcedureEnv implements ConfigurationObserver { return master.isInitialized(); } - public boolean waitInitialized(Procedure proc) { + public boolean waitInitialized(Procedure proc) { return master.getInitializedEvent().suspendIfNotReady(proc); } - public boolean waitServerCrashProcessingEnabled(Procedure proc) { + public boolean waitServerCrashProcessingEnabled(Procedure proc) { if (master instanceof HMaster) { return ((HMaster)master).getServerCrashProcessingEnabledEvent().suspendIfNotReady(proc); } return false; } - public boolean waitFailoverCleanup(Procedure proc) { + public boolean waitFailoverCleanup(Procedure proc) { return master.getAssignmentManager().getFailoverCleanupEvent().suspendIfNotReady(proc); } - public void setEventReady(ProcedureEvent event, boolean isReady) { + public void setEventReady(ProcedureEvent event, boolean isReady) { if (isReady) { event.wake(procSched); } else { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java index dc9c69d..0400de4 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java @@ -24,7 +24,7 @@ import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Map.Entry; +import java.util.function.Function; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.ServerName; @@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.master.locking.LockProcedure; +import org.apache.hadoop.hbase.master.procedure.PeerProcedureInterface.PeerOperationType; import org.apache.hadoop.hbase.master.procedure.TableProcedureInterface.TableOperationType; import org.apache.hadoop.hbase.procedure2.AbstractProcedureScheduler; import org.apache.hadoop.hbase.procedure2.LockAndQueue; @@ -109,12 +110,17 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler { new ServerQueueKeyComparator(); private final static TableQueueKeyComparator TABLE_QUEUE_KEY_COMPARATOR = new TableQueueKeyComparator(); + private final static PeerQueueKeyComparator PEER_QUEUE_KEY_COMPARATOR = + new PeerQueueKeyComparator(); private final FairQueue serverRunQueue = new FairQueue<>(); private final FairQueue tableRunQueue = new FairQueue<>(); + private final FairQueue peerRunQueue = new FairQueue<>(); private final ServerQueue[] serverBuckets = new ServerQueue[128]; private TableQueue tableMap = null; + private PeerQueue peerMap = null; + private final SchemaLocking locking = new SchemaLocking(); /** @@ -161,6 +167,8 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler { doAdd(tableRunQueue, getTableQueue(getTableName(proc)), proc, addFront); } else if (isServerProcedure(proc)) { doAdd(serverRunQueue, getServerQueue(getServerName(proc)), proc, addFront); + } else if (isPeerProcedure(proc)) { + doAdd(peerRunQueue, getPeerQueue(getPeerId(proc)), proc, addFront); } else { // TODO: at the moment we only have Table and Server procedures // if you are implementing a non-table/non-server procedure, you have two options: create @@ -172,7 +180,7 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler { } private > void doAdd(final FairQueue fairq, - final Queue queue, final Procedure proc, final boolean addFront) { + final Queue queue, final Procedure proc, final boolean addFront) { queue.add(proc, addFront); if (!queue.getLockStatus().hasExclusiveLock() || queue.getLockStatus().isLockOwner(proc.getProcId())) { // if the queue was not remove for an xlock execution @@ -189,7 +197,8 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler { @Override protected boolean queueHasRunnables() { - return tableRunQueue.hasRunnables() || serverRunQueue.hasRunnables(); + return tableRunQueue.hasRunnables() || serverRunQueue.hasRunnables() || + peerRunQueue.hasRunnables(); } @Override @@ -197,7 +206,10 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler { // For now, let server handling have precedence over table handling; presumption is that it // is more important handling crashed servers than it is running the // enabling/disabling tables, etc. - Procedure pollResult = doPoll(serverRunQueue); + Procedure pollResult = doPoll(serverRunQueue); + if (pollResult == null) { + pollResult = doPoll(peerRunQueue); + } if (pollResult == null) { pollResult = doPoll(tableRunQueue); } @@ -267,60 +279,30 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler { exclusiveLockOwnerProcedure, sharedLockCount, waitingProcedures); } + private void addToLockedResources(List lockedResources, + Map locks, Function keyTransformer, + LockedResourceType resourcesType) { + locks.entrySet().stream().filter(e -> e.getValue().isLocked()) + .map( + e -> createLockedResource(resourcesType, keyTransformer.apply(e.getKey()), e.getValue())) + .forEachOrdered(lockedResources::add); + } + @Override public List getLocks() { schedLock(); - try { List lockedResources = new ArrayList<>(); - - for (Entry entry : locking.serverLocks - .entrySet()) { - String serverName = entry.getKey().getServerName(); - LockAndQueue queue = entry.getValue(); - - if (queue.isLocked()) { - LockedResource lockedResource = - createLockedResource(LockedResourceType.SERVER, serverName, queue); - lockedResources.add(lockedResource); - } - } - - for (Entry entry : locking.namespaceLocks - .entrySet()) { - String namespaceName = entry.getKey(); - LockAndQueue queue = entry.getValue(); - - if (queue.isLocked()) { - LockedResource lockedResource = - createLockedResource(LockedResourceType.NAMESPACE, namespaceName, queue); - lockedResources.add(lockedResource); - } - } - - for (Entry entry : locking.tableLocks - .entrySet()) { - String tableName = entry.getKey().getNameAsString(); - LockAndQueue queue = entry.getValue(); - - if (queue.isLocked()) { - LockedResource lockedResource = - createLockedResource(LockedResourceType.TABLE, tableName, queue); - lockedResources.add(lockedResource); - } - } - - for (Entry entry : locking.regionLocks.entrySet()) { - String regionName = entry.getKey(); - LockAndQueue queue = entry.getValue(); - - if (queue.isLocked()) { - LockedResource lockedResource = - createLockedResource(LockedResourceType.REGION, regionName, queue); - lockedResources.add(lockedResource); - } - } - + addToLockedResources(lockedResources, locking.serverLocks, sn -> sn.getServerName(), + LockedResourceType.SERVER); + addToLockedResources(lockedResources, locking.namespaceLocks, Function.identity(), + LockedResourceType.NAMESPACE); + addToLockedResources(lockedResources, locking.tableLocks, tn -> tn.getNameAsString(), + LockedResourceType.TABLE); + addToLockedResources(lockedResources, locking.regionLocks, Function.identity(), + LockedResourceType.REGION); + addToLockedResources(lockedResources, locking.peerLocks, Function.identity(), + LockedResourceType.PEER); return lockedResources; } finally { schedUnlock(); @@ -328,8 +310,7 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler { } @Override - public LockedResource getLockResource(LockedResourceType resourceType, - String resourceName) { + public LockedResource getLockResource(LockedResourceType resourceType, String resourceName) { LockAndQueue queue = null; schedLock(); try { @@ -346,8 +327,10 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler { case REGION: queue = locking.regionLocks.get(resourceName); break; + case PEER: + queue = locking.peerLocks.get(resourceName); + break; } - return queue != null ? createLockedResource(resourceType, resourceName, queue) : null; } finally { schedUnlock(); @@ -406,13 +389,20 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler { while (tableIter.hasNext()) { count += tableIter.next().size(); } + + // Peer queues + final AvlTreeIterator peerIter = new AvlTreeIterator<>(peerMap); + while (peerIter.hasNext()) { + count += peerIter.next().size(); + } + return count; } @Override public void completionCleanup(final Procedure proc) { if (proc instanceof TableProcedureInterface) { - TableProcedureInterface iProcTable = (TableProcedureInterface)proc; + TableProcedureInterface iProcTable = (TableProcedureInterface) proc; boolean tableDeleted; if (proc.hasException()) { Exception procEx = proc.getException().unwrapRemoteException(); @@ -431,6 +421,9 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler { markTableAsDeleted(iProcTable.getTableName(), proc); return; } + } else if (proc instanceof PeerProcedureInterface) { + PeerProcedureInterface iProcPeer = (PeerProcedureInterface) proc; + tryCleanupPeerQueue(iProcPeer.getPeerId(), proc); } else { // No cleanup for ServerProcedureInterface types, yet. return; @@ -468,12 +461,11 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler { locking.removeTableLock(tableName); } - - private static boolean isTableProcedure(Procedure proc) { + private static boolean isTableProcedure(Procedure proc) { return proc instanceof TableProcedureInterface; } - private static TableName getTableName(Procedure proc) { + private static TableName getTableName(Procedure proc) { return ((TableProcedureInterface)proc).getTableName(); } @@ -494,15 +486,59 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler { return Math.abs(hashCode) % buckets.length; } - private static boolean isServerProcedure(Procedure proc) { + private static boolean isServerProcedure(Procedure proc) { return proc instanceof ServerProcedureInterface; } - private static ServerName getServerName(Procedure proc) { + private static ServerName getServerName(Procedure proc) { return ((ServerProcedureInterface)proc).getServerName(); } // ============================================================================ + // Peer Queue Lookup Helpers + // ============================================================================ + private PeerQueue getPeerQueue(String peerId) { + PeerQueue node = AvlTree.get(peerMap, peerId, PEER_QUEUE_KEY_COMPARATOR); + if (node != null) { + return node; + } + node = new PeerQueue(peerId, locking.getPeerLock(peerId)); + peerMap = AvlTree.insert(peerMap, node); + return node; + } + + private void removePeerQueue(String peerId) { + peerMap = AvlTree.remove(peerMap, peerId, PEER_QUEUE_KEY_COMPARATOR); + locking.removePeerLock(peerId); + } + + private void tryCleanupPeerQueue(String peerId, Procedure procedure) { + schedLock(); + try { + PeerQueue queue = AvlTree.get(peerMap, peerId, PEER_QUEUE_KEY_COMPARATOR); + if (queue == null) { + return; + } + + final LockAndQueue lock = locking.getPeerLock(peerId); + if (queue.isEmpty() && lock.tryExclusiveLock(procedure)) { + removeFromRunQueue(peerRunQueue, queue); + removePeerQueue(peerId); + } + } finally { + schedUnlock(); + } + } + + private static boolean isPeerProcedure(Procedure proc) { + return proc instanceof PeerProcedureInterface; + } + + private static String getPeerId(Procedure proc) { + return ((PeerProcedureInterface) proc).getPeerId(); + } + + // ============================================================================ // Table and Server Queue Implementation // ============================================================================ private static class ServerQueueKeyComparator implements AvlKeyComparator { @@ -571,6 +607,40 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler { } } + private static class PeerQueueKeyComparator implements AvlKeyComparator { + + @Override + public int compareKey(PeerQueue node, Object key) { + return node.compareKey((String) key); + } + } + + public static class PeerQueue extends Queue { + + public PeerQueue(String peerId, LockStatus lockStatus) { + super(peerId, lockStatus); + } + + @Override + public boolean requireExclusiveLock(Procedure proc) { + return requirePeerExclusiveLock((PeerProcedureInterface) proc); + } + + @Override + public boolean isAvailable() { + if (isEmpty()) { + return false; + } + if (getLockStatus().hasExclusiveLock()) { + // if we have an exclusive lock already taken + // only child of the lock owner can be executed + Procedure nextProc = peek(); + return nextProc != null && getLockStatus().hasLockAccess(nextProc); + } + return true; + } + } + // ============================================================================ // Table Locking Helpers // ============================================================================ @@ -958,7 +1028,8 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler { * @param serverName Server to lock * @return true if the procedure has to wait for the server to be available */ - public boolean waitServerExclusiveLock(final Procedure procedure, final ServerName serverName) { + public boolean waitServerExclusiveLock(final Procedure procedure, + final ServerName serverName) { schedLock(); try { final LockAndQueue lock = locking.getServerLock(serverName); @@ -980,7 +1051,7 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler { * @param procedure the procedure releasing the lock * @param serverName the server that has the exclusive lock */ - public void wakeServerExclusiveLock(final Procedure procedure, final ServerName serverName) { + public void wakeServerExclusiveLock(final Procedure procedure, final ServerName serverName) { schedLock(); try { final LockAndQueue lock = locking.getServerLock(serverName); @@ -994,6 +1065,56 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler { } // ============================================================================ + // Peer Locking Helpers + // ============================================================================ + + private static boolean requirePeerExclusiveLock(PeerProcedureInterface proc) { + return proc.getPeerOperationType() != PeerOperationType.REFRESH; + } + + /** + * Try to acquire the exclusive lock on the specified peer. + * @see #wakePeerExclusiveLock(Procedure, String) + * @param procedure the procedure trying to acquire the lock + * @param peerId peer to lock + * @return true if the procedure has to wait for the peer to be available + */ + public boolean waitPeerExclusiveLock(Procedure procedure, String peerId) { + schedLock(); + try { + final LockAndQueue lock = locking.getPeerLock(peerId); + if (lock.tryExclusiveLock(procedure)) { + removeFromRunQueue(peerRunQueue, getPeerQueue(peerId)); + return false; + } + waitProcedure(lock, procedure); + logLockedResource(LockedResourceType.PEER, peerId); + return true; + } finally { + schedUnlock(); + } + } + + /** + * Wake the procedures waiting for the specified peer + * @see #waitPeerExclusiveLock(Procedure, String) + * @param procedure the procedure releasing the lock + * @param peerId the peer that has the exclusive lock + */ + public void wakePeerExclusiveLock(Procedure procedure, String peerId) { + schedLock(); + try { + final LockAndQueue lock = locking.getPeerLock(peerId); + lock.releaseExclusiveLock(procedure); + addToRunQueue(peerRunQueue, getPeerQueue(peerId)); + int waitingCount = wakeWaitingProcedures(lock); + wakePollIfNeeded(waitingCount); + } finally { + schedUnlock(); + } + } + + // ============================================================================ // Generic Helpers // ============================================================================ private static abstract class Queue> @@ -1098,6 +1219,7 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler { final Map tableLocks = new HashMap<>(); // Single map for all regions irrespective of tables. Key is encoded region name. final Map regionLocks = new HashMap<>(); + final Map peerLocks = new HashMap<>(); private LockAndQueue getLock(Map map, T key) { LockAndQueue lock = map.get(key); @@ -1132,6 +1254,14 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler { return getLock(serverLocks, serverName); } + LockAndQueue getPeerLock(String peerId) { + return getLock(peerLocks, peerId); + } + + LockAndQueue removePeerLock(String peerId) { + return peerLocks.remove(peerId); + } + /** * Removes all locks by clearing the maps. * Used when procedure executor is stopped for failure and recovery testing. @@ -1142,6 +1272,7 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler { namespaceLocks.clear(); tableLocks.clear(); regionLocks.clear(); + peerLocks.clear(); } @Override @@ -1149,7 +1280,8 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler { return "serverLocks=" + filterUnlocked(this.serverLocks) + ", namespaceLocks=" + filterUnlocked(this.namespaceLocks) + ", tableLocks=" + filterUnlocked(this.tableLocks) + - ", regionLocks=" + filterUnlocked(this.regionLocks); + ", regionLocks=" + filterUnlocked(this.regionLocks) + + ", peerLocks=" + filterUnlocked(this.peerLocks); } private String filterUnlocked(Map locks) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/PeerProcedureInterface.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/PeerProcedureInterface.java new file mode 100644 index 0000000..4abc9ad --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/PeerProcedureInterface.java @@ -0,0 +1,34 @@ +/** + * 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.master.procedure; + +import org.apache.yetus.audience.InterfaceAudience; +import org.apache.yetus.audience.InterfaceStability; + +@InterfaceAudience.Private +@InterfaceStability.Evolving +public interface PeerProcedureInterface { + + enum PeerOperationType { + ADD, REMOVE, ENABLE, DISABLE, UPDATE_CONFIG, REFRESH + } + + String getPeerId(); + + PeerOperationType getPeerOperationType(); +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedurePrepareLatch.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedurePrepareLatch.java index 09d05e6..dbea6fa 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedurePrepareLatch.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedurePrepareLatch.java @@ -64,7 +64,7 @@ public abstract class ProcedurePrepareLatch { protected abstract void countDown(final Procedure proc); public abstract void await() throws IOException; - protected static void releaseLatch(final ProcedurePrepareLatch latch, final Procedure proc) { + public static void releaseLatch(final ProcedurePrepareLatch latch, final Procedure proc) { if (latch != null) { latch.countDown(proc); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java index 65c4d08..a6d57d2 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.master.procedure; import java.io.IOException; @@ -24,7 +23,6 @@ import java.util.List; import java.util.Set; import java.util.concurrent.Callable; import java.util.concurrent.TimeUnit; - import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.client.RegionInfo; @@ -36,10 +34,13 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.ipc.RemoteException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.hbase.thirdparty.com.google.common.collect.ArrayListMultimap; +import org.apache.hbase.thirdparty.com.google.protobuf.ByteString; import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback; import org.apache.hbase.thirdparty.com.google.protobuf.RpcController; import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException; + import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter; import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService; @@ -49,6 +50,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ExecuteProc import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ExecuteProceduresResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RemoteProcedureRequest; /** * A remote procecdure dispatcher for regionservers. @@ -222,7 +224,10 @@ public class RSProcedureDispatcher private interface RemoteProcedureResolver { void dispatchOpenRequests(MasterProcedureEnv env, List operations); + void dispatchCloseRequests(MasterProcedureEnv env, List operations); + + void dispatchServerOperations(MasterProcedureEnv env, List operations); } /** @@ -231,22 +236,28 @@ public class RSProcedureDispatcher * Then {@code resolver} is used to dispatch {@link RegionOpenOperation}s and * {@link RegionCloseOperation}s. * @param serverName RegionServer to which the remote operations are sent - * @param remoteProcedures Remote procedures which are dispatched to the given server + * @param operations Remote procedures which are dispatched to the given server * @param resolver Used to dispatch remote procedures to given server. */ - public void splitAndResolveOperation(final ServerName serverName, - final Set remoteProcedures, final RemoteProcedureResolver resolver) { - final ArrayListMultimap, RemoteOperation> reqsByType = - buildAndGroupRequestByType(procedureEnv, serverName, remoteProcedures); + public void splitAndResolveOperation(ServerName serverName, Set operations, + RemoteProcedureResolver resolver) { + MasterProcedureEnv env = master.getMasterProcedureExecutor().getEnvironment(); + ArrayListMultimap, RemoteOperation> reqsByType = + buildAndGroupRequestByType(env, serverName, operations); - final List openOps = fetchType(reqsByType, RegionOpenOperation.class); + List openOps = fetchType(reqsByType, RegionOpenOperation.class); if (!openOps.isEmpty()) { - resolver.dispatchOpenRequests(procedureEnv, openOps); + resolver.dispatchOpenRequests(env, openOps); } - final List closeOps = fetchType(reqsByType, RegionCloseOperation.class); + List closeOps = fetchType(reqsByType, RegionCloseOperation.class); if (!closeOps.isEmpty()) { - resolver.dispatchCloseRequests(procedureEnv, closeOps); + resolver.dispatchCloseRequests(env, closeOps); + } + + List refreshOps = fetchType(reqsByType, ServerOperation.class); + if (!refreshOps.isEmpty()) { + resolver.dispatchServerOperations(env, refreshOps); } if (!reqsByType.isEmpty()) { @@ -277,8 +288,7 @@ public class RSProcedureDispatcher splitAndResolveOperation(getServerName(), remoteProcedures, this); try { - final ExecuteProceduresResponse response = sendRequest(getServerName(), request.build()); - remoteCallCompleted(procedureEnv, response); + sendRequest(getServerName(), request.build()); } catch (IOException e) { e = unwrapException(e); // TODO: In the future some operation may want to bail out early. @@ -302,6 +312,11 @@ public class RSProcedureDispatcher } } + @Override + public void dispatchServerOperations(MasterProcedureEnv env, List operations) { + operations.stream().map(o -> o.buildRequest()).forEachOrdered(request::addProc); + } + protected ExecuteProceduresResponse sendRequest(final ServerName serverName, final ExecuteProceduresRequest request) throws IOException { try { @@ -311,17 +326,8 @@ public class RSProcedureDispatcher } } - - private void remoteCallCompleted(final MasterProcedureEnv env, - final ExecuteProceduresResponse response) { - /* - for (RemoteProcedure proc: operations) { - proc.remoteCallCompleted(env, getServerName(), response); - }*/ - } - private void remoteCallFailed(final MasterProcedureEnv env, final IOException e) { - for (RemoteProcedure proc: remoteProcedures) { + for (RemoteProcedure proc : remoteProcedures) { proc.remoteCallFailed(env, getServerName(), e); } } @@ -362,8 +368,7 @@ public class RSProcedureDispatcher buildOpenRegionRequest(procedureEnv, getServerName(), operations); try { - OpenRegionResponse response = sendRequest(getServerName(), request); - remoteCallCompleted(procedureEnv, response); + sendRequest(getServerName(), request); } catch (IOException e) { e = unwrapException(e); // TODO: In the future some operation may want to bail out early. @@ -384,16 +389,6 @@ public class RSProcedureDispatcher } } - private void remoteCallCompleted(final MasterProcedureEnv env, - final OpenRegionResponse response) { - int index = 0; - for (RegionOpenOperation op: operations) { - OpenRegionResponse.RegionOpeningState state = response.getOpeningState(index++); - op.setFailedOpen(state == OpenRegionResponse.RegionOpeningState.FAILED_OPENING); - op.getRemoteProcedure().remoteCallCompleted(env, getServerName(), op); - } - } - private void remoteCallFailed(final MasterProcedureEnv env, final IOException e) { for (RegionOpenOperation op: operations) { op.getRemoteProcedure().remoteCallFailed(env, getServerName(), e); @@ -443,7 +438,6 @@ public class RSProcedureDispatcher private void remoteCallCompleted(final MasterProcedureEnv env, final CloseRegionResponse response) { operation.setClosed(response.getClosed()); - operation.getRemoteProcedure().remoteCallCompleted(env, getServerName(), operation); } private void remoteCallFailed(final MasterProcedureEnv env, final IOException e) { @@ -482,6 +476,11 @@ public class RSProcedureDispatcher submitTask(new CloseRegionRemoteCall(serverName, op)); } } + + @Override + public void dispatchServerOperations(MasterProcedureEnv env, List operations) { + throw new UnsupportedOperationException(); + } } // ========================================================================== @@ -489,13 +488,28 @@ public class RSProcedureDispatcher // - ServerOperation: refreshConfig, grant, revoke, ... (TODO) // - RegionOperation: open, close, flush, snapshot, ... // ========================================================================== - /* Currently unused - public static abstract class ServerOperation extends RemoteOperation { - protected ServerOperation(final RemoteProcedure remoteProcedure) { + + public static final class ServerOperation extends RemoteOperation { + + private final long procId; + + private final Class rsProcClass; + + private final byte[] rsProcData; + + public ServerOperation(RemoteProcedure remoteProcedure, long procId, Class rsProcClass, + byte[] rsProcData) { super(remoteProcedure); + this.procId = procId; + this.rsProcClass = rsProcClass; + this.rsProcData = rsProcData; + } + + public RemoteProcedureRequest buildRequest() { + return RemoteProcedureRequest.newBuilder().setProcId(procId) + .setProcClass(rsProcClass.getName()).setProcData(ByteString.copyFrom(rsProcData)).build(); } } - */ public static abstract class RegionOperation extends RemoteOperation { private final RegionInfo regionInfo; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AddPeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AddPeerProcedure.java new file mode 100644 index 0000000..f0f7704 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AddPeerProcedure.java @@ -0,0 +1,98 @@ +/** + * 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.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; +import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer; +import org.apache.hadoop.hbase.replication.ReplicationException; +import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.AddPeerStateData; + +/** + * The procedure for adding a new replication peer. + */ +@InterfaceAudience.Private +public class AddPeerProcedure extends ModifyPeerProcedure { + + private static final Logger LOG = LoggerFactory.getLogger(AddPeerProcedure.class); + + private ReplicationPeerConfig peerConfig; + + private boolean enabled; + + public AddPeerProcedure() { + } + + public AddPeerProcedure(String peerId, ReplicationPeerConfig peerConfig, boolean enabled) { + super(peerId); + this.peerConfig = peerConfig; + this.enabled = enabled; + } + + @Override + public PeerOperationType getPeerOperationType() { + return PeerOperationType.ADD; + } + + @Override + protected void prePeerModification(MasterProcedureEnv env) + throws IOException, ReplicationException { + MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost(); + if (cpHost != null) { + cpHost.preAddReplicationPeer(peerId, peerConfig); + } + env.getReplicationPeerManager().preAddPeer(peerId, peerConfig); + } + + @Override + protected void updatePeerStorage(MasterProcedureEnv env) throws ReplicationException { + env.getReplicationPeerManager().addPeer(peerId, peerConfig, enabled); + } + + @Override + protected void postPeerModification(MasterProcedureEnv env) throws IOException { + LOG.info("Successfully added {} peer {}, config {}", enabled ? "ENABLED" : "DISABLED", peerId, + peerConfig); + MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost(); + if (cpHost != null) { + env.getMasterCoprocessorHost().postAddReplicationPeer(peerId, peerConfig); + } + } + + @Override + protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException { + super.serializeStateData(serializer); + serializer.serialize(AddPeerStateData.newBuilder() + .setPeerConfig(ReplicationPeerConfigUtil.convert(peerConfig)).setEnabled(enabled).build()); + } + + @Override + protected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException { + super.deserializeStateData(serializer); + AddPeerStateData data = serializer.deserialize(AddPeerStateData.class); + peerConfig = ReplicationPeerConfigUtil.convert(data.getPeerConfig()); + enabled = data.getEnabled(); + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/DisablePeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/DisablePeerProcedure.java new file mode 100644 index 0000000..0871575 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/DisablePeerProcedure.java @@ -0,0 +1,70 @@ +/** + * 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.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; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * The procedure for disabling a replication peer. + */ +@InterfaceAudience.Private +public class DisablePeerProcedure extends ModifyPeerProcedure { + + private static final Logger LOG = LoggerFactory.getLogger(DisablePeerProcedure.class); + + public DisablePeerProcedure() { + } + + public DisablePeerProcedure(String peerId) { + super(peerId); + } + + @Override + public PeerOperationType getPeerOperationType() { + return PeerOperationType.DISABLE; + } + + @Override + protected void prePeerModification(MasterProcedureEnv env) throws IOException { + MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost(); + if (cpHost != null) { + cpHost.preDisableReplicationPeer(peerId); + } + env.getReplicationPeerManager().preDisablePeer(peerId); + } + + @Override + protected void updatePeerStorage(MasterProcedureEnv env) throws ReplicationException { + env.getReplicationPeerManager().disablePeer(peerId); + } + + @Override + protected void postPeerModification(MasterProcedureEnv env) throws IOException { + LOG.info("Successfully disabled peer {}", peerId); + MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost(); + if (cpHost != null) { + cpHost.postDisableReplicationPeer(peerId); + } + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/EnablePeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/EnablePeerProcedure.java new file mode 100644 index 0000000..890462f --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/EnablePeerProcedure.java @@ -0,0 +1,70 @@ +/** + * 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.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; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * The procedure for enabling a replication peer. + */ +@InterfaceAudience.Private +public class EnablePeerProcedure extends ModifyPeerProcedure { + + private static final Logger LOG = LoggerFactory.getLogger(EnablePeerProcedure.class); + + public EnablePeerProcedure() { + } + + public EnablePeerProcedure(String peerId) { + super(peerId); + } + + @Override + public PeerOperationType getPeerOperationType() { + return PeerOperationType.ENABLE; + } + + @Override + protected void prePeerModification(MasterProcedureEnv env) throws IOException { + MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost(); + if (cpHost != null) { + cpHost.preEnableReplicationPeer(peerId); + } + env.getReplicationPeerManager().preEnablePeer(peerId); + } + + @Override + protected void updatePeerStorage(MasterProcedureEnv env) throws ReplicationException { + env.getReplicationPeerManager().enablePeer(peerId); + } + + @Override + protected void postPeerModification(MasterProcedureEnv env) throws IOException { + LOG.info("Successfully enabled peer {}", peerId); + MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost(); + if (cpHost != null) { + cpHost.postEnableReplicationPeer(peerId); + } + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java new file mode 100644 index 0000000..c225619 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java @@ -0,0 +1,214 @@ +/** + * 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.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; +import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer; +import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException; +import org.apache.hadoop.hbase.procedure2.ProcedureYieldException; +import org.apache.hadoop.hbase.procedure2.StateMachineProcedure; +import org.apache.hadoop.hbase.replication.ReplicationException; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyPeerStateData; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.PeerModificationState; + +/** + * The base class for all replication peer related procedure. + */ +@InterfaceAudience.Private +public abstract class ModifyPeerProcedure + extends StateMachineProcedure + implements PeerProcedureInterface { + + private static final Logger LOG = LoggerFactory.getLogger(ModifyPeerProcedure.class); + + protected String peerId; + + private volatile boolean locked; + + // used to keep compatible with old client where we can only returns after updateStorage. + protected ProcedurePrepareLatch latch; + + protected ModifyPeerProcedure() { + } + + protected ModifyPeerProcedure(String peerId) { + this.peerId = peerId; + this.latch = ProcedurePrepareLatch.createLatch(2, 0); + } + + public ProcedurePrepareLatch getLatch() { + return latch; + } + + @Override + public String getPeerId() { + return peerId; + } + + /** + * Called before we start the actual processing. The implementation should call the pre CP hook, + * and also the pre-check for the peer modification. + *

    + * If an IOException is thrown then we will give up and mark the procedure as failed directly. If + * all checks passes then the procedure can not be rolled back any more. + */ + protected abstract void prePeerModification(MasterProcedureEnv env) + throws IOException, ReplicationException; + + protected abstract void updatePeerStorage(MasterProcedureEnv env) throws ReplicationException; + + /** + * Called before we finish the procedure. The implementation can do some logging work, and also + * call the coprocessor hook if any. + *

    + * 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, ReplicationException; + + private void releaseLatch() { + ProcedurePrepareLatch.releaseLatch(latch, this); + } + + @Override + protected Flow executeFromState(MasterProcedureEnv env, PeerModificationState state) + throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException { + switch (state) { + case PRE_PEER_MODIFICATION: + try { + prePeerModification(env); + } catch (IOException 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("{} failed to call prePeerModification for peer {}, retry", getClass().getName(), + peerId, e); + throw new ProcedureYieldException(); + } + setNextState(PeerModificationState.UPDATE_PEER_STORAGE); + return Flow.HAS_MORE_STATE; + case UPDATE_PEER_STORAGE: + try { + updatePeerStorage(env); + } catch (ReplicationException e) { + LOG.warn("{} update peer storage for peer {} failed, retry", getClass().getName(), peerId, + e); + throw new ProcedureYieldException(); + } + setNextState(PeerModificationState.REFRESH_PEER_ON_RS); + return Flow.HAS_MORE_STATE; + case REFRESH_PEER_ON_RS: + addChildProcedure(env.getMasterServices().getServerManager().getOnlineServersList().stream() + .map(sn -> new RefreshPeerProcedure(peerId, getPeerOperationType(), sn)) + .toArray(RefreshPeerProcedure[]::new)); + setNextState(PeerModificationState.POST_PEER_MODIFICATION); + return Flow.HAS_MORE_STATE; + 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("{} 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; + default: + throw new UnsupportedOperationException("unhandled state=" + state); + } + } + + @Override + protected LockState acquireLock(MasterProcedureEnv env) { + if (env.getProcedureScheduler().waitPeerExclusiveLock(this, peerId)) { + return LockState.LOCK_EVENT_WAIT; + } + locked = true; + return LockState.LOCK_ACQUIRED; + } + + @Override + protected void releaseLock(MasterProcedureEnv env) { + locked = false; + env.getProcedureScheduler().wakePeerExclusiveLock(this, peerId); + } + + @Override + protected boolean holdLock(MasterProcedureEnv env) { + return true; + } + + @Override + protected boolean hasLock(MasterProcedureEnv env) { + return locked; + } + + @Override + protected void rollbackState(MasterProcedureEnv env, PeerModificationState state) + 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 yet, we can just return. + return; + } + throw new UnsupportedOperationException(); + } + + @Override + protected PeerModificationState getState(int stateId) { + return PeerModificationState.forNumber(stateId); + } + + @Override + protected int getStateId(PeerModificationState state) { + return state.getNumber(); + } + + @Override + protected PeerModificationState getInitialState() { + return PeerModificationState.PRE_PEER_MODIFICATION; + } + + @Override + protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException { + super.serializeStateData(serializer); + serializer.serialize(ModifyPeerStateData.newBuilder().setPeerId(peerId).build()); + } + + @Override + protected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException { + super.deserializeStateData(serializer); + peerId = serializer.deserialize(ModifyPeerStateData.class).getPeerId(); + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerProcedure.java new file mode 100644 index 0000000..1253ef9 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerProcedure.java @@ -0,0 +1,204 @@ +/** + * 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.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; +import org.apache.hadoop.hbase.master.procedure.RSProcedureDispatcher.ServerOperation; +import org.apache.hadoop.hbase.procedure2.Procedure; +import org.apache.hadoop.hbase.procedure2.ProcedureEvent; +import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer; +import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException; +import org.apache.hadoop.hbase.procedure2.ProcedureYieldException; +import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteOperation; +import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteProcedure; +import org.apache.hadoop.hbase.procedure2.RemoteProcedureException; +import org.apache.hadoop.hbase.replication.regionserver.RefreshPeerCallable; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.PeerModificationType; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RefreshPeerParameter; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RefreshPeerStateData; + +@InterfaceAudience.Private +public class RefreshPeerProcedure extends Procedure + implements PeerProcedureInterface, RemoteProcedure { + + private static final Logger LOG = LoggerFactory.getLogger(RefreshPeerProcedure.class); + + private String peerId; + + private PeerOperationType type; + + private ServerName targetServer; + + private boolean dispatched; + + private ProcedureEvent event; + + private boolean succ; + + public RefreshPeerProcedure() { + } + + public RefreshPeerProcedure(String peerId, PeerOperationType type, ServerName targetServer) { + this.peerId = peerId; + this.type = type; + this.targetServer = targetServer; + } + + @Override + public String getPeerId() { + return peerId; + } + + @Override + public PeerOperationType getPeerOperationType() { + return PeerOperationType.REFRESH; + } + + private static PeerModificationType toPeerModificationType(PeerOperationType type) { + switch (type) { + case ADD: + return PeerModificationType.ADD_PEER; + case REMOVE: + return PeerModificationType.REMOVE_PEER; + case ENABLE: + return PeerModificationType.ENABLE_PEER; + case DISABLE: + return PeerModificationType.DISABLE_PEER; + case UPDATE_CONFIG: + return PeerModificationType.UPDATE_PEER_CONFIG; + default: + throw new IllegalArgumentException("Unknown type: " + type); + } + } + + private static PeerOperationType toPeerOperationType(PeerModificationType type) { + switch (type) { + case ADD_PEER: + return PeerOperationType.ADD; + case REMOVE_PEER: + return PeerOperationType.REMOVE; + case ENABLE_PEER: + return PeerOperationType.ENABLE; + case DISABLE_PEER: + return PeerOperationType.DISABLE; + case UPDATE_PEER_CONFIG: + return PeerOperationType.UPDATE_CONFIG; + default: + throw new IllegalArgumentException("Unknown type: " + type); + } + } + + @Override + public RemoteOperation remoteCallBuild(MasterProcedureEnv env, ServerName remote) { + assert targetServer.equals(remote); + return new ServerOperation(this, getProcId(), RefreshPeerCallable.class, + RefreshPeerParameter.newBuilder().setPeerId(peerId).setType(toPeerModificationType(type)) + .setTargetServer(ProtobufUtil.toServerName(remote)).build().toByteArray()); + } + + private void complete(MasterProcedureEnv env, Throwable error) { + if (event == null) { + LOG.warn("procedure event for {} is null, maybe the procedure is created when recovery", + getProcId()); + return; + } + if (error != null) { + LOG.warn("Refresh peer {} for {} on {} failed", peerId, type, targetServer, error); + this.succ = false; + } else { + LOG.info("Refresh peer {} for {} on {} suceeded", peerId, type, targetServer); + this.succ = true; + } + + event.wake(env.getProcedureScheduler()); + event = null; + } + + @Override + public synchronized void remoteCallFailed(MasterProcedureEnv env, ServerName remote, + IOException exception) { + complete(env, exception); + } + + @Override + public synchronized void remoteOperationCompleted(MasterProcedureEnv env) { + complete(env, null); + } + + @Override + public synchronized void remoteOperationFailed(MasterProcedureEnv env, + RemoteProcedureException error) { + complete(env, error); + } + + @Override + protected synchronized Procedure[] execute(MasterProcedureEnv env) + throws ProcedureYieldException, ProcedureSuspendedException, InterruptedException { + if (dispatched) { + if (succ) { + return null; + } + // retry + dispatched = false; + } + if (!env.getRemoteDispatcher().addOperationToNode(targetServer, this)) { + 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; + event = new ProcedureEvent<>(this); + event.suspendIfNotReady(this); + throw new ProcedureSuspendedException(); + } + + @Override + protected void rollback(MasterProcedureEnv env) throws IOException, InterruptedException { + throw new UnsupportedOperationException(); + } + + @Override + protected boolean abort(MasterProcedureEnv env) { + // TODO: no correctness problem if we just ignore this, implement later. + return false; + } + + @Override + protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException { + serializer.serialize( + RefreshPeerStateData.newBuilder().setPeerId(peerId).setType(toPeerModificationType(type)) + .setTargetServer(ProtobufUtil.toServerName(targetServer)).build()); + } + + @Override + protected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException { + RefreshPeerStateData data = serializer.deserialize(RefreshPeerStateData.class); + peerId = data.getPeerId(); + type = toPeerOperationType(data.getType()); + targetServer = ProtobufUtil.toServerName(data.getTargetServer()); + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RemovePeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RemovePeerProcedure.java new file mode 100644 index 0000000..64faf2b --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RemovePeerProcedure.java @@ -0,0 +1,72 @@ +/** + * 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.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; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * The procedure for removing a replication peer. + */ +@InterfaceAudience.Private +public class RemovePeerProcedure extends ModifyPeerProcedure { + + private static final Logger LOG = LoggerFactory.getLogger(RemovePeerProcedure.class); + + public RemovePeerProcedure() { + } + + public RemovePeerProcedure(String peerId) { + super(peerId); + } + + @Override + public PeerOperationType getPeerOperationType() { + return PeerOperationType.REMOVE; + } + + @Override + protected void prePeerModification(MasterProcedureEnv env) throws IOException { + MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost(); + if (cpHost != null) { + cpHost.preRemoveReplicationPeer(peerId); + } + env.getReplicationPeerManager().preRemovePeer(peerId); + } + + @Override + protected void updatePeerStorage(MasterProcedureEnv env) throws ReplicationException { + env.getReplicationPeerManager().removePeer(peerId); + } + + @Override + 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); + } + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationManager.java deleted file mode 100644 index f36b2e2..0000000 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationManager.java +++ /dev/null @@ -1,203 +0,0 @@ -/** - * 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.master.replication; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collection; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.regex.Pattern; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.Abortable; -import org.apache.hadoop.hbase.DoNotRetryIOException; -import org.apache.hadoop.hbase.ReplicationPeerNotFoundException; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.zookeeper.ZKWatcher; -import org.apache.hadoop.hbase.replication.BaseReplicationEndpoint; -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.ReplicationPeerDescription; -import org.apache.hadoop.hbase.replication.ReplicationPeers; -import org.apache.hadoop.hbase.replication.ReplicationQueuesClient; -import org.apache.hadoop.hbase.replication.ReplicationQueuesClientArguments; -import org.apache.yetus.audience.InterfaceAudience; - -/** - * Manages and performs all replication admin operations. - * Used to add/remove a replication peer. - */ -@InterfaceAudience.Private -public class ReplicationManager { - - private final Configuration conf; - private final ZKWatcher zkw; - private final ReplicationQueuesClient replicationQueuesClient; - private final ReplicationPeers replicationPeers; - - public ReplicationManager(Configuration conf, ZKWatcher zkw, Abortable abortable) - throws IOException { - this.conf = conf; - this.zkw = zkw; - try { - this.replicationQueuesClient = ReplicationFactory - .getReplicationQueuesClient(new ReplicationQueuesClientArguments(conf, abortable, zkw)); - this.replicationQueuesClient.init(); - this.replicationPeers = ReplicationFactory.getReplicationPeers(zkw, conf, - this.replicationQueuesClient, abortable); - this.replicationPeers.init(); - } catch (Exception e) { - throw new IOException("Failed to construct ReplicationManager", e); - } - } - - public void addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled) - throws ReplicationException, IOException { - checkPeerConfig(peerConfig); - replicationPeers.registerPeer(peerId, peerConfig, enabled); - replicationPeers.peerConnected(peerId); - } - - public void removeReplicationPeer(String peerId) throws ReplicationException { - replicationPeers.peerDisconnected(peerId); - replicationPeers.unregisterPeer(peerId); - } - - public void enableReplicationPeer(String peerId) throws ReplicationException { - this.replicationPeers.enablePeer(peerId); - } - - public void disableReplicationPeer(String peerId) throws ReplicationException { - this.replicationPeers.disablePeer(peerId); - } - - public ReplicationPeerConfig getPeerConfig(String peerId) throws ReplicationException, - ReplicationPeerNotFoundException { - ReplicationPeerConfig peerConfig = replicationPeers.getReplicationPeerConfig(peerId); - if (peerConfig == null) { - throw new ReplicationPeerNotFoundException(peerId); - } - return peerConfig; - } - - public void updatePeerConfig(String peerId, ReplicationPeerConfig peerConfig) - throws ReplicationException, IOException { - checkPeerConfig(peerConfig); - this.replicationPeers.updatePeerConfig(peerId, peerConfig); - } - - public List listReplicationPeers(Pattern pattern) - throws ReplicationException { - List peers = new ArrayList<>(); - List peerIds = replicationPeers.getAllPeerIds(); - for (String peerId : peerIds) { - if (pattern == null || (pattern != null && pattern.matcher(peerId).matches())) { - peers.add(new ReplicationPeerDescription(peerId, replicationPeers - .getStatusOfPeerFromBackingStore(peerId), replicationPeers - .getReplicationPeerConfig(peerId))); - } - } - return peers; - } - - /** - * If replicate_all flag is true, it means all user tables will be replicated to peer cluster. - * Then allow config exclude namespaces or exclude table-cfs which can't be replicated to - * peer cluster. - * - * If replicate_all flag is false, it means all user tables can't be replicated to peer cluster. - * Then allow to config namespaces or table-cfs which will be replicated to peer cluster. - */ - private void checkPeerConfig(ReplicationPeerConfig peerConfig) - throws ReplicationException, IOException { - if (peerConfig.replicateAllUserTables()) { - if ((peerConfig.getNamespaces() != null && !peerConfig.getNamespaces().isEmpty()) - || (peerConfig.getTableCFsMap() != null && !peerConfig.getTableCFsMap().isEmpty())) { - throw new ReplicationException("Need clean namespaces or table-cfs config firstly" - + " when replicate_all flag is true"); - } - checkNamespacesAndTableCfsConfigConflict(peerConfig.getExcludeNamespaces(), - peerConfig.getExcludeTableCFsMap()); - } else { - if ((peerConfig.getExcludeNamespaces() != null - && !peerConfig.getExcludeNamespaces().isEmpty()) - || (peerConfig.getExcludeTableCFsMap() != null - && !peerConfig.getExcludeTableCFsMap().isEmpty())) { - throw new ReplicationException( - "Need clean exclude-namespaces or exclude-table-cfs config firstly" - + " when replicate_all flag is false"); - } - checkNamespacesAndTableCfsConfigConflict(peerConfig.getNamespaces(), - peerConfig.getTableCFsMap()); - } - checkConfiguredWALEntryFilters(peerConfig); - } - - /** - * Set a namespace in the peer config means that all tables in this namespace will be replicated - * to the peer cluster. - * 1. If peer config already has a namespace, then not allow set any table of this namespace - * to the peer config. - * 2. If peer config already has a table, then not allow set this table's namespace to the peer - * config. - * - * Set a exclude namespace in the peer config means that all tables in this namespace can't be - * replicated to the peer cluster. - * 1. If peer config already has a exclude namespace, then not allow set any exclude table of - * this namespace to the peer config. - * 2. If peer config already has a exclude table, then not allow set this table's namespace - * as a exclude namespace. - */ - private void checkNamespacesAndTableCfsConfigConflict(Set namespaces, - Map> tableCfs) throws ReplicationException { - if (namespaces == null || namespaces.isEmpty()) { - return; - } - if (tableCfs == null || tableCfs.isEmpty()) { - return; - } - for (Map.Entry> entry : tableCfs.entrySet()) { - TableName table = entry.getKey(); - if (namespaces.contains(table.getNamespaceAsString())) { - throw new ReplicationException("Table-cfs " + table + " is conflict with namespaces " - + table.getNamespaceAsString() + " in peer config"); - } - } - } - - private void checkConfiguredWALEntryFilters(ReplicationPeerConfig peerConfig) - throws IOException { - String filterCSV = peerConfig.getConfiguration(). - get(BaseReplicationEndpoint.REPLICATION_WALENTRYFILTER_CONFIG_KEY); - if (filterCSV != null && !filterCSV.isEmpty()){ - String [] filters = filterCSV.split(","); - for (String filter : filters) { - try { - Class clazz = Class.forName(filter); - Object o = clazz.newInstance(); - } catch (Exception e) { - throw new DoNotRetryIOException("Configured WALEntryFilter " + filter + - " could not be created. Failing add/update " + "peer operation.", e); - } - } - } - } -} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java new file mode 100644 index 0000000..696b2d7 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java @@ -0,0 +1,344 @@ +/** + * 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.master.replication; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +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; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.replication.BaseReplicationEndpoint; +import org.apache.hadoop.hbase.replication.ReplicationException; +import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; +import org.apache.hadoop.hbase.replication.ReplicationPeerConfigBuilder; +import org.apache.hadoop.hbase.replication.ReplicationPeerDescription; +import org.apache.hadoop.hbase.replication.ReplicationPeerStorage; +import org.apache.hadoop.hbase.replication.ReplicationQueueInfo; +import org.apache.hadoop.hbase.replication.ReplicationQueueStorage; +import org.apache.hadoop.hbase.replication.ReplicationStorageFactory; +import org.apache.hadoop.hbase.replication.ReplicationUtils; +import org.apache.hadoop.hbase.zookeeper.ZKConfig; +import org.apache.hadoop.hbase.zookeeper.ZKWatcher; +import org.apache.yetus.audience.InterfaceAudience; + +/** + * Manages and performs all replication admin operations. + *

    + * Used to add/remove a replication peer. + */ +@InterfaceAudience.Private +public class ReplicationPeerManager { + + private final ReplicationPeerStorage peerStorage; + + private final ReplicationQueueStorage queueStorage; + + private final ConcurrentMap peers; + + ReplicationPeerManager(ReplicationPeerStorage peerStorage, ReplicationQueueStorage queueStorage, + ConcurrentMap peers) { + this.peerStorage = peerStorage; + this.queueStorage = queueStorage; + this.peers = peers; + } + + private void checkQueuesDeleted(String peerId) + throws ReplicationException, DoNotRetryIOException { + for (ServerName replicator : queueStorage.getListOfReplicators()) { + List queueIds = queueStorage.getAllQueues(replicator); + for (String queueId : queueIds) { + ReplicationQueueInfo queueInfo = new ReplicationQueueInfo(queueId); + if (queueInfo.getPeerId().equals(peerId)) { + throw new DoNotRetryIOException("undeleted queue for peerId: " + peerId + + ", replicator: " + replicator + ", queueId: " + queueId); + } + } + } + if (queueStorage.getAllPeersFromHFileRefsQueue().contains(peerId)) { + throw new DoNotRetryIOException("Undeleted queue for peer " + peerId + " in hfile-refs"); + } + } + + public void preAddPeer(String peerId, ReplicationPeerConfig peerConfig) + throws DoNotRetryIOException, ReplicationException { + if (peerId.contains("-")) { + throw new DoNotRetryIOException("Found invalid peer name: " + peerId); + } + checkPeerConfig(peerConfig); + if (peers.containsKey(peerId)) { + throw new DoNotRetryIOException("Replication peer " + peerId + " already exists"); + } + // make sure that there is no queues with the same peer id. This may happen when we create a + // peer with the same id with a old deleted peer. If the replication queues for the old peer + // have not been cleaned up yet then we should not create the new peer, otherwise the old wal + // file may also be replicated. + checkQueuesDeleted(peerId); + } + + private ReplicationPeerDescription checkPeerExists(String peerId) throws DoNotRetryIOException { + ReplicationPeerDescription desc = peers.get(peerId); + if (desc == null) { + throw new DoNotRetryIOException("Replication peer " + peerId + " does not exist"); + } + return desc; + } + + public void preRemovePeer(String peerId) throws DoNotRetryIOException { + checkPeerExists(peerId); + } + + public void preEnablePeer(String peerId) throws DoNotRetryIOException { + ReplicationPeerDescription desc = checkPeerExists(peerId); + if (desc.isEnabled()) { + throw new DoNotRetryIOException("Replication peer " + peerId + " has already been enabled"); + } + } + + public void preDisablePeer(String peerId) throws DoNotRetryIOException { + ReplicationPeerDescription desc = checkPeerExists(peerId); + if (!desc.isEnabled()) { + throw new DoNotRetryIOException("Replication peer " + peerId + " has already been disabled"); + } + } + + public void preUpdatePeerConfig(String peerId, ReplicationPeerConfig peerConfig) + throws DoNotRetryIOException { + checkPeerConfig(peerConfig); + ReplicationPeerDescription desc = checkPeerExists(peerId); + ReplicationPeerConfig oldPeerConfig = desc.getPeerConfig(); + if (!StringUtils.isBlank(peerConfig.getClusterKey()) && + !peerConfig.getClusterKey().equals(oldPeerConfig.getClusterKey())) { + throw new DoNotRetryIOException( + "Changing the cluster key on an existing peer is not allowed. Existing key '" + + oldPeerConfig.getClusterKey() + "' for peer " + peerId + " does not match new key '" + + peerConfig.getClusterKey() + "'"); + } + + if (!StringUtils.isBlank(peerConfig.getReplicationEndpointImpl()) && + !peerConfig.getReplicationEndpointImpl().equals(oldPeerConfig.getReplicationEndpointImpl())) { + throw new DoNotRetryIOException("Changing the replication endpoint implementation class " + + "on an existing peer is not allowed. Existing class '" + + oldPeerConfig.getReplicationEndpointImpl() + "' for peer " + peerId + + " does not match new class '" + peerConfig.getReplicationEndpointImpl() + "'"); + } + } + + public void addPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled) + throws ReplicationException { + if (peers.containsKey(peerId)) { + // this should be a retry, just return + return; + } + ReplicationPeerConfig copiedPeerConfig = ReplicationPeerConfig.newBuilder(peerConfig).build(); + peerStorage.addPeer(peerId, copiedPeerConfig, enabled); + peers.put(peerId, new ReplicationPeerDescription(peerId, enabled, copiedPeerConfig)); + } + + public void removePeer(String peerId) throws ReplicationException { + if (!peers.containsKey(peerId)) { + // this should be a retry, just return + return; + } + peerStorage.removePeer(peerId); + peers.remove(peerId); + } + + private void setPeerState(String peerId, boolean enabled) throws ReplicationException { + ReplicationPeerDescription desc = peers.get(peerId); + if (desc.isEnabled() == enabled) { + // this should be a retry, just return + return; + } + peerStorage.setPeerState(peerId, enabled); + peers.put(peerId, new ReplicationPeerDescription(peerId, enabled, desc.getPeerConfig())); + } + + public void enablePeer(String peerId) throws ReplicationException { + setPeerState(peerId, true); + } + + public void disablePeer(String peerId) throws ReplicationException { + setPeerState(peerId, false); + } + + public void updatePeerConfig(String peerId, ReplicationPeerConfig peerConfig) + throws ReplicationException { + // the checking rules are too complicated here so we give up checking whether this is a retry. + ReplicationPeerDescription desc = peers.get(peerId); + ReplicationPeerConfig oldPeerConfig = desc.getPeerConfig(); + ReplicationPeerConfigBuilder newPeerConfigBuilder = + ReplicationPeerConfig.newBuilder(peerConfig); + // we need to use the new conf to overwrite the old one. + newPeerConfigBuilder.putAllConfiguration(oldPeerConfig.getConfiguration()); + newPeerConfigBuilder.putAllConfiguration(peerConfig.getConfiguration()); + newPeerConfigBuilder.putAllConfiguration(oldPeerConfig.getConfiguration()); + newPeerConfigBuilder.putAllConfiguration(peerConfig.getConfiguration()); + ReplicationPeerConfig newPeerConfig = newPeerConfigBuilder.build(); + peerStorage.updatePeerConfig(peerId, newPeerConfig); + peers.put(peerId, new ReplicationPeerDescription(peerId, desc.isEnabled(), newPeerConfig)); + } + + public List listPeers(Pattern pattern) { + if (pattern == null) { + return new ArrayList<>(peers.values()); + } + return peers.values().stream().filter(r -> pattern.matcher(r.getPeerId()).matches()) + .collect(Collectors.toList()); + } + + public Optional getPeerConfig(String peerId) { + ReplicationPeerDescription desc = peers.get(peerId); + return desc != null ? Optional.of(desc.getPeerConfig()) : Optional.empty(); + } + + 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. + ReplicationUtils.removeAllQueues(queueStorage, peerId); + ReplicationUtils.removeAllQueues(queueStorage, peerId); + queueStorage.removePeerFromHFileRefs(peerId); + } + + private void checkPeerConfig(ReplicationPeerConfig peerConfig) throws DoNotRetryIOException { + checkClusterKey(peerConfig.getClusterKey()); + + if (peerConfig.replicateAllUserTables()) { + // If replicate_all flag is true, it means all user tables will be replicated to peer cluster. + // Then allow config exclude namespaces or exclude table-cfs which can't be replicated to peer + // cluster. + if ((peerConfig.getNamespaces() != null && !peerConfig.getNamespaces().isEmpty()) + || (peerConfig.getTableCFsMap() != null && !peerConfig.getTableCFsMap().isEmpty())) { + throw new DoNotRetryIOException("Need clean namespaces or table-cfs config firstly " + + "when you want replicate all cluster"); + } + checkNamespacesAndTableCfsConfigConflict(peerConfig.getExcludeNamespaces(), + peerConfig.getExcludeTableCFsMap()); + } else { + // If replicate_all flag is false, it means all user tables can't be replicated to peer + // cluster. Then allow to config namespaces or table-cfs which will be replicated to peer + // cluster. + if ((peerConfig.getExcludeNamespaces() != null + && !peerConfig.getExcludeNamespaces().isEmpty()) + || (peerConfig.getExcludeTableCFsMap() != null + && !peerConfig.getExcludeTableCFsMap().isEmpty())) { + throw new DoNotRetryIOException( + "Need clean exclude-namespaces or exclude-table-cfs config firstly" + + " when replicate_all flag is false"); + } + checkNamespacesAndTableCfsConfigConflict(peerConfig.getNamespaces(), + peerConfig.getTableCFsMap()); + } + + checkConfiguredWALEntryFilters(peerConfig); + } + + /** + * Set a namespace in the peer config means that all tables in this namespace will be replicated + * to the peer cluster. + *

      + *
    1. If peer config already has a namespace, then not allow set any table of this namespace to + * the peer config.
    2. + *
    3. If peer config already has a table, then not allow set this table's namespace to the peer + * config.
    4. + *
    + *

    + * Set a exclude namespace in the peer config means that all tables in this namespace can't be + * replicated to the peer cluster. + *

      + *
    1. If peer config already has a exclude namespace, then not allow set any exclude table of + * this namespace to the peer config.
    2. + *
    3. If peer config already has a exclude table, then not allow set this table's namespace as a + * exclude namespace.
    4. + *
    + */ + private void checkNamespacesAndTableCfsConfigConflict(Set namespaces, + Map> tableCfs) throws DoNotRetryIOException { + if (namespaces == null || namespaces.isEmpty()) { + return; + } + if (tableCfs == null || tableCfs.isEmpty()) { + return; + } + for (Map.Entry> entry : tableCfs.entrySet()) { + TableName table = entry.getKey(); + if (namespaces.contains(table.getNamespaceAsString())) { + throw new DoNotRetryIOException("Table-cfs " + table + " is conflict with namespaces " + + table.getNamespaceAsString() + " in peer config"); + } + } + } + + private void checkConfiguredWALEntryFilters(ReplicationPeerConfig peerConfig) + throws DoNotRetryIOException { + String filterCSV = peerConfig.getConfiguration() + .get(BaseReplicationEndpoint.REPLICATION_WALENTRYFILTER_CONFIG_KEY); + if (filterCSV != null && !filterCSV.isEmpty()) { + String[] filters = filterCSV.split(","); + for (String filter : filters) { + try { + Class.forName(filter).newInstance(); + } catch (Exception e) { + throw new DoNotRetryIOException("Configured WALEntryFilter " + filter + + " could not be created. Failing add/update " + "peer operation.", e); + } + } + } + } + + private void checkClusterKey(String clusterKey) throws DoNotRetryIOException { + try { + ZKConfig.validateClusterKey(clusterKey); + } catch (IOException e) { + throw new DoNotRetryIOException("Invalid cluster key: " + clusterKey, e); + } + } + + public static ReplicationPeerManager create(ZKWatcher zk, Configuration conf) + throws ReplicationException { + ReplicationPeerStorage peerStorage = + ReplicationStorageFactory.getReplicationPeerStorage(zk, conf); + ConcurrentMap peers = new ConcurrentHashMap<>(); + for (String peerId : peerStorage.listPeerIds()) { + ReplicationPeerConfig peerConfig = peerStorage.getPeerConfig(peerId); + boolean enabled = peerStorage.isPeerEnabled(peerId); + peers.put(peerId, new ReplicationPeerDescription(peerId, enabled, peerConfig)); + } + return new ReplicationPeerManager(peerStorage, + ReplicationStorageFactory.getReplicationQueueStorage(zk, conf), peers); + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/UpdatePeerConfigProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/UpdatePeerConfigProcedure.java new file mode 100644 index 0000000..3497447 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/UpdatePeerConfigProcedure.java @@ -0,0 +1,92 @@ +/** + * 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.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; +import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer; +import org.apache.hadoop.hbase.replication.ReplicationException; +import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.UpdatePeerConfigStateData; + +/** + * The procedure for updating the config for a replication peer. + */ +@InterfaceAudience.Private +public class UpdatePeerConfigProcedure extends ModifyPeerProcedure { + + private static final Logger LOG = LoggerFactory.getLogger(UpdatePeerConfigProcedure.class); + + private ReplicationPeerConfig peerConfig; + + public UpdatePeerConfigProcedure() { + } + + public UpdatePeerConfigProcedure(String peerId, ReplicationPeerConfig peerConfig) { + super(peerId); + this.peerConfig = peerConfig; + } + + @Override + public PeerOperationType getPeerOperationType() { + return PeerOperationType.UPDATE_CONFIG; + } + + @Override + protected void prePeerModification(MasterProcedureEnv env) throws IOException { + MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost(); + if (cpHost != null) { + cpHost.preUpdateReplicationPeerConfig(peerId, peerConfig); + } + env.getReplicationPeerManager().preUpdatePeerConfig(peerId, peerConfig); + } + + @Override + protected void updatePeerStorage(MasterProcedureEnv env) throws ReplicationException { + env.getReplicationPeerManager().updatePeerConfig(peerId, peerConfig); + } + + @Override + protected void postPeerModification(MasterProcedureEnv env) throws IOException { + LOG.info("Successfully updated peer config of {} to {}", peerId, peerConfig); + MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost(); + if (cpHost != null) { + cpHost.postUpdateReplicationPeerConfig(peerId, peerConfig); + } + } + + @Override + protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException { + super.serializeStateData(serializer); + serializer.serialize(UpdatePeerConfigStateData.newBuilder() + .setPeerConfig(ReplicationPeerConfigUtil.convert(peerConfig)).build()); + } + + @Override + protected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException { + super.deserializeStateData(serializer); + peerConfig = ReplicationPeerConfigUtil + .convert(serializer.deserialize(UpdatePeerConfigStateData.class).getPeerConfig()); + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure2/RSProcedureCallable.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure2/RSProcedureCallable.java new file mode 100644 index 0000000..62c2e36 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure2/RSProcedureCallable.java @@ -0,0 +1,43 @@ +/** + * 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.procedure2; + +import java.util.concurrent.Callable; + +import org.apache.hadoop.hbase.executor.EventType; +import org.apache.hadoop.hbase.regionserver.HRegionServer; +import org.apache.yetus.audience.InterfaceAudience; + +/** + * A general interface for a sub procedure runs at RS side. + */ +@InterfaceAudience.Private +public interface RSProcedureCallable extends Callable { + + /** + * Initialize the callable + * @param parameter the parameter passed from master. + * @param rs the regionserver instance + */ + void init(byte[] parameter, HRegionServer rs); + + /** + * Event type used to select thread pool. + */ + EventType getEventType(); +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java index 63451c6..449119d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -17,9 +17,6 @@ */ package org.apache.hadoop.hbase.regionserver; -import javax.management.MalformedObjectNameException; -import javax.management.ObjectName; -import javax.servlet.http.HttpServlet; import java.io.IOException; import java.lang.Thread.UncaughtExceptionHandler; import java.lang.management.MemoryType; @@ -50,7 +47,9 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.function.Function; - +import javax.management.MalformedObjectNameException; +import javax.management.ObjectName; +import javax.servlet.http.HttpServlet; import org.apache.commons.lang3.RandomUtils; import org.apache.commons.lang3.SystemUtils; import org.apache.hadoop.conf.Configuration; @@ -117,6 +116,7 @@ import org.apache.hadoop.hbase.master.LoadBalancer; import org.apache.hadoop.hbase.master.RegionState.State; import org.apache.hadoop.hbase.mob.MobCacheConfig; import org.apache.hadoop.hbase.procedure.RegionServerProcedureManagerHost; +import org.apache.hadoop.hbase.procedure2.RSProcedureCallable; import org.apache.hadoop.hbase.quotas.FileSystemUtilizationChore; import org.apache.hadoop.hbase.quotas.QuotaUtil; import org.apache.hadoop.hbase.quotas.RegionServerRpcQuotaManager; @@ -127,6 +127,7 @@ import org.apache.hadoop.hbase.regionserver.compactions.CompactionProgress; import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequester; import org.apache.hadoop.hbase.regionserver.handler.CloseMetaHandler; import org.apache.hadoop.hbase.regionserver.handler.CloseRegionHandler; +import org.apache.hadoop.hbase.regionserver.handler.RSProcedureHandler; import org.apache.hadoop.hbase.regionserver.handler.RegionReplicaFlushHandler; import org.apache.hadoop.hbase.regionserver.throttle.FlushThroughputControllerFactory; import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController; @@ -173,6 +174,9 @@ import org.apache.yetus.audience.InterfaceAudience; import org.apache.zookeeper.KeeperException; import org.slf4j.Logger; 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; @@ -181,6 +185,7 @@ 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; @@ -206,13 +211,11 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProto import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition; import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode; +import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportProcedureDoneRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse; -import sun.misc.Signal; -import sun.misc.SignalHandler; - /** * HRegionServer makes a set of HRegions available to clients. It checks in with * the HMaster. There are many HRegionServers in a single HBase deployment. @@ -376,6 +379,9 @@ public class HRegionServer extends HasThread implements // eclipse warning when accessed by inner classes protected LogRoller walRoller; + // A thread which calls reportProcedureDone + private RemoteProcedureResultReporter procedureResultReporter; + // flag set after we're done setting up server threads final AtomicBoolean online = new AtomicBoolean(false); @@ -1895,6 +1901,7 @@ public class HRegionServer extends HasThread implements this.walRoller = new LogRoller(this, this); this.flushThroughputController = FlushThroughputControllerFactory.create(this, conf); + this.procedureResultReporter = new RemoteProcedureResultReporter(this); // Create the CompactedFileDischarger chore executorService. This chore helps to // remove the compacted files @@ -1932,10 +1939,14 @@ public class HRegionServer extends HasThread implements conf.getInt("hbase.regionserver.region.replica.flusher.threads", conf.getInt("hbase.regionserver.executor.openregion.threads", 3))); } + this.executorService.startExecutorService(ExecutorType.RS_REFRESH_PEER, + conf.getInt("hbase.regionserver.executor.refresh.peer.threads", 2)); Threads.setDaemonThreadRunning(this.walRoller.getThread(), getName() + ".logRoller", uncaughtExceptionHandler); this.cacheFlusher.start(uncaughtExceptionHandler); + Threads.setDaemonThreadRunning(this.procedureResultReporter, + getName() + ".procedureResultReporter", uncaughtExceptionHandler); if (this.compactionChecker != null) choreService.scheduleChore(compactionChecker); if (this.periodicFlusher != null) choreService.scheduleChore(periodicFlusher); @@ -3725,4 +3736,31 @@ public class HRegionServer extends HasThread implements return ConnectionUtils.createShortCircuitConnection(conf, null, user, this.serverName, this.rpcServices, this.rpcServices); } + + public void executeProcedure(long procId, RSProcedureCallable callable) { + executorService.submit(new RSProcedureHandler(this, procId, callable)); + } + + public void remoteProcedureComplete(long procId, Throwable error) { + procedureResultReporter.complete(procId, error); + } + + void reportProcedureDone(ReportProcedureDoneRequest request) throws IOException { + RegionServerStatusService.BlockingInterface rss = rssStub; + for (;;) { + rss = rssStub; + if (rss != null) { + break; + } + createRegionServerStatusStub(); + } + try { + rss.reportProcedureDone(null, request); + } catch (ServiceException se) { + if (rssStub == rss) { + rssStub = null; + } + throw ProtobufUtil.getRemoteException(se); + } + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java index b6c0ebe..695b859 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -42,7 +41,6 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.LongAdder; - import org.apache.commons.lang3.mutable.MutableObject; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; @@ -100,6 +98,7 @@ import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException; import org.apache.hadoop.hbase.ipc.ServerRpcController; import org.apache.hadoop.hbase.log.HBaseMarkers; import org.apache.hadoop.hbase.master.MasterRpcServices; +import org.apache.hadoop.hbase.procedure2.RSProcedureCallable; import org.apache.hadoop.hbase.quotas.ActivePolicyEnforcement; import org.apache.hadoop.hbase.quotas.OperationQuota; import org.apache.hadoop.hbase.quotas.QuotaUtil; @@ -142,6 +141,7 @@ 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.ResponseConverter; @@ -172,6 +172,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionR import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest.RegionOpenInfo; import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponse.RegionOpeningState; +import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RemoteProcedureRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterRequest; @@ -3435,23 +3436,6 @@ public class RSRpcServices implements HBaseRPCErrorHandler, } @Override - public ExecuteProceduresResponse executeProcedures(RpcController controller, - ExecuteProceduresRequest request) throws ServiceException { - ExecuteProceduresResponse.Builder builder = ExecuteProceduresResponse.newBuilder(); - if (request.getOpenRegionCount() > 0) { - for (OpenRegionRequest req : request.getOpenRegionList()) { - builder.addOpenRegion(openRegion(controller, req)); - } - } - if (request.getCloseRegionCount() > 0) { - for (CloseRegionRequest req : request.getCloseRegionList()) { - builder.addCloseRegion(closeRegion(controller, req)); - } - } - return builder.build(); - } - - @Override public ClearRegionBlockCacheResponse clearRegionBlockCache(RpcController controller, ClearRegionBlockCacheRequest request) { ClearRegionBlockCacheResponse.Builder builder = @@ -3468,4 +3452,42 @@ public class RSRpcServices implements HBaseRPCErrorHandler, stats.withMaxCacheSize(regionServer.getCacheConfig().getBlockCache().getMaxSize()); return builder.setStats(ProtobufUtil.toCacheEvictionStats(stats.build())).build(); } + + @Override + @QosPriority(priority = HConstants.ADMIN_QOS) + public ExecuteProceduresResponse executeProcedures(RpcController controller, + ExecuteProceduresRequest request) throws ServiceException { + try { + checkOpen(); + regionServer.getRegionServerCoprocessorHost().preExecuteProcedures(); + if (request.getOpenRegionCount() > 0) { + for (OpenRegionRequest req : request.getOpenRegionList()) { + openRegion(controller, req); + } + } + if (request.getCloseRegionCount() > 0) { + for (CloseRegionRequest req : request.getCloseRegionList()) { + closeRegion(controller, req); + } + } + if (request.getProcCount() > 0) { + for (RemoteProcedureRequest req : request.getProcList()) { + RSProcedureCallable callable; + try { + callable = + Class.forName(req.getProcClass()).asSubclass(RSProcedureCallable.class).newInstance(); + } catch (Exception e) { + regionServer.remoteProcedureComplete(req.getProcId(), e); + continue; + } + callable.init(req.getProcData().toByteArray(), regionServer); + regionServer.executeProcedure(req.getProcId(), callable); + } + } + regionServer.getRegionServerCoprocessorHost().postExecuteProcedures(); + return ExecuteProceduresResponse.getDefaultInstance(); + } catch (IOException e) { + throw new ServiceException(e); + } + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerCoprocessorHost.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerCoprocessorHost.java index dc1708c..09617c4 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerCoprocessorHost.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerCoprocessorHost.java @@ -205,6 +205,24 @@ public class RegionServerCoprocessorHost extends }); } + public void preExecuteProcedures() throws IOException { + execOperation(coprocEnvironments.isEmpty() ? null : new RegionServerObserverOperation() { + @Override + public void call(RegionServerObserver observer) throws IOException { + observer.preExecuteProcedures(this); + } + }); + } + + public void postExecuteProcedures() throws IOException { + execOperation(coprocEnvironments.isEmpty() ? null : new RegionServerObserverOperation() { + @Override + public void call(RegionServerObserver observer) throws IOException { + observer.postExecuteProcedures(this); + } + }); + } + /** * Coprocessor environment extension providing access to region server * related services. diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RemoteProcedureResultReporter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RemoteProcedureResultReporter.java new file mode 100644 index 0000000..ac3e95a --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RemoteProcedureResultReporter.java @@ -0,0 +1,112 @@ +/** + * 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.regionserver; + +import java.io.IOException; +import java.util.concurrent.LinkedBlockingQueue; +import org.apache.hadoop.hbase.PleaseHoldException; +import org.apache.hadoop.hbase.client.ConnectionUtils; +import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException; +import org.apache.hadoop.hbase.util.ForeignExceptionUtil; +import org.apache.hadoop.hbase.util.Threads; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +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; + +/** + * A thread which calls {@code reportProcedureDone} to tell master the result of a remote procedure. + */ +@InterfaceAudience.Private +class RemoteProcedureResultReporter extends Thread { + + private static final Logger LOG = LoggerFactory.getLogger(RemoteProcedureResultReporter.class); + + // Time to pause if master says 'please hold'. Make configurable if needed. + private static final int INIT_PAUSE_TIME_MS = 1000; + + private static final int MAX_BATCH = 100; + + private final HRegionServer server; + + private final LinkedBlockingQueue results = new LinkedBlockingQueue<>(); + + public RemoteProcedureResultReporter(HRegionServer server) { + this.server = server; + } + + public void complete(long procId, Throwable error) { + RemoteProcedureResult.Builder builder = RemoteProcedureResult.newBuilder().setProcId(procId); + if (error != null) { + builder.setStatus(RemoteProcedureResult.Status.ERROR).setError( + ForeignExceptionUtil.toProtoForeignException(server.getServerName().toString(), error)); + } else { + builder.setStatus(RemoteProcedureResult.Status.SUCCESS); + } + results.add(builder.build()); + } + + @Override + public void run() { + ReportProcedureDoneRequest.Builder builder = ReportProcedureDoneRequest.newBuilder(); + int tries = 0; + while (!server.isStopped()) { + if (builder.getResultCount() == 0) { + try { + builder.addResult(results.take()); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + continue; + } + } + while (builder.getResultCount() < MAX_BATCH) { + RemoteProcedureResult result = results.poll(); + if (result == null) { + break; + } + builder.addResult(result); + } + ReportProcedureDoneRequest request = builder.build(); + try { + server.reportProcedureDone(builder.build()); + builder.clear(); + tries = 0; + } catch (IOException e) { + boolean pause = + e instanceof ServerNotRunningYetException || e instanceof PleaseHoldException; + long pauseTime; + if (pause) { + // Do backoff else we flood the Master with requests. + pauseTime = ConnectionUtils.getPauseTime(INIT_PAUSE_TIME_MS, tries); + } else { + pauseTime = INIT_PAUSE_TIME_MS; // Reset. + } + LOG.info("Failed report procedure " + TextFormat.shortDebugString(request) + "; retry (#" + + tries + ")" + (pause ? " after " + pauseTime + "ms delay (Master is coming online...)." + : " immediately."), + e); + Threads.sleep(pauseTime); + tries++; + } + } + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationSourceService.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationSourceService.java index 835ffbf..2aef0a8 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationSourceService.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationSourceService.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -18,6 +17,7 @@ */ package org.apache.hadoop.hbase.regionserver; +import org.apache.hadoop.hbase.replication.regionserver.PeerProcedureHandler; import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener; @@ -29,8 +29,14 @@ import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener; @InterfaceAudience.Private public interface ReplicationSourceService extends ReplicationService { /** - * Returns a WALObserver for the service. This is needed to + * Returns a WALObserver for the service. This is needed to * observe log rolls and log archival events. */ WALActionsListener getWALActionsListener(); + + + /** + * Returns a Handler to handle peer procedures. + */ + PeerProcedureHandler getPeerProcedureHandler(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/RSProcedureHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/RSProcedureHandler.java new file mode 100644 index 0000000..d2175d0 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/RSProcedureHandler.java @@ -0,0 +1,54 @@ +/** + * 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.regionserver.handler; + +import org.apache.hadoop.hbase.executor.EventHandler; +import org.apache.hadoop.hbase.procedure2.RSProcedureCallable; +import org.apache.hadoop.hbase.regionserver.HRegionServer; +import org.apache.log4j.Logger; +import org.apache.yetus.audience.InterfaceAudience; + +/** + * A event handler for running procedure. + */ +@InterfaceAudience.Private +public class RSProcedureHandler extends EventHandler { + + private static final Logger LOG = Logger.getLogger(RSProcedureHandler.class); + private final long procId; + + private final RSProcedureCallable callable; + + public RSProcedureHandler(HRegionServer rs, long procId, RSProcedureCallable callable) { + super(rs, callable.getEventType()); + this.procId = procId; + this.callable = callable; + } + + @Override + public void process() { + Exception error = null; + try { + callable.call(); + } catch (Exception e) { + LOG.error("Catch exception when call RSProcedureCallable: ", e); + error = e; + } + ((HRegionServer) server).remoteProcedureComplete(procId, error); + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/BaseReplicationEndpoint.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/BaseReplicationEndpoint.java index 2093421..7b91bee 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/BaseReplicationEndpoint.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/BaseReplicationEndpoint.java @@ -49,7 +49,7 @@ public abstract class BaseReplicationEndpoint extends AbstractService if (this.ctx != null){ ReplicationPeer peer = this.ctx.getReplicationPeer(); if (peer != null){ - peer.trackPeerConfigChanges(this); + peer.registerPeerConfigListener(this); } else { LOG.warn("Not tracking replication peer config changes for Peer Id " + this.ctx.getPeerId() + " because there's no such peer"); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationHFileCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationHFileCleaner.java index 5f1df44..7b62169 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationHFileCleaner.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationHFileCleaner.java @@ -1,42 +1,43 @@ -/* - * 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. +/** + * 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.master; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; -import org.apache.hbase.thirdparty.com.google.common.base.Predicate; -import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableSet; -import org.apache.hbase.thirdparty.com.google.common.collect.Iterables; -import org.apache.hbase.thirdparty.com.google.common.collect.Sets; - import java.io.IOException; import java.util.Collections; -import java.util.List; import java.util.Set; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.hbase.Abortable; import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.HConstants; -import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.master.cleaner.BaseHFileCleanerDelegate; import org.apache.hadoop.hbase.master.cleaner.HFileCleaner; -import org.apache.hadoop.hbase.replication.ReplicationFactory; -import org.apache.hadoop.hbase.replication.ReplicationQueuesClient; -import org.apache.hadoop.hbase.replication.ReplicationQueuesClientArguments; +import org.apache.hadoop.hbase.replication.ReplicationException; +import org.apache.hadoop.hbase.replication.ReplicationQueueStorage; +import org.apache.hadoop.hbase.replication.ReplicationStorageFactory; import org.apache.hadoop.hbase.zookeeper.ZKWatcher; -import org.apache.zookeeper.KeeperException; +import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; +import org.apache.hbase.thirdparty.com.google.common.base.Predicate; +import org.apache.hbase.thirdparty.com.google.common.collect.Iterables; + /** * Implementation of a file cleaner that checks if a hfile is still scheduled for replication before * deleting it from hfile archive directory. @@ -45,7 +46,7 @@ import org.slf4j.LoggerFactory; public class ReplicationHFileCleaner extends BaseHFileCleanerDelegate { private static final Logger LOG = LoggerFactory.getLogger(ReplicationHFileCleaner.class); private ZKWatcher zkw; - private ReplicationQueuesClient rqc; + private ReplicationQueueStorage rqs; private boolean stopped = false; @Override @@ -60,8 +61,8 @@ public class ReplicationHFileCleaner extends BaseHFileCleanerDelegate { try { // The concurrently created new hfile entries in ZK may not be included in the return list, // but they won't be deleted because they're not in the checking set. - hfileRefs = loadHFileRefsFromPeers(); - } catch (KeeperException e) { + hfileRefs = rqs.getAllHFileRefs(); + } catch (ReplicationException e) { LOG.warn("Failed to read hfile references from zookeeper, skipping checking deletable files"); return Collections.emptyList(); } @@ -82,37 +83,6 @@ public class ReplicationHFileCleaner extends BaseHFileCleanerDelegate { }); } - /** - * Load all hfile references in all replication queues from ZK. This method guarantees to return a - * snapshot which contains all hfile references in the zookeeper at the start of this call. - * However, some newly created hfile references during the call may not be included. - */ - private Set loadHFileRefsFromPeers() throws KeeperException { - Set hfileRefs = Sets.newHashSet(); - List listOfPeers; - for (int retry = 0;; retry++) { - int v0 = rqc.getHFileRefsNodeChangeVersion(); - hfileRefs.clear(); - listOfPeers = rqc.getAllPeersFromHFileRefsQueue(); - if (listOfPeers == null) { - LOG.debug("Didn't find any peers with hfile references, won't prevent any deletions."); - return ImmutableSet.of(); - } - for (String id : listOfPeers) { - List peerHFileRefs = rqc.getReplicableHFiles(id); - if (peerHFileRefs != null) { - hfileRefs.addAll(peerHFileRefs); - } - } - int v1 = rqc.getHFileRefsNodeChangeVersion(); - if (v0 == v1) { - return hfileRefs; - } - LOG.debug(String.format("Replication hfile references node cversion changed from " - + "%d to %d, retry = %d", v0, v1, retry)); - } - } - @Override public void setConf(Configuration config) { // If either replication or replication of bulk load hfiles is disabled, keep all members null @@ -139,17 +109,15 @@ public class ReplicationHFileCleaner extends BaseHFileCleanerDelegate { public void setConf(Configuration conf, ZKWatcher zk) { super.setConf(conf); try { - initReplicationQueuesClient(conf, zk); + initReplicationQueueStorage(conf, zk); } catch (Exception e) { LOG.error("Error while configuring " + this.getClass().getName(), e); } } - private void initReplicationQueuesClient(Configuration conf, ZKWatcher zk) - throws Exception { + private void initReplicationQueueStorage(Configuration conf, ZKWatcher zk) { this.zkw = zk; - this.rqc = ReplicationFactory.getReplicationQueuesClient(new ReplicationQueuesClientArguments( - conf, new WarnOnlyAbortable(), zkw)); + this.rqs = ReplicationStorageFactory.getReplicationQueueStorage(zk, conf); } @Override @@ -179,25 +147,12 @@ public class ReplicationHFileCleaner extends BaseHFileCleanerDelegate { } try { - hfileRefsFromQueue = loadHFileRefsFromPeers(); - } catch (KeeperException e) { + hfileRefsFromQueue = rqs.getAllHFileRefs(); + } catch (ReplicationException e) { LOG.warn("Failed to read hfile references from zookeeper, skipping checking deletable " + "file for " + fStat.getPath()); return false; } return !hfileRefsFromQueue.contains(fStat.getPath().getName()); } - - private static class WarnOnlyAbortable implements Abortable { - @Override - public void abort(String why, Throwable e) { - LOG.warn("ReplicationHFileCleaner received abort, ignoring. Reason: " + why); - LOG.debug(e.toString(), e); - } - - @Override - public boolean isAborted() { - return false; - } - } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java index 86f98da..15aa21a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -24,16 +23,14 @@ import java.util.Set; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.hbase.Abortable; import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.master.cleaner.BaseLogCleanerDelegate; -import org.apache.hadoop.hbase.replication.ReplicationFactory; -import org.apache.hadoop.hbase.replication.ReplicationQueuesClient; -import org.apache.hadoop.hbase.replication.ReplicationQueuesClientArguments; +import org.apache.hadoop.hbase.replication.ReplicationException; +import org.apache.hadoop.hbase.replication.ReplicationQueueStorage; +import org.apache.hadoop.hbase.replication.ReplicationStorageFactory; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.zookeeper.ZKWatcher; import org.apache.yetus.audience.InterfaceAudience; -import org.apache.zookeeper.KeeperException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -49,7 +46,7 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Iterables; public class ReplicationLogCleaner extends BaseLogCleanerDelegate { private static final Logger LOG = LoggerFactory.getLogger(ReplicationLogCleaner.class); private ZKWatcher zkw; - private ReplicationQueuesClient replicationQueues; + private ReplicationQueueStorage queueStorage; private boolean stopped = false; private Set wals; private long readZKTimestamp = 0; @@ -60,8 +57,8 @@ public class ReplicationLogCleaner extends BaseLogCleanerDelegate { try { // The concurrently created new WALs may not be included in the return list, // but they won't be deleted because they're not in the checking set. - wals = replicationQueues.getAllWALs(); - } catch (KeeperException e) { + wals = queueStorage.getAllWALs(); + } catch (ReplicationException e) { LOG.warn("Failed to read zookeeper, skipping checking deletable files"); wals = null; } @@ -110,9 +107,7 @@ public class ReplicationLogCleaner extends BaseLogCleanerDelegate { super.setConf(conf); try { this.zkw = zk; - this.replicationQueues = ReplicationFactory.getReplicationQueuesClient( - new ReplicationQueuesClientArguments(conf, new WarnOnlyAbortable(), zkw)); - this.replicationQueues.init(); + this.queueStorage = ReplicationStorageFactory.getReplicationQueueStorage(zk, conf); } catch (Exception e) { LOG.error("Error while configuring " + this.getClass().getName(), e); } @@ -132,18 +127,4 @@ public class ReplicationLogCleaner extends BaseLogCleanerDelegate { public boolean isStopped() { return this.stopped; } - - private static class WarnOnlyAbortable implements Abortable { - - @Override - public void abort(String why, Throwable e) { - LOG.warn("ReplicationLogCleaner received abort, ignoring. Reason: " + why); - LOG.debug(e.toString(), e); - } - - @Override - public boolean isAborted() { - return false; - } - } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationPeerConfigUpgrader.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationPeerConfigUpgrader.java index ea5509f..f524fd7 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationPeerConfigUpgrader.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationPeerConfigUpgrader.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -18,96 +17,108 @@ */ package org.apache.hadoop.hbase.replication.master; -import java.io.IOException; -import java.util.List; +import static org.apache.hadoop.hbase.replication.ZKReplicationPeerStorage.PEERS_ZNODE; +import static org.apache.hadoop.hbase.replication.ZKReplicationPeerStorage.PEERS_ZNODE_DEFAULT; +import static org.apache.hadoop.hbase.replication.ZKReplicationStorageBase.REPLICATION_ZNODE; +import static org.apache.hadoop.hbase.replication.ZKReplicationStorageBase.REPLICATION_ZNODE_DEFAULT; +import java.io.IOException; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.Abortable; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil; -import org.apache.hadoop.hbase.exceptions.DeserializationException; +import org.apache.hadoop.hbase.replication.ReplicationException; import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; -import org.apache.hadoop.hbase.replication.ReplicationStateZKBase; +import org.apache.hadoop.hbase.replication.ReplicationPeerStorage; +import org.apache.hadoop.hbase.replication.ReplicationStorageFactory; import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hbase.zookeeper.ZKWatcher; +import org.apache.hadoop.hbase.zookeeper.ZNodePaths; import org.apache.yetus.audience.InterfaceAudience; import org.apache.yetus.audience.InterfaceStability; import org.apache.zookeeper.KeeperException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + +import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; + import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos; /** - * This class is used to upgrade TableCFs from HBase 1.0, 1.1, 1.2, 1.3 to HBase 1.4 or 2.x. - * It will be removed in HBase 3.x. See HBASE-11393 + * This class is used to upgrade TableCFs from HBase 1.0, 1.1, 1.2, 1.3 to HBase 1.4 or 2.x. It will + * be removed in HBase 3.x. See HBASE-11393 */ @InterfaceAudience.Private @InterfaceStability.Unstable -public class ReplicationPeerConfigUpgrader extends ReplicationStateZKBase { +public class ReplicationPeerConfigUpgrader{ + + private static final String TABLE_CFS_ZNODE = "zookeeper.znode.replication.peers.tableCFs"; + private static final String TABLE_CFS_ZNODE_DEFAULT = "tableCFs"; private static final Logger LOG = LoggerFactory.getLogger(ReplicationPeerConfigUpgrader.class); + private final Configuration conf; + private final ZKWatcher zookeeper; + private final ReplicationPeerStorage peerStorage; - public ReplicationPeerConfigUpgrader(ZKWatcher zookeeper, - Configuration conf, Abortable abortable) { - super(zookeeper, conf, abortable); + public ReplicationPeerConfigUpgrader(ZKWatcher zookeeper, Configuration conf) { + this.zookeeper = zookeeper; + this.conf = conf; + this.peerStorage = ReplicationStorageFactory.getReplicationPeerStorage(zookeeper, conf); } public void upgrade() throws Exception { try (Connection conn = ConnectionFactory.createConnection(conf)) { Admin admin = conn.getAdmin(); - admin.listReplicationPeers().forEach( - (peerDesc) -> { - String peerId = peerDesc.getPeerId(); - ReplicationPeerConfig peerConfig = peerDesc.getPeerConfig(); - if ((peerConfig.getNamespaces() != null && !peerConfig.getNamespaces().isEmpty()) - || (peerConfig.getTableCFsMap() != null && !peerConfig.getTableCFsMap().isEmpty())) { - peerConfig.setReplicateAllUserTables(false); - try { - admin.updateReplicationPeerConfig(peerId, peerConfig); - } catch (Exception e) { - LOG.error("Failed to upgrade replication peer config for peerId=" + peerId, e); - } + admin.listReplicationPeers().forEach((peerDesc) -> { + String peerId = peerDesc.getPeerId(); + ReplicationPeerConfig peerConfig = peerDesc.getPeerConfig(); + if ((peerConfig.getNamespaces() != null && !peerConfig.getNamespaces().isEmpty()) + || (peerConfig.getTableCFsMap() != null && !peerConfig.getTableCFsMap().isEmpty())) { + peerConfig.setReplicateAllUserTables(false); + try { + admin.updateReplicationPeerConfig(peerId, peerConfig); + } catch (Exception e) { + LOG.error("Failed to upgrade replication peer config for peerId=" + peerId, e); } - }); + } + }); } } - public void copyTableCFs() { - List znodes = null; - try { - znodes = ZKUtil.listChildrenNoWatch(this.zookeeper, this.peersZNode); - } catch (KeeperException e) { - LOG.error("Failed to get peers znode", e); - } - if (znodes != null) { - for (String peerId : znodes) { - if (!copyTableCFs(peerId)) { - LOG.error("upgrade tableCFs failed for peerId=" + peerId); - } + public void copyTableCFs() throws ReplicationException { + for (String peerId : peerStorage.listPeerIds()) { + if (!copyTableCFs(peerId)) { + LOG.error("upgrade tableCFs failed for peerId=" + peerId); } } } - public boolean copyTableCFs(String peerId) { + @VisibleForTesting + protected String getTableCFsNode(String peerId) { + String replicationZNode = ZNodePaths.joinZNode(zookeeper.znodePaths.baseZNode, + conf.get(REPLICATION_ZNODE, REPLICATION_ZNODE_DEFAULT)); + String peersZNode = + ZNodePaths.joinZNode(replicationZNode, conf.get(PEERS_ZNODE, PEERS_ZNODE_DEFAULT)); + return ZNodePaths.joinZNode(peersZNode, + ZNodePaths.joinZNode(peerId, conf.get(TABLE_CFS_ZNODE, TABLE_CFS_ZNODE_DEFAULT))); + } + + public boolean copyTableCFs(String peerId) throws ReplicationException { String tableCFsNode = getTableCFsNode(peerId); try { if (ZKUtil.checkExists(zookeeper, tableCFsNode) != -1) { - String peerNode = getPeerNode(peerId); - ReplicationPeerConfig rpc = getReplicationPeerConig(peerNode); + ReplicationPeerConfig rpc = peerStorage.getPeerConfig(peerId); // We only need to copy data from tableCFs node to rpc Node the first time hmaster start. if (rpc.getTableCFsMap() == null || rpc.getTableCFsMap().isEmpty()) { // we copy TableCFs node into PeerNode LOG.info("copy tableCFs into peerNode:" + peerId); ReplicationProtos.TableCF[] tableCFs = - ReplicationPeerConfigUtil.parseTableCFs( - ZKUtil.getData(this.zookeeper, tableCFsNode)); + ReplicationPeerConfigUtil.parseTableCFs(ZKUtil.getData(this.zookeeper, tableCFsNode)); if (tableCFs != null && tableCFs.length > 0) { rpc.setTableCFsMap(ReplicationPeerConfigUtil.convert2Map(tableCFs)); - ZKUtil.setData(this.zookeeper, peerNode, - ReplicationPeerConfigUtil.toByteArray(rpc)); + peerStorage.updatePeerConfig(peerId, rpc); } } else { LOG.info("No tableCFs in peerNode:" + peerId); @@ -126,23 +137,6 @@ public class ReplicationPeerConfigUpgrader extends ReplicationStateZKBase { return true; } - private ReplicationPeerConfig getReplicationPeerConig(String peerNode) - throws KeeperException, InterruptedException { - byte[] data = null; - data = ZKUtil.getData(this.zookeeper, peerNode); - if (data == null) { - LOG.error("Could not get configuration for " + - "peer because it doesn't exist. peer=" + peerNode); - return null; - } - try { - return ReplicationPeerConfigUtil.parsePeerFrom(data); - } catch (DeserializationException e) { - LOG.warn("Failed to parse cluster key from peer=" + peerNode); - return null; - } - } - private static void printUsageAndExit() { System.err.printf( "Usage: hbase org.apache.hadoop.hbase.replication.master.ReplicationPeerConfigUpgrader" @@ -163,19 +157,17 @@ public class ReplicationPeerConfigUpgrader extends ReplicationStateZKBase { printUsageAndExit(); } else if (args[0].equals("copyTableCFs")) { Configuration conf = HBaseConfiguration.create(); - ZKWatcher zkw = new ZKWatcher(conf, "ReplicationPeerConfigUpgrader", null); - try { - ReplicationPeerConfigUpgrader tableCFsUpdater = new ReplicationPeerConfigUpgrader(zkw, - conf, null); + try (ZKWatcher zkw = new ZKWatcher(conf, "ReplicationPeerConfigUpgrader", null)) { + ReplicationPeerConfigUpgrader tableCFsUpdater = + new ReplicationPeerConfigUpgrader(zkw, conf); tableCFsUpdater.copyTableCFs(); - } finally { - zkw.close(); } } else if (args[0].equals("upgrade")) { Configuration conf = HBaseConfiguration.create(); - ZKWatcher zkw = new ZKWatcher(conf, "ReplicationPeerConfigUpgrader", null); - ReplicationPeerConfigUpgrader upgrader = new ReplicationPeerConfigUpgrader(zkw, conf, null); - upgrader.upgrade(); + try (ZKWatcher zkw = new ZKWatcher(conf, "ReplicationPeerConfigUpgrader", null)) { + ReplicationPeerConfigUpgrader upgrader = new ReplicationPeerConfigUpgrader(zkw, conf); + upgrader.upgrade(); + } } else { printUsageAndExit(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/DumpReplicationQueues.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/DumpReplicationQueues.java index 6e27a21..22e8628 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/DumpReplicationQueues.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/DumpReplicationQueues.java @@ -21,13 +21,13 @@ import java.io.FileNotFoundException; import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; +import java.util.HashSet; import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Queue; import java.util.Set; import java.util.stream.Collectors; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configured; import org.apache.hadoop.fs.FileStatus; @@ -46,19 +46,17 @@ import org.apache.hadoop.hbase.procedure2.util.StringUtils; import org.apache.hadoop.hbase.replication.ReplicationFactory; import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; import org.apache.hadoop.hbase.replication.ReplicationPeerDescription; -import org.apache.hadoop.hbase.replication.ReplicationPeers; import org.apache.hadoop.hbase.replication.ReplicationQueueInfo; -import org.apache.hadoop.hbase.replication.ReplicationQueues; -import org.apache.hadoop.hbase.replication.ReplicationQueuesClient; -import org.apache.hadoop.hbase.replication.ReplicationQueuesClientArguments; +import org.apache.hadoop.hbase.replication.ReplicationQueueStorage; +import org.apache.hadoop.hbase.replication.ReplicationStorageFactory; import org.apache.hadoop.hbase.replication.ReplicationTracker; import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hbase.zookeeper.ZKWatcher; import org.apache.hadoop.util.Tool; import org.apache.hadoop.util.ToolRunner; -import org.apache.zookeeper.KeeperException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.hbase.thirdparty.com.google.common.util.concurrent.AtomicLongMap; /** @@ -238,7 +236,7 @@ public class DumpReplicationQueues extends Configured implements Tool { LOG.info("Found [--distributed], will poll each RegionServer."); Set peerIds = peers.stream().map((peer) -> peer.getPeerId()) .collect(Collectors.toSet()); - System.out.println(dumpQueues(connection, zkw, peerIds, opts.isHdfs())); + System.out.println(dumpQueues(zkw, peerIds, opts.isHdfs())); System.out.println(dumpReplicationSummary()); } else { // use ZK instead @@ -302,58 +300,44 @@ public class DumpReplicationQueues extends Configured implements Tool { return sb.toString(); } - public String dumpQueues(ClusterConnection connection, ZKWatcher zkw, Set peerIds, - boolean hdfs) throws Exception { - ReplicationQueuesClient queuesClient; - ReplicationPeers replicationPeers; - ReplicationQueues replicationQueues; + public String dumpQueues(ZKWatcher zkw, Set peerIds, + boolean hdfs) throws Exception { + ReplicationQueueStorage queueStorage; ReplicationTracker replicationTracker; - ReplicationQueuesClientArguments replicationArgs = - new ReplicationQueuesClientArguments(getConf(), new WarnOnlyAbortable(), zkw); StringBuilder sb = new StringBuilder(); - queuesClient = ReplicationFactory.getReplicationQueuesClient(replicationArgs); - queuesClient.init(); - replicationQueues = ReplicationFactory.getReplicationQueues(replicationArgs); - replicationPeers = ReplicationFactory.getReplicationPeers(zkw, getConf(), queuesClient, connection); - replicationTracker = ReplicationFactory.getReplicationTracker(zkw, replicationPeers, getConf(), - new WarnOnlyAbortable(), new WarnOnlyStoppable()); - List liveRegionServers = replicationTracker.getListOfRegionServers(); + queueStorage = ReplicationStorageFactory.getReplicationQueueStorage(zkw, getConf()); + replicationTracker = ReplicationFactory.getReplicationTracker(zkw, new WarnOnlyAbortable(), + new WarnOnlyStoppable()); + Set liveRegionServers = new HashSet<>(replicationTracker.getListOfRegionServers()); // Loops each peer on each RS and dumps the queues - try { - List regionservers = queuesClient.getListOfReplicators(); - if (regionservers == null || regionservers.isEmpty()) { - return sb.toString(); + List regionservers = queueStorage.getListOfReplicators(); + if (regionservers == null || regionservers.isEmpty()) { + return sb.toString(); + } + for (ServerName regionserver : regionservers) { + List queueIds = queueStorage.getAllQueues(regionserver); + if (!liveRegionServers.contains(regionserver.getServerName())) { + deadRegionServers.add(regionserver.getServerName()); } - for (String regionserver : regionservers) { - List queueIds = queuesClient.getAllQueues(regionserver); - replicationQueues.init(regionserver); - if (!liveRegionServers.contains(regionserver)) { - deadRegionServers.add(regionserver); - } - for (String queueId : queueIds) { - ReplicationQueueInfo queueInfo = new ReplicationQueueInfo(queueId); - List wals = queuesClient.getLogsInQueue(regionserver, queueId); - if (!peerIds.contains(queueInfo.getPeerId())) { - deletedQueues.add(regionserver + "/" + queueId); - sb.append(formatQueue(regionserver, replicationQueues, queueInfo, queueId, wals, true, - hdfs)); - } else { - sb.append(formatQueue(regionserver, replicationQueues, queueInfo, queueId, wals, false, - hdfs)); - } + for (String queueId : queueIds) { + ReplicationQueueInfo queueInfo = new ReplicationQueueInfo(queueId); + List wals = queueStorage.getWALsInQueue(regionserver, queueId); + if (!peerIds.contains(queueInfo.getPeerId())) { + deletedQueues.add(regionserver + "/" + queueId); + sb.append(formatQueue(regionserver, queueStorage, queueInfo, queueId, wals, true, hdfs)); + } else { + sb.append(formatQueue(regionserver, queueStorage, queueInfo, queueId, wals, false, hdfs)); } } - } catch (KeeperException ke) { - throw new IOException(ke); } return sb.toString(); } - private String formatQueue(String regionserver, ReplicationQueues replicationQueues, ReplicationQueueInfo queueInfo, - String queueId, List wals, boolean isDeleted, boolean hdfs) throws Exception { - + private String formatQueue(ServerName regionserver, ReplicationQueueStorage queueStorage, + ReplicationQueueInfo queueInfo, String queueId, List wals, boolean isDeleted, + boolean hdfs) throws Exception { StringBuilder sb = new StringBuilder(); List deadServers; @@ -373,7 +357,7 @@ public class DumpReplicationQueues extends Configured implements Tool { peersQueueSize.addAndGet(queueInfo.getPeerId(), wals.size()); for (String wal : wals) { - long position = replicationQueues.getLogPosition(queueInfo.getPeerId(), wal); + long position = queueStorage.getWALPosition(regionserver, queueInfo.getPeerId(), wal); sb.append(" Replication position for " + wal + ": " + (position > 0 ? position : "0" + " (not started or nothing to replicate)") + "\n"); } @@ -389,13 +373,14 @@ public class DumpReplicationQueues extends Configured implements Tool { /** * return total size in bytes from a list of WALs */ - private long getTotalWALSize(FileSystem fs, List wals, String server) throws IOException { + private long getTotalWALSize(FileSystem fs, List wals, ServerName server) + throws IOException { long size = 0; FileStatus fileStatus; for (String wal : wals) { try { - fileStatus = (new WALLink(getConf(), server, wal)).getFileStatus(fs); + fileStatus = (new WALLink(getConf(), server.getServerName(), wal)).getFileStatus(fs); } catch (IOException e) { if (e instanceof FileNotFoundException) { numWalsNotFound++; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java index 40f59b7..1ef1198 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java @@ -18,8 +18,6 @@ package org.apache.hadoop.hbase.replication.regionserver; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - import java.io.IOException; import java.net.ConnectException; import java.net.SocketTimeoutException; @@ -38,7 +36,6 @@ import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; import java.util.regex.Matcher; import java.util.regex.Pattern; - import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; @@ -47,22 +44,24 @@ import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotFoundException; -import org.apache.yetus.audience.InterfaceAudience; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import org.apache.hadoop.hbase.client.ClusterConnection; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.ipc.RpcServer; import org.apache.hadoop.hbase.protobuf.ReplicationProtbufUtil; -import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface; import org.apache.hadoop.hbase.replication.HBaseReplicationEndpoint; -import org.apache.hadoop.hbase.replication.ReplicationPeer.PeerState; import org.apache.hadoop.hbase.replication.regionserver.ReplicationSinkManager.SinkPeer; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.wal.WAL.Entry; import org.apache.hadoop.ipc.RemoteException; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; + +import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface; /** * A {@link org.apache.hadoop.hbase.replication.ReplicationEndpoint} @@ -415,7 +414,7 @@ public class HBaseInterClusterReplicationEndpoint extends HBaseReplicationEndpoi } protected boolean isPeerEnabled() { - return ctx.getReplicationPeer().getPeerState() == PeerState.ENABLED; + return ctx.getReplicationPeer().isPeerEnabled(); } @Override diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandler.java new file mode 100644 index 0000000..65da9af --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandler.java @@ -0,0 +1,41 @@ +/** + * 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 java.io.IOException; + +import org.apache.hadoop.hbase.replication.ReplicationException; +import org.apache.yetus.audience.InterfaceAudience; + +/** + * A handler for modifying replication peer in peer procedures. + */ +@InterfaceAudience.Private +public interface PeerProcedureHandler { + + public void addPeer(String peerId) throws ReplicationException, IOException; + + public void removePeer(String peerId) throws ReplicationException, IOException; + + public void disablePeer(String peerId) throws ReplicationException, IOException; + + public void enablePeer(String peerId) throws ReplicationException, IOException; + + public void updatePeerConfig(String peerId) throws ReplicationException, IOException; +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandlerImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandlerImpl.java new file mode 100644 index 0000000..ce8fdae --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandlerImpl.java @@ -0,0 +1,110 @@ +/** + * 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 java.io.IOException; +import java.util.concurrent.locks.Lock; +import org.apache.hadoop.hbase.replication.ReplicationException; +import org.apache.hadoop.hbase.replication.ReplicationPeer.PeerState; +import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; +import org.apache.hadoop.hbase.replication.ReplicationPeerImpl; +import org.apache.hadoop.hbase.replication.ReplicationUtils; +import org.apache.hadoop.hbase.util.KeyLocker; +import org.apache.yetus.audience.InterfaceAudience; + +@InterfaceAudience.Private +public class PeerProcedureHandlerImpl implements PeerProcedureHandler { + + private final ReplicationSourceManager replicationSourceManager; + private final KeyLocker peersLock = new KeyLocker<>(); + + public PeerProcedureHandlerImpl(ReplicationSourceManager replicationSourceManager) { + this.replicationSourceManager = replicationSourceManager; + } + + @Override + public void addPeer(String peerId) throws IOException { + Lock peerLock = peersLock.acquireLock(peerId); + try { + replicationSourceManager.addPeer(peerId); + } finally { + peerLock.unlock(); + } + } + + @Override + public void removePeer(String peerId) throws IOException { + Lock peerLock = peersLock.acquireLock(peerId); + try { + if (replicationSourceManager.getReplicationPeers().getPeer(peerId) != null) { + replicationSourceManager.removePeer(peerId); + } + } finally { + peerLock.unlock(); + } + } + + private void refreshPeerState(String peerId) throws ReplicationException, IOException { + PeerState newState; + Lock peerLock = peersLock.acquireLock(peerId); + try { + ReplicationPeerImpl peer = replicationSourceManager.getReplicationPeers().getPeer(peerId); + if (peer == null) { + throw new ReplicationException("Peer with id=" + peerId + " is not cached."); + } + PeerState oldState = peer.getPeerState(); + newState = replicationSourceManager.getReplicationPeers().refreshPeerState(peerId); + // RS need to start work with the new replication state change + if (oldState.equals(PeerState.ENABLED) && newState.equals(PeerState.DISABLED)) { + replicationSourceManager.refreshSources(peerId); + } + } finally { + peerLock.unlock(); + } + } + + @Override + public void enablePeer(String peerId) throws ReplicationException, IOException { + refreshPeerState(peerId); + } + + @Override + public void disablePeer(String peerId) throws ReplicationException, IOException { + refreshPeerState(peerId); + } + + @Override + public void updatePeerConfig(String peerId) throws ReplicationException, IOException { + Lock peerLock = peersLock.acquireLock(peerId); + try { + ReplicationPeerImpl peer = replicationSourceManager.getReplicationPeers().getPeer(peerId); + if (peer == null) { + throw new ReplicationException("Peer with id=" + peerId + " is not cached."); + } + ReplicationPeerConfig oldConfig = peer.getPeerConfig(); + ReplicationPeerConfig newConfig = + replicationSourceManager.getReplicationPeers().refreshPeerConfig(peerId); + // RS need to start work with the new replication config change + if (!ReplicationUtils.isKeyConfigEqual(oldConfig, newConfig)) { + replicationSourceManager.refreshSources(peerId); + } + } finally { + peerLock.unlock(); + } + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSource.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSource.java index bd191e3..3cae0f2 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSource.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSource.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -29,15 +28,14 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.ServerName; -import org.apache.yetus.audience.InterfaceAudience; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.apache.hadoop.hbase.replication.ReplicationEndpoint; -import org.apache.hadoop.hbase.replication.ReplicationPeers; -import org.apache.hadoop.hbase.replication.ReplicationQueues; +import org.apache.hadoop.hbase.replication.ReplicationPeer; +import org.apache.hadoop.hbase.replication.ReplicationQueueStorage; import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.hbase.wal.AbstractFSWALProvider; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Class that handles the recovered source of a replication stream, which is transfered from @@ -52,11 +50,11 @@ public class RecoveredReplicationSource extends ReplicationSource { @Override public void init(Configuration conf, FileSystem fs, ReplicationSourceManager manager, - ReplicationQueues replicationQueues, ReplicationPeers replicationPeers, Server server, - String peerClusterZnode, UUID clusterId, ReplicationEndpoint replicationEndpoint, - WALFileLengthProvider walFileLengthProvider, MetricsSource metrics) throws IOException { - super.init(conf, fs, manager, replicationQueues, replicationPeers, server, peerClusterZnode, - clusterId, replicationEndpoint, walFileLengthProvider, metrics); + ReplicationQueueStorage queueStorage, ReplicationPeer replicationPeer, Server server, + String peerClusterZnode, UUID clusterId, WALFileLengthProvider walFileLengthProvider, + MetricsSource metrics) throws IOException { + super.init(conf, fs, manager, queueStorage, replicationPeer, server, peerClusterZnode, + clusterId, walFileLengthProvider, metrics); this.actualPeerId = this.replicationQueueInfo.getPeerId(); } @@ -64,13 +62,13 @@ public class RecoveredReplicationSource extends ReplicationSource { protected void tryStartNewShipper(String walGroupId, PriorityBlockingQueue queue) { final RecoveredReplicationSourceShipper worker = new RecoveredReplicationSourceShipper(conf, walGroupId, queue, this, - this.replicationQueues); + this.queueStorage); ReplicationSourceShipper extant = workerThreads.putIfAbsent(walGroupId, worker); if (extant != null) { LOG.debug("Someone has beat us to start a worker thread for wal group " + walGroupId); } else { LOG.debug("Starting up worker for wal group " + walGroupId); - worker.startup(getUncaughtExceptionHandler()); + worker.startup(this::uncaughtException); worker.setWALReader( startNewWALReader(worker.getName(), walGroupId, queue, worker.getStartPosition())); workerThreads.put(walGroupId, worker); @@ -78,13 +76,13 @@ public class RecoveredReplicationSource extends ReplicationSource { } @Override - protected ReplicationSourceWALReader startNewWALReader(String threadName, - String walGroupId, PriorityBlockingQueue queue, long startPosition) { - ReplicationSourceWALReader walReader = new RecoveredReplicationSourceWALReader(fs, - conf, queue, startPosition, walEntryFilter, this); - Threads.setDaemonThreadRunning(walReader, threadName - + ".replicationSource.replicationWALReaderThread." + walGroupId + "," + peerClusterZnode, - getUncaughtExceptionHandler()); + protected ReplicationSourceWALReader startNewWALReader(String threadName, String walGroupId, + PriorityBlockingQueue queue, long startPosition) { + ReplicationSourceWALReader walReader = + new RecoveredReplicationSourceWALReader(fs, conf, queue, startPosition, walEntryFilter, this); + Threads.setDaemonThreadRunning(walReader, + threadName + ".replicationSource.replicationWALReaderThread." + walGroupId + "," + queueId, + this::uncaughtException); return walReader; } @@ -180,8 +178,8 @@ public class RecoveredReplicationSource extends ReplicationSource { } } if (allTasksDone) { - manager.closeRecoveredQueue(this); - LOG.info("Finished recovering queue " + peerClusterZnode + " with the following stats: " + manager.removeRecoveredSource(this); + LOG.info("Finished recovering queue " + queueId + " with the following stats: " + getStats()); } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSourceShipper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSourceShipper.java index 630b90b..1e45496 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSourceShipper.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSourceShipper.java @@ -23,13 +23,13 @@ import java.util.concurrent.PriorityBlockingQueue; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; -import org.apache.yetus.audience.InterfaceAudience; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import org.apache.hadoop.hbase.replication.ReplicationException; -import org.apache.hadoop.hbase.replication.ReplicationQueues; +import org.apache.hadoop.hbase.replication.ReplicationQueueStorage; import org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceWALReader.WALEntryBatch; import org.apache.hadoop.hbase.util.Threads; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Used by a {@link RecoveredReplicationSource}. @@ -40,14 +40,14 @@ public class RecoveredReplicationSourceShipper extends ReplicationSourceShipper LoggerFactory.getLogger(RecoveredReplicationSourceShipper.class); protected final RecoveredReplicationSource source; - private final ReplicationQueues replicationQueues; + private final ReplicationQueueStorage replicationQueues; public RecoveredReplicationSourceShipper(Configuration conf, String walGroupId, PriorityBlockingQueue queue, RecoveredReplicationSource source, - ReplicationQueues replicationQueues) { + ReplicationQueueStorage queueStorage) { super(conf, walGroupId, queue, source); this.source = source; - this.replicationQueues = replicationQueues; + this.replicationQueues = queueStorage; } @Override @@ -77,7 +77,7 @@ public class RecoveredReplicationSourceShipper extends ReplicationSourceShipper if (entryBatch.getWalEntries().isEmpty() && entryBatch.getLastSeqIds().isEmpty()) { LOG.debug("Finished recovering queue for group " + walGroupId + " of peer " - + source.getPeerClusterZnode()); + + source.getQueueId()); source.getSourceMetrics().incrCompletedRecoveryQueue(); setWorkerState(WorkerState.FINISHED); continue; @@ -114,13 +114,13 @@ public class RecoveredReplicationSourceShipper extends ReplicationSourceShipper // normally has a position (unless the RS failed between 2 logs) private long getRecoveredQueueStartPos() { long startPosition = 0; - String peerClusterZnode = source.getPeerClusterZnode(); + String peerClusterZnode = source.getQueueId(); try { - startPosition = this.replicationQueues.getLogPosition(peerClusterZnode, - this.queue.peek().getName()); + startPosition = this.replicationQueues.getWALPosition(source.getServerWALsBelongTo(), + peerClusterZnode, this.queue.peek().getName()); if (LOG.isTraceEnabled()) { - LOG.trace("Recovered queue started with log " + this.queue.peek() + " at position " - + startPosition); + LOG.trace("Recovered queue started with log " + this.queue.peek() + " at position " + + startPosition); } } catch (ReplicationException e) { terminate("Couldn't get the position of this recovered queue " + peerClusterZnode, e); @@ -130,8 +130,8 @@ public class RecoveredReplicationSourceShipper extends ReplicationSourceShipper @Override protected void updateLogPosition(long lastReadPosition) { - source.getSourceManager().logPositionAndCleanOldLogs(currentPath, source.getPeerClusterZnode(), - lastReadPosition, true, false); + source.getSourceManager().logPositionAndCleanOldLogs(currentPath, source.getQueueId(), + lastReadPosition, true); lastLoggedPosition = lastReadPosition; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RefreshPeerCallable.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RefreshPeerCallable.java new file mode 100644 index 0000000..7ada24b --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RefreshPeerCallable.java @@ -0,0 +1,92 @@ +/** + * 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.hadoop.hbase.executor.EventType; +import org.apache.hadoop.hbase.procedure2.RSProcedureCallable; +import org.apache.hadoop.hbase.regionserver.HRegionServer; +import org.apache.log4j.Logger; +import org.apache.yetus.audience.InterfaceAudience; + +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; + +/** + * The callable executed at RS side to refresh the peer config/state.
    + */ +@InterfaceAudience.Private +public class RefreshPeerCallable implements RSProcedureCallable { + + private static final Logger LOG = Logger.getLogger(RefreshPeerCallable.class); + private HRegionServer rs; + + private String peerId; + + private PeerModificationType type; + + private Exception initError; + + @Override + public Void call() throws Exception { + if (initError != null) { + throw initError; + } + + LOG.info("Received a peer change event, peerId=" + peerId + ", type=" + type); + PeerProcedureHandler handler = rs.getReplicationSourceService().getPeerProcedureHandler(); + switch (type) { + case ADD_PEER: + handler.addPeer(this.peerId); + break; + case REMOVE_PEER: + handler.removePeer(this.peerId); + break; + case ENABLE_PEER: + handler.enablePeer(this.peerId); + break; + case DISABLE_PEER: + handler.disablePeer(this.peerId); + break; + case UPDATE_PEER_CONFIG: + handler.updatePeerConfig(this.peerId); + break; + default: + throw new IllegalArgumentException("Unknown peer modification type: " + type); + } + return null; + } + + @Override + public void init(byte[] parameter, HRegionServer rs) { + this.rs = rs; + try { + RefreshPeerParameter param = RefreshPeerParameter.parseFrom(parameter); + this.peerId = param.getPeerId(); + this.type = param.getType(); + } catch (InvalidProtocolBufferException e) { + initError = e; + } + } + + @Override + public EventType getEventType() { + return EventType.RS_REFRESH_PEER; + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java index dcd79a6..d1a3266 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -28,12 +27,6 @@ import java.util.UUID; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; - -import org.apache.hadoop.hbase.wal.WALKey; -import org.apache.hadoop.hbase.wal.WALKeyImpl; -import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; -import org.apache.hadoop.hbase.master.cleaner.HFileCleaner; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -43,27 +36,32 @@ import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WALEntry; -import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos; -import org.apache.hadoop.hbase.util.Pair; +import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; +import org.apache.hadoop.hbase.master.cleaner.HFileCleaner; import org.apache.hadoop.hbase.regionserver.ReplicationSinkService; import org.apache.hadoop.hbase.regionserver.ReplicationSourceService; -import org.apache.hadoop.hbase.replication.ReplicationException; +import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener; import org.apache.hadoop.hbase.replication.ReplicationFactory; import org.apache.hadoop.hbase.replication.ReplicationPeers; -import org.apache.hadoop.hbase.replication.ReplicationQueues; -import org.apache.hadoop.hbase.replication.ReplicationQueuesArguments; +import org.apache.hadoop.hbase.replication.ReplicationQueueStorage; +import org.apache.hadoop.hbase.replication.ReplicationStorageFactory; import org.apache.hadoop.hbase.replication.ReplicationTracker; -import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener; -import org.apache.hadoop.hbase.wal.WALEdit; import org.apache.hadoop.hbase.replication.master.ReplicationHFileCleaner; import org.apache.hadoop.hbase.replication.master.ReplicationLogCleaner; +import org.apache.hadoop.hbase.util.Pair; +import org.apache.hadoop.hbase.wal.WALEdit; +import org.apache.hadoop.hbase.wal.WALKey; +import org.apache.hadoop.hbase.wal.WALKeyImpl; import org.apache.hadoop.hbase.zookeeper.ZKClusterId; +import org.apache.yetus.audience.InterfaceAudience; import org.apache.zookeeper.KeeperException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WALEntry; +import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos; /** * Gateway to Replication. Used by {@link org.apache.hadoop.hbase.regionserver.HRegionServer}. */ @@ -74,7 +72,7 @@ public class Replication implements LoggerFactory.getLogger(Replication.class); private boolean replicationForBulkLoadData; private ReplicationSourceManager replicationManager; - private ReplicationQueues replicationQueues; + private ReplicationQueueStorage queueStorage; private ReplicationPeers replicationPeers; private ReplicationTracker replicationTracker; private Configuration conf; @@ -87,6 +85,8 @@ public class Replication implements // ReplicationLoad to access replication metrics private ReplicationLoad replicationLoad; + private PeerProcedureHandler peerProcedureHandler; + /** * Instantiate the replication management (if rep is enabled). * @param server Hosting server @@ -125,16 +125,13 @@ public class Replication implements } try { - this.replicationQueues = - ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(conf, this.server, - server.getZooKeeper())); - this.replicationQueues.init(this.server.getServerName().toString()); + this.queueStorage = + ReplicationStorageFactory.getReplicationQueueStorage(server.getZooKeeper(), conf); this.replicationPeers = - ReplicationFactory.getReplicationPeers(server.getZooKeeper(), this.conf, this.server); + ReplicationFactory.getReplicationPeers(server.getZooKeeper(), this.conf); this.replicationPeers.init(); this.replicationTracker = - ReplicationFactory.getReplicationTracker(server.getZooKeeper(), this.replicationPeers, - this.conf, this.server, this.server); + ReplicationFactory.getReplicationTracker(server.getZooKeeper(), this.server, this.server); } catch (Exception e) { throw new IOException("Failed replication handler create", e); } @@ -145,12 +142,14 @@ public class Replication implements throw new IOException("Could not read cluster id", ke); } this.replicationManager = - new ReplicationSourceManager(replicationQueues, replicationPeers, replicationTracker, conf, + new ReplicationSourceManager(queueStorage, replicationPeers, replicationTracker, conf, this.server, fs, logDir, oldLogDir, clusterId, walFileLengthProvider); this.statsThreadPeriod = this.conf.getInt("replication.stats.thread.period.seconds", 5 * 60); LOG.debug("ReplicationStatisticsThread " + this.statsThreadPeriod); this.replicationLoad = new ReplicationLoad(); + + this.peerProcedureHandler = new PeerProcedureHandlerImpl(replicationManager); } /** @@ -168,6 +167,12 @@ public class Replication implements public WALActionsListener getWALActionsListener() { return this; } + + @Override + public PeerProcedureHandler getPeerProcedureHandler() { + return peerProcedureHandler; + } + /** * Stops replication service. */ @@ -212,11 +217,7 @@ public class Replication implements * @throws IOException */ public void startReplicationService() throws IOException { - try { - this.replicationManager.init(); - } catch (ReplicationException e) { - throw new IOException(e); - } + this.replicationManager.init(); this.replicationSink = new ReplicationSink(this.conf, this.server); this.scheduleThreadPool.scheduleAtFixedRate( new ReplicationStatisticsThread(this.replicationSink, this.replicationManager), @@ -274,9 +275,9 @@ public class Replication implements throws IOException { try { this.replicationManager.addHFileRefs(tableName, family, pairs); - } catch (ReplicationException e) { + } catch (IOException e) { LOG.error("Failed to add hfile references in the replication queue.", e); - throw new IOException(e); + throw e; } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java index 16fb4a7..923d893 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -31,7 +30,6 @@ import java.util.concurrent.PriorityBlockingQueue; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicLong; - import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; @@ -40,20 +38,18 @@ import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.ServerName; -import org.apache.hadoop.hbase.Stoppable; +import org.apache.hadoop.hbase.TableDescriptors; import org.apache.hadoop.hbase.TableName; -import org.apache.yetus.audience.InterfaceAudience; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; +import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.RSRpcServices; +import org.apache.hadoop.hbase.regionserver.RegionServerCoprocessorHost; import org.apache.hadoop.hbase.replication.ChainWALEntryFilter; import org.apache.hadoop.hbase.replication.ClusterMarkingEntryFilter; import org.apache.hadoop.hbase.replication.ReplicationEndpoint; import org.apache.hadoop.hbase.replication.ReplicationException; import org.apache.hadoop.hbase.replication.ReplicationPeer; -import org.apache.hadoop.hbase.replication.ReplicationPeers; import org.apache.hadoop.hbase.replication.ReplicationQueueInfo; -import org.apache.hadoop.hbase.replication.ReplicationQueues; +import org.apache.hadoop.hbase.replication.ReplicationQueueStorage; import org.apache.hadoop.hbase.replication.SystemTableWALEntryFilter; import org.apache.hadoop.hbase.replication.WALEntryFilter; import org.apache.hadoop.hbase.util.Bytes; @@ -61,6 +57,10 @@ import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.hbase.wal.AbstractFSWALProvider; import org.apache.hadoop.hbase.wal.WAL.Entry; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import org.apache.hbase.thirdparty.com.google.common.collect.Lists; @@ -76,7 +76,7 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Lists; *

    */ @InterfaceAudience.Private -public class ReplicationSource extends Thread implements ReplicationSourceInterface { +public class ReplicationSource implements ReplicationSourceInterface { private static final Logger LOG = LoggerFactory.getLogger(ReplicationSource.class); // Queues of logs to process, entry in format of walGroupId->queue, @@ -84,8 +84,8 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf private Map> queues = new HashMap<>(); // per group queue size, keep no more than this number of logs in each wal group protected int queueSizePerGroup; - protected ReplicationQueues replicationQueues; - private ReplicationPeers replicationPeers; + protected ReplicationQueueStorage queueStorage; + private ReplicationPeer replicationPeer; protected Configuration conf; protected ReplicationQueueInfo replicationQueueInfo; @@ -106,17 +106,17 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf // total number of edits we replicated private AtomicLong totalReplicatedEdits = new AtomicLong(0); // The znode we currently play with - protected String peerClusterZnode; + protected String queueId; // Maximum number of retries before taking bold actions private int maxRetriesMultiplier; // Indicates if this particular source is running private volatile boolean sourceRunning = false; // Metrics for this source private MetricsSource metrics; - //WARN threshold for the number of queued logs, defaults to 2 + // WARN threshold for the number of queued logs, defaults to 2 private int logQueueWarnThreshold; // ReplicationEndpoint which will handle the actual replication - private ReplicationEndpoint replicationEndpoint; + private volatile ReplicationEndpoint replicationEndpoint; // A filter (or a chain of filters) for the WAL entries. protected WALEntryFilter walEntryFilter; // throttler @@ -134,24 +134,23 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf public static final int DEFAULT_WAIT_ON_ENDPOINT_SECONDS = 30; private int waitOnEndpointSeconds = -1; + private Thread initThread; + /** * Instantiation method used by region servers - * * @param conf configuration to use * @param fs file system to use * @param manager replication manager to ping to * @param server the server for this region server - * @param peerClusterZnode the name of our znode + * @param queueId the id of our replication queue * @param clusterId unique UUID for the cluster - * @param replicationEndpoint the replication endpoint implementation * @param metrics metrics for replication source - * @throws IOException */ @Override public void init(Configuration conf, FileSystem fs, ReplicationSourceManager manager, - ReplicationQueues replicationQueues, ReplicationPeers replicationPeers, Server server, - String peerClusterZnode, UUID clusterId, ReplicationEndpoint replicationEndpoint, - WALFileLengthProvider walFileLengthProvider, MetricsSource metrics) throws IOException { + ReplicationQueueStorage queueStorage, ReplicationPeer replicationPeer, Server server, + String queueId, UUID clusterId, WALFileLengthProvider walFileLengthProvider, + MetricsSource metrics) throws IOException { this.server = server; this.conf = HBaseConfiguration.create(conf); this.waitOnEndpointSeconds = @@ -162,26 +161,25 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf this.maxRetriesMultiplier = this.conf.getInt("replication.source.maxretriesmultiplier", 300); // 5 minutes @ 1 sec per this.queueSizePerGroup = this.conf.getInt("hbase.regionserver.maxlogs", 32); - this.replicationQueues = replicationQueues; - this.replicationPeers = replicationPeers; + this.queueStorage = queueStorage; + this.replicationPeer = replicationPeer; this.manager = manager; this.fs = fs; this.metrics = metrics; this.clusterId = clusterId; - this.peerClusterZnode = peerClusterZnode; - this.replicationQueueInfo = new ReplicationQueueInfo(peerClusterZnode); + this.queueId = queueId; + this.replicationQueueInfo = new ReplicationQueueInfo(queueId); // ReplicationQueueInfo parses the peerId out of the znode for us this.peerId = this.replicationQueueInfo.getPeerId(); this.logQueueWarnThreshold = this.conf.getInt("replication.source.log.queue.warn", 2); - this.replicationEndpoint = replicationEndpoint; defaultBandwidth = this.conf.getLong("replication.source.per.peer.node.bandwidth", 0); currentBandwidth = getCurrentBandwidth(); this.throttler = new ReplicationThrottler((double) currentBandwidth / 10.0); this.totalBufferUsed = manager.getTotalBufferUsed(); this.walFileLengthProvider = walFileLengthProvider; - LOG.info("peerClusterZnode=" + peerClusterZnode + ", ReplicationSource : " + peerId + LOG.info("queueId=" + queueId + ", ReplicationSource : " + peerId + ", currentBandwidth=" + this.currentBandwidth); } @@ -199,7 +197,7 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf if (queue == null) { queue = new PriorityBlockingQueue<>(queueSizePerGroup, new LogsComparator()); queues.put(logPrefix, queue); - if (this.sourceRunning) { + if (this.isSourceActive() && this.replicationEndpoint != null) { // new wal group observed after source startup, start a new worker thread to track it // notice: it's possible that log enqueued when this.running is set but worker thread // still not launched, so it's necessary to check workerThreads before start the worker @@ -219,18 +217,12 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf @Override public void addHFileRefs(TableName tableName, byte[] family, List> pairs) throws ReplicationException { - String peerId = peerClusterZnode; - if (peerId.contains("-")) { - // peerClusterZnode will be in the form peerId + "-" + rsZNode. - // A peerId will not have "-" in its name, see HBASE-11394 - peerId = peerClusterZnode.split("-")[0]; - } - Map> tableCFMap = replicationPeers.getConnectedPeer(peerId).getTableCFs(); + Map> tableCFMap = replicationPeer.getTableCFs(); if (tableCFMap != null) { List tableCfs = tableCFMap.get(tableName); if (tableCFMap.containsKey(tableName) && (tableCfs == null || tableCfs.contains(Bytes.toString(family)))) { - this.replicationQueues.addHFileRefs(peerId, pairs); + this.queueStorage.addHFileRefs(peerId, pairs); metrics.incrSizeOfHFileRefsQueue(pairs.size()); } else { LOG.debug("HFiles will not be replicated belonging to the table " + tableName + " family " @@ -239,60 +231,52 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf } else { // user has explicitly not defined any table cfs for replication, means replicate all the // data - this.replicationQueues.addHFileRefs(peerId, pairs); + this.queueStorage.addHFileRefs(peerId, pairs); metrics.incrSizeOfHFileRefsQueue(pairs.size()); } } - @Override - public void run() { - // mark we are running now - this.sourceRunning = true; - try { - // start the endpoint, connect to the cluster - this.replicationEndpoint.start(); - this.replicationEndpoint.awaitRunning(this.waitOnEndpointSeconds, TimeUnit.SECONDS); - } catch (Exception ex) { - LOG.warn("Error starting ReplicationEndpoint, exiting", ex); - uninitialize(); - throw new RuntimeException(ex); + private ReplicationEndpoint createReplicationEndpoint() + throws InstantiationException, IllegalAccessException, ClassNotFoundException, IOException { + RegionServerCoprocessorHost rsServerHost = null; + if (server instanceof HRegionServer) { + rsServerHost = ((HRegionServer) server).getRegionServerCoprocessorHost(); } - - int sleepMultiplier = 1; - // delay this until we are in an asynchronous thread - while (this.isSourceActive() && this.peerClusterId == null) { - this.peerClusterId = replicationEndpoint.getPeerUUID(); - if (this.isSourceActive() && this.peerClusterId == null) { - if (sleepForRetries("Cannot contact the peer's zk ensemble", sleepMultiplier)) { - sleepMultiplier++; - } + String replicationEndpointImpl = replicationPeer.getPeerConfig().getReplicationEndpointImpl(); + if (replicationEndpointImpl == null) { + // Default to HBase inter-cluster replication endpoint + replicationEndpointImpl = HBaseInterClusterReplicationEndpoint.class.getName(); + } + ReplicationEndpoint replicationEndpoint = + Class.forName(replicationEndpointImpl).asSubclass(ReplicationEndpoint.class).newInstance(); + 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; } } + return replicationEndpoint; + } - // In rare case, zookeeper setting may be messed up. That leads to the incorrect - // peerClusterId value, which is the same as the source clusterId - if (clusterId.equals(peerClusterId) && !replicationEndpoint.canReplicateToSameCluster()) { - this.terminate("ClusterId " + clusterId + " is replicating to itself: peerClusterId " - + peerClusterId + " which is not allowed by ReplicationEndpoint:" - + replicationEndpoint.getClass().getName(), null, false); - this.manager.closeQueue(this); - return; - } - LOG.info("Replicating " + clusterId + " -> " + peerClusterId); - - initializeWALEntryFilter(); - // start workers - for (Map.Entry> entry : queues.entrySet()) { - String walGroupId = entry.getKey(); - PriorityBlockingQueue queue = entry.getValue(); - tryStartNewShipper(walGroupId, queue); + private void initAndStartReplicationEndpoint(ReplicationEndpoint replicationEndpoint) + throws IOException, TimeoutException { + TableDescriptors tableDescriptors = null; + if (server instanceof HRegionServer) { + tableDescriptors = ((HRegionServer) server).getTableDescriptors(); } + replicationEndpoint + .init(new ReplicationEndpoint.Context(conf, replicationPeer.getConfiguration(), fs, peerId, + clusterId, replicationPeer, metrics, tableDescriptors, server)); + replicationEndpoint.start(); + replicationEndpoint.awaitRunning(waitOnEndpointSeconds, TimeUnit.SECONDS); } private void initializeWALEntryFilter() { // get the WALEntryFilter from ReplicationEndpoint and add it to default filters - ArrayList filters = Lists.newArrayList( - (WALEntryFilter)new SystemTableWALEntryFilter()); + ArrayList filters = + Lists. newArrayList(new SystemTableWALEntryFilter()); WALEntryFilter filterFromEndpoint = this.replicationEndpoint.getWALEntryfilter(); if (filterFromEndpoint != null) { filters.add(filterFromEndpoint); @@ -302,38 +286,31 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf } protected void tryStartNewShipper(String walGroupId, PriorityBlockingQueue queue) { - final ReplicationSourceShipper worker = new ReplicationSourceShipper(conf, - walGroupId, queue, this); + ReplicationSourceShipper worker = new ReplicationSourceShipper(conf, walGroupId, queue, this); ReplicationSourceShipper extant = workerThreads.putIfAbsent(walGroupId, worker); if (extant != null) { LOG.debug("Someone has beat us to start a worker thread for wal group " + walGroupId); } else { LOG.debug("Starting up worker for wal group " + walGroupId); - worker.startup(getUncaughtExceptionHandler()); - worker.setWALReader(startNewWALReader(worker.getName(), walGroupId, queue, - worker.getStartPosition())); - workerThreads.put(walGroupId, worker); + worker.startup(this::uncaughtException); + worker.setWALReader( + startNewWALReader(worker.getName(), walGroupId, queue, worker.getStartPosition())); } } protected ReplicationSourceWALReader startNewWALReader(String threadName, String walGroupId, PriorityBlockingQueue queue, long startPosition) { ReplicationSourceWALReader walReader = - new ReplicationSourceWALReader(fs, conf, queue, startPosition, walEntryFilter, this); + new ReplicationSourceWALReader(fs, conf, queue, startPosition, walEntryFilter, this); return (ReplicationSourceWALReader) Threads.setDaemonThreadRunning(walReader, - threadName + ".replicationSource.wal-reader." + walGroupId + "," + peerClusterZnode, - getUncaughtExceptionHandler()); + threadName + ".replicationSource.wal-reader." + walGroupId + "," + queueId, + this::uncaughtException); } - public Thread.UncaughtExceptionHandler getUncaughtExceptionHandler() { - return new Thread.UncaughtExceptionHandler() { - @Override - public void uncaughtException(final Thread t, final Throwable e) { - RSRpcServices.exitIfOOME(e); - LOG.error("Unexpected exception in " + t.getName() + " currentPath=" + getCurrentPath(), e); - server.stop("Unexpected exception in " + t.getName()); - } - }; + protected final void uncaughtException(Thread t, Throwable e) { + RSRpcServices.exitIfOOME(e); + LOG.error("Unexpected exception in " + t.getName() + " currentPath=" + getCurrentPath(), e); + server.abort("Unexpected exception in " + t.getName(), e); } @Override @@ -373,25 +350,11 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf } private long getCurrentBandwidth() { - ReplicationPeer replicationPeer = this.replicationPeers.getConnectedPeer(peerId); - long peerBandwidth = replicationPeer != null ? replicationPeer.getPeerBandwidth() : 0; + long peerBandwidth = replicationPeer.getPeerBandwidth(); // user can set peer bandwidth to 0 to use default bandwidth return peerBandwidth != 0 ? peerBandwidth : defaultBandwidth; } - private void uninitialize() { - LOG.debug("Source exiting " + this.peerId); - metrics.clear(); - if (this.replicationEndpoint.isRunning() || this.replicationEndpoint.isStarting()) { - this.replicationEndpoint.stop(); - try { - this.replicationEndpoint.awaitTerminated(this.waitOnEndpointSeconds, TimeUnit.SECONDS); - } catch (TimeoutException e) { - LOG.warn("Failed termination after " + this.waitOnEndpointSeconds + " seconds."); - } - } - } - /** * Do the sleeping logic * @param msg Why we sleep @@ -413,25 +376,83 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf /** * check whether the peer is enabled or not - * * @return true if the peer is enabled, otherwise false */ @Override public boolean isPeerEnabled() { - return this.replicationPeers.getStatusOfPeer(this.peerId); + return replicationPeer.isPeerEnabled(); + } + + private void initialize() { + int sleepMultiplier = 1; + while (this.isSourceActive()) { + ReplicationEndpoint replicationEndpoint; + try { + replicationEndpoint = createReplicationEndpoint(); + } catch (Exception e) { + LOG.warn("error creating ReplicationEndpoint, retry", e); + if (sleepForRetries("Error creating ReplicationEndpoint", sleepMultiplier)) { + sleepMultiplier++; + } + continue; + } + + try { + initAndStartReplicationEndpoint(replicationEndpoint); + this.replicationEndpoint = replicationEndpoint; + break; + } catch (Exception e) { + LOG.warn("Error starting ReplicationEndpoint, retry", e); + replicationEndpoint.stop(); + if (sleepForRetries("Error starting ReplicationEndpoint", sleepMultiplier)) { + sleepMultiplier++; + } + } + } + + if (!this.isSourceActive()) { + return; + } + + sleepMultiplier = 1; + // delay this until we are in an asynchronous thread + while (this.isSourceActive() && this.peerClusterId == null) { + this.peerClusterId = replicationEndpoint.getPeerUUID(); + if (this.isSourceActive() && this.peerClusterId == null) { + if (sleepForRetries("Cannot contact the peer's zk ensemble", sleepMultiplier)) { + sleepMultiplier++; + } + } + } + + // In rare case, zookeeper setting may be messed up. That leads to the incorrect + // peerClusterId value, which is the same as the source clusterId + if (clusterId.equals(peerClusterId) && !replicationEndpoint.canReplicateToSameCluster()) { + this.terminate("ClusterId " + clusterId + " is replicating to itself: peerClusterId " + + peerClusterId + " which is not allowed by ReplicationEndpoint:" + + replicationEndpoint.getClass().getName(), null, false); + this.manager.removeSource(this); + return; + } + LOG.info("Replicating " + clusterId + " -> " + peerClusterId); + + initializeWALEntryFilter(); + // start workers + for (Map.Entry> entry : queues.entrySet()) { + String walGroupId = entry.getKey(); + PriorityBlockingQueue queue = entry.getValue(); + tryStartNewShipper(walGroupId, queue); + } } @Override public void startup() { - String n = Thread.currentThread().getName(); - Thread.UncaughtExceptionHandler handler = new Thread.UncaughtExceptionHandler() { - @Override - public void uncaughtException(final Thread t, final Throwable e) { - LOG.error("Unexpected exception in ReplicationSource", e); - } - }; - Threads - .setDaemonThreadRunning(this, n + ".replicationSource," + this.peerClusterZnode, handler); + // mark we are running now + this.sourceRunning = true; + initThread = new Thread(this::initialize); + Threads.setDaemonThreadRunning(initThread, + Thread.currentThread().getName() + ".replicationSource," + this.queueId, + this::uncaughtException); } @Override @@ -446,14 +467,19 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf public void terminate(String reason, Exception cause, boolean join) { if (cause == null) { - LOG.info("Closing source " - + this.peerClusterZnode + " because: " + reason); - + LOG.info("Closing source " + this.queueId + " because: " + reason); } else { - LOG.error("Closing source " + this.peerClusterZnode - + " because an error occurred: " + reason, cause); + LOG.error("Closing source " + this.queueId + " because an error occurred: " + reason, + cause); } this.sourceRunning = false; + if (initThread != null && Thread.currentThread() != initThread) { + // This usually won't happen but anyway, let's wait until the initialization thread exits. + // And notice that we may call terminate directly from the initThread so here we need to + // avoid join on ourselves. + initThread.interrupt(); + Threads.shutdown(initThread, this.sleepForRetries); + } Collection workers = workerThreads.values(); for (ReplicationSourceShipper worker : workers) { worker.stopWorker(); @@ -470,19 +496,19 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf } if (this.replicationEndpoint != null) { try { - this.replicationEndpoint.awaitTerminated(sleepForRetries * maxRetriesMultiplier, TimeUnit.MILLISECONDS); + this.replicationEndpoint.awaitTerminated(sleepForRetries * maxRetriesMultiplier, + TimeUnit.MILLISECONDS); } catch (TimeoutException te) { - LOG.warn("Got exception while waiting for endpoint to shutdown for replication source :" - + this.peerClusterZnode, - te); + LOG.warn("Got exception while waiting for endpoint to shutdown for replication source :" + + this.queueId, te); } } } } @Override - public String getPeerClusterZnode() { - return this.peerClusterZnode; + public String getQueueId() { + return this.queueId; } @Override diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceFactory.java index 865a202..93e8331 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceFactory.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceFactory.java @@ -32,8 +32,8 @@ public class ReplicationSourceFactory { private static final Logger LOG = LoggerFactory.getLogger(ReplicationSourceFactory.class); - static ReplicationSourceInterface create(Configuration conf, String peerId) { - ReplicationQueueInfo replicationQueueInfo = new ReplicationQueueInfo(peerId); + static ReplicationSourceInterface create(Configuration conf, String queueId) { + ReplicationQueueInfo replicationQueueInfo = new ReplicationQueueInfo(queueId); boolean isQueueRecovered = replicationQueueInfo.isQueueRecovered(); ReplicationSourceInterface src; try { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java index b6cf54d..d7cf9a3 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java @@ -22,7 +22,6 @@ import java.io.IOException; import java.util.List; import java.util.UUID; -import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -31,10 +30,11 @@ import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.replication.ReplicationEndpoint; import org.apache.hadoop.hbase.replication.ReplicationException; -import org.apache.hadoop.hbase.replication.ReplicationPeers; -import org.apache.hadoop.hbase.replication.ReplicationQueues; +import org.apache.hadoop.hbase.replication.ReplicationPeer; +import org.apache.hadoop.hbase.replication.ReplicationQueueStorage; import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.wal.WAL.Entry; +import org.apache.yetus.audience.InterfaceAudience; /** * Interface that defines a replication source @@ -47,17 +47,12 @@ public interface ReplicationSourceInterface { * @param conf the configuration to use * @param fs the file system to use * @param manager the manager to use - * @param replicationQueues - * @param replicationPeers * @param server the server for this region server - * @param peerClusterZnode - * @param clusterId - * @throws IOException */ void init(Configuration conf, FileSystem fs, ReplicationSourceManager manager, - ReplicationQueues replicationQueues, ReplicationPeers replicationPeers, Server server, - String peerClusterZnode, UUID clusterId, ReplicationEndpoint replicationEndpoint, - WALFileLengthProvider walFileLengthProvider, MetricsSource metrics) throws IOException; + ReplicationQueueStorage queueStorage, ReplicationPeer replicationPeer, Server server, + String queueId, UUID clusterId, WALFileLengthProvider walFileLengthProvider, + MetricsSource metrics) throws IOException; /** * Add a log to the list of logs to replicate @@ -101,11 +96,11 @@ public interface ReplicationSourceInterface { Path getCurrentPath(); /** - * Get the id that the source is replicating to + * Get the queue id that the source is replicating to * - * @return peer cluster id + * @return queue id */ - String getPeerClusterZnode(); + String getQueueId(); /** * Get the id that the source is replicating to. 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 23ae704..e087127 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 @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -16,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.replication.regionserver; import java.io.IOException; @@ -33,34 +31,31 @@ import java.util.SortedSet; import java.util.TreeSet; import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.Future; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; - +import java.util.stream.Collectors; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.Server; -import org.apache.hadoop.hbase.TableDescriptors; +import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; -import org.apache.hadoop.hbase.regionserver.HRegionServer; -import org.apache.hadoop.hbase.regionserver.RegionServerCoprocessorHost; -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.ReplicationQueues; +import org.apache.hadoop.hbase.replication.ReplicationQueueStorage; import org.apache.hadoop.hbase.replication.ReplicationTracker; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Pair; @@ -68,45 +63,77 @@ import org.apache.hadoop.hbase.wal.AbstractFSWALProvider; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; /** - * This class is responsible to manage all the replication - * sources. There are two classes of sources: + * This class is responsible to manage all the replication sources. There are two classes of + * sources: *
      - *
    • Normal sources are persistent and one per peer cluster
    • - *
    • Old sources are recovered from a failed region server and our - * only goal is to finish replicating the WAL queue it had up in ZK
    • + *
    • Normal sources are persistent and one per peer cluster
    • + *
    • Old sources are recovered from a failed region server and our only goal is to finish + * replicating the WAL queue it had
    • + *
    + *

    + * When a region server dies, this class uses a watcher to get notified and it tries to grab a lock + * in order to transfer all the queues in a local old source. + *

    + * Synchronization specification: + *

      + *
    • No need synchronized on {@link #sources}. {@link #sources} is a ConcurrentHashMap and there + * is a Lock for peer id in {@link PeerProcedureHandlerImpl}. So there is no race for peer + * operations.
    • + *
    • Need synchronized on {@link #walsById}. There are four methods which modify it, + * {@link #addPeer(String)}, {@link #removePeer(String)}, + * {@link #cleanOldLogs(SortedSet, String, String)} and {@link #preLogRoll(Path)}. {@link #walsById} + * is a ConcurrentHashMap and there is a Lock for peer id in {@link PeerProcedureHandlerImpl}. So + * there is no race between {@link #addPeer(String)} and {@link #removePeer(String)}. + * {@link #cleanOldLogs(SortedSet, String, String)} is called by {@link ReplicationSourceInterface}. + * So no race with {@link #addPeer(String)}. {@link #removePeer(String)} will terminate the + * {@link ReplicationSourceInterface} firstly, then remove the wals from {@link #walsById}. So no + * race with {@link #removePeer(String)}. The only case need synchronized is + * {@link #cleanOldLogs(SortedSet, String, String)} and {@link #preLogRoll(Path)}.
    • + *
    • No need synchronized on {@link #walsByIdRecoveredQueues}. There are three methods which + * modify it, {@link #removePeer(String)} , {@link #cleanOldLogs(SortedSet, String, String)} and + * {@link ReplicationSourceManager.NodeFailoverWorker#run()}. + * {@link #cleanOldLogs(SortedSet, String, String)} is called by {@link ReplicationSourceInterface}. + * {@link #removePeer(String)} will terminate the {@link ReplicationSourceInterface} firstly, then + * remove the wals from {@link #walsByIdRecoveredQueues}. And + * {@link ReplicationSourceManager.NodeFailoverWorker#run()} will add the wals to + * {@link #walsByIdRecoveredQueues} firstly, then start up a {@link ReplicationSourceInterface}. So + * there is no race here. For {@link ReplicationSourceManager.NodeFailoverWorker#run()} and + * {@link #removePeer(String)}, there is already synchronized on {@link #oldsources}. So no need + * synchronized on {@link #walsByIdRecoveredQueues}.
    • + *
    • Need synchronized on {@link #latestPaths} to avoid the new open source miss new log.
    • + *
    • Need synchronized on {@link #oldsources} to avoid adding recovered source for the + * to-be-removed peer.
    • *
    - * - * When a region server dies, this class uses a watcher to get notified and it - * tries to grab a lock in order to transfer all the queues in a local - * old source. - * - * This class implements the ReplicationListener interface so that it can track changes in - * replication state. */ @InterfaceAudience.Private public class ReplicationSourceManager implements ReplicationListener { - private static final Logger LOG = - LoggerFactory.getLogger(ReplicationSourceManager.class); - // List of all the sources that read this RS's logs - private final List sources; + private static final Logger LOG = LoggerFactory.getLogger(ReplicationSourceManager.class); + // all the sources that read this RS's logs and every peer only has one replication source + private final ConcurrentMap sources; // List of all the sources we got from died RSs private final List oldsources; - private final ReplicationQueues replicationQueues; + private final ReplicationQueueStorage queueStorage; private final ReplicationTracker replicationTracker; private final ReplicationPeers replicationPeers; // UUID for this cluster private final UUID clusterId; // All about stopping private final Server server; + // All logs we are currently tracking - // Index structure of the map is: peer_id->logPrefix/logGroup->logs - private final Map>> walsById; + // Index structure of the map is: queue_id->logPrefix/logGroup->logs + // For normal replication source, the peer id is same with the queue id + private final ConcurrentMap>> walsById; // Logs for recovered sources we are currently tracking - private final Map>> walsByIdRecoveredQueues; + // the map is: queue_id->logPrefix/logGroup->logs + // For recovered source, the queue id's format is peer_id-servername-* + private final ConcurrentMap>> walsByIdRecoveredQueues; + private final Configuration conf; private final FileSystem fs; // The paths to the latest log of each wal group, for new coming peers @@ -130,7 +157,7 @@ public class ReplicationSourceManager implements ReplicationListener { /** * Creates a replication manager and sets the watch on all the other registered region servers - * @param replicationQueues the interface for manipulating replication queues + * @param queueStorage the interface for manipulating replication queues * @param replicationPeers * @param replicationTracker * @param conf the configuration to use @@ -140,163 +167,197 @@ public class ReplicationSourceManager implements ReplicationListener { * @param oldLogDir the directory where old logs are archived * @param clusterId */ - public ReplicationSourceManager(ReplicationQueues replicationQueues, + public ReplicationSourceManager(ReplicationQueueStorage queueStorage, ReplicationPeers replicationPeers, ReplicationTracker replicationTracker, Configuration conf, Server server, FileSystem fs, Path logDir, Path oldLogDir, UUID clusterId, WALFileLengthProvider walFileLengthProvider) throws IOException { - //CopyOnWriteArrayList is thread-safe. - //Generally, reading is more than modifying. - this.sources = new CopyOnWriteArrayList<>(); - this.replicationQueues = replicationQueues; + // CopyOnWriteArrayList is thread-safe. + // Generally, reading is more than modifying. + this.sources = new ConcurrentHashMap<>(); + this.queueStorage = queueStorage; this.replicationPeers = replicationPeers; this.replicationTracker = replicationTracker; this.server = server; - this.walsById = new HashMap<>(); + this.walsById = new ConcurrentHashMap<>(); this.walsByIdRecoveredQueues = new ConcurrentHashMap<>(); - this.oldsources = new CopyOnWriteArrayList<>(); + this.oldsources = new ArrayList<>(); this.conf = conf; this.fs = fs; this.logDir = logDir; this.oldLogDir = oldLogDir; - this.sleepBeforeFailover = - conf.getLong("replication.sleep.before.failover", 30000); // 30 seconds + this.sleepBeforeFailover = conf.getLong("replication.sleep.before.failover", 30000); // 30 + // seconds this.clusterId = clusterId; this.walFileLengthProvider = walFileLengthProvider; this.replicationTracker.registerListener(this); - this.replicationPeers.getAllPeerIds(); // It's preferable to failover 1 RS at a time, but with good zk servers // more could be processed at the same time. int nbWorkers = conf.getInt("replication.executor.workers", 1); // use a short 100ms sleep since this could be done inline with a RS startup // even if we fail, other region servers can take care of it - this.executor = new ThreadPoolExecutor(nbWorkers, nbWorkers, - 100, TimeUnit.MILLISECONDS, new LinkedBlockingQueue<>()); + this.executor = new ThreadPoolExecutor(nbWorkers, nbWorkers, 100, TimeUnit.MILLISECONDS, + new LinkedBlockingQueue<>()); ThreadFactoryBuilder tfb = new ThreadFactoryBuilder(); tfb.setNameFormat("ReplicationExecutor-%d"); tfb.setDaemon(true); this.executor.setThreadFactory(tfb.build()); this.latestPaths = new HashSet(); - replicationForBulkLoadDataEnabled = - conf.getBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY, - HConstants.REPLICATION_BULKLOAD_ENABLE_DEFAULT); + replicationForBulkLoadDataEnabled = conf.getBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY, + HConstants.REPLICATION_BULKLOAD_ENABLE_DEFAULT); this.replicationWaitTime = conf.getLong(HConstants.REPLICATION_SERIALLY_WAITING_KEY, - HConstants.REPLICATION_SERIALLY_WAITING_DEFAULT); + HConstants.REPLICATION_SERIALLY_WAITING_DEFAULT); connection = ConnectionFactory.createConnection(conf); } /** - * Provide the id of the peer and a log key and this method will figure which - * wal it belongs to and will log, for this region server, the current - * position. It will also clean old logs from the queue. - * @param log Path to the log currently being replicated from - * replication status in zookeeper. It will also delete older entries. - * @param id id of the peer cluster - * @param position current location in the log - * @param queueRecovered indicates if this queue comes from another region server - * @param holdLogInZK if true then the log is retained in ZK + * Adds a normal source per registered peer cluster and tries to process all old region server wal + * queues + *

    + * The returned future is for adoptAbandonedQueues task. */ - public void logPositionAndCleanOldLogs(Path log, String id, long position, - boolean queueRecovered, boolean holdLogInZK) { - String fileName = log.getName(); - this.replicationQueues.setLogPosition(id, fileName, position); - if (holdLogInZK) { - return; + Future init() throws IOException { + for (String id : this.replicationPeers.getAllPeerIds()) { + addSource(id); + if (replicationForBulkLoadDataEnabled) { + // Check if peer exists in hfile-refs queue, if not add it. This can happen in the case + // when a peer was added before replication for bulk loaded data was enabled. + throwIOExceptionWhenFail(() -> this.queueStorage.addPeerToHFileRefs(id)); + } } - cleanOldLogs(fileName, id, queueRecovered); + return this.executor.submit(this::adoptAbandonedQueues); } - /** - * Cleans a log file and all older files from ZK. Called when we are sure that a - * log file is closed and has no more entries. - * @param key Path to the log - * @param id id of the peer cluster - * @param queueRecovered Whether this is a recovered queue - */ - public void cleanOldLogs(String key, String id, boolean queueRecovered) { - String logPrefix = AbstractFSWALProvider.getWALPrefixFromWALName(key); - if (queueRecovered) { - SortedSet wals = walsByIdRecoveredQueues.get(id).get(logPrefix); - if (wals != null && !wals.first().equals(key)) { - cleanOldLogs(wals, key, id); - } - } else { - synchronized (this.walsById) { - SortedSet wals = walsById.get(id).get(logPrefix); - if (wals != null && !wals.first().equals(key)) { - cleanOldLogs(wals, key, id); - } + private void adoptAbandonedQueues() { + List currentReplicators = null; + try { + currentReplicators = queueStorage.getListOfReplicators(); + } catch (ReplicationException e) { + server.abort("Failed to get all replicators", e); + return; + } + if (currentReplicators == null || currentReplicators.isEmpty()) { + return; + } + List otherRegionServers = replicationTracker.getListOfRegionServers().stream() + .map(ServerName::valueOf).collect(Collectors.toList()); + LOG.info( + "Current list of replicators: " + currentReplicators + " other RSs: " + otherRegionServers); + + // Look if there's anything to process after a restart + for (ServerName rs : currentReplicators) { + if (!otherRegionServers.contains(rs)) { + transferQueues(rs); } } - } + } - private void cleanOldLogs(SortedSet wals, String key, String id) { - SortedSet walSet = wals.headSet(key); - LOG.debug("Removing " + walSet.size() + " logs in the list: " + walSet); - for (String wal : walSet) { - this.replicationQueues.removeLog(id, wal); + /** + * 1. Add peer to replicationPeers 2. Add the normal source and related replication queue 3. Add + * HFile Refs + * @param peerId the id of replication peer + */ + public void addPeer(String peerId) throws IOException { + boolean added = false; + try { + added = this.replicationPeers.addPeer(peerId); + } catch (ReplicationException e) { + throw new IOException(e); + } + if (added) { + addSource(peerId); + if (replicationForBulkLoadDataEnabled) { + throwIOExceptionWhenFail(() -> this.queueStorage.addPeerToHFileRefs(peerId)); + } } - walSet.clear(); } /** - * Adds a normal source per registered peer cluster and tries to process all - * old region server wal queues + * 1. Remove peer for replicationPeers 2. Remove all the recovered sources for the specified id + * and related replication queues 3. Remove the normal source and related replication queue 4. + * Remove HFile Refs + * @param peerId the id of the replication peer */ - void init() throws IOException, ReplicationException { - for (String id : this.replicationPeers.getConnectedPeerIds()) { - addSource(id); - if (replicationForBulkLoadDataEnabled) { - // Check if peer exists in hfile-refs queue, if not add it. This can happen in the case - // when a peer was added before replication for bulk loaded data was enabled. - this.replicationQueues.addPeerToHFileRefs(id); + public void removePeer(String peerId) { + replicationPeers.removePeer(peerId); + String terminateMessage = "Replication stream was removed by a user"; + List oldSourcesToDelete = new ArrayList<>(); + // synchronized on oldsources to avoid adding recovered source for the to-be-removed peer + // see NodeFailoverWorker.run + synchronized (this.oldsources) { + // First close all the recovered sources for this peer + for (ReplicationSourceInterface src : oldsources) { + if (peerId.equals(src.getPeerId())) { + oldSourcesToDelete.add(src); + } + } + for (ReplicationSourceInterface src : oldSourcesToDelete) { + src.terminate(terminateMessage); + removeRecoveredSource(src); } } - AdoptAbandonedQueuesWorker adoptionWorker = new AdoptAbandonedQueuesWorker(); - try { - this.executor.execute(adoptionWorker); - } catch (RejectedExecutionException ex) { - LOG.info("Cancelling the adoption of abandoned queues because of " + ex.getMessage()); + LOG.info( + "Number of deleted recovered sources for " + peerId + ": " + oldSourcesToDelete.size()); + // Now close the normal source for this peer + ReplicationSourceInterface srcToRemove = this.sources.get(peerId); + if (srcToRemove != null) { + srcToRemove.terminate(terminateMessage); + removeSource(srcToRemove); + } else { + // This only happened in unit test TestReplicationSourceManager#testPeerRemovalCleanup + // Delete queue from storage and memory and queue id is same with peer id for normal + // source + deleteQueue(peerId); + this.walsById.remove(peerId); } + + // Remove HFile Refs + abortWhenFail(() -> this.queueStorage.removePeerFromHFileRefs(peerId)); + } + + /** + * Factory method to create a replication source + * @param queueId the id of the replication queue + * @return the created source + */ + private ReplicationSourceInterface createSource(String queueId, ReplicationPeer replicationPeer) + throws IOException { + ReplicationSourceInterface src = ReplicationSourceFactory.create(conf, queueId); + + MetricsSource metrics = new MetricsSource(queueId); + // init replication source + src.init(conf, fs, this, queueStorage, replicationPeer, server, queueId, clusterId, + walFileLengthProvider, metrics); + return src; } /** - * Add sources for the given peer cluster on this region server. For the newly added peer, we only - * need to enqueue the latest log of each wal group and do replication - * @param id the id of the peer cluster + * Add a normal source for the given peer on this region server. Meanwhile, add new replication + * queue to storage. For the newly added peer, we only need to enqueue the latest log of each wal + * group and do replication + * @param peerId the id of the replication peer * @return the source that was created - * @throws IOException */ @VisibleForTesting - ReplicationSourceInterface addSource(String id) throws IOException, ReplicationException { - ReplicationPeerConfig peerConfig = replicationPeers.getReplicationPeerConfig(id); - ReplicationPeer peer = replicationPeers.getConnectedPeer(id); - ReplicationSourceInterface src = getReplicationSource(this.conf, this.fs, this, - this.replicationQueues, this.replicationPeers, server, id, this.clusterId, peerConfig, peer, - walFileLengthProvider); - synchronized (this.walsById) { - this.sources.add(src); + ReplicationSourceInterface addSource(String peerId) throws IOException { + ReplicationPeer peer = replicationPeers.getPeer(peerId); + ReplicationSourceInterface src = createSource(peerId, peer); + // synchronized on latestPaths to avoid missing the new log + synchronized (this.latestPaths) { + this.sources.put(peerId, src); Map> walsByGroup = new HashMap<>(); - this.walsById.put(id, walsByGroup); + this.walsById.put(peerId, walsByGroup); // Add the latest wal to that source's queue - synchronized (latestPaths) { - if (this.latestPaths.size() > 0) { - for (Path logPath : latestPaths) { - String name = logPath.getName(); - String walPrefix = AbstractFSWALProvider.getWALPrefixFromWALName(name); - SortedSet logs = new TreeSet<>(); - logs.add(name); - walsByGroup.put(walPrefix, logs); - try { - this.replicationQueues.addLog(id, name); - } catch (ReplicationException e) { - String message = - "Cannot add log to queue when creating a new source, queueId=" + id - + ", filename=" + name; - server.stop(message); - throw e; - } - src.enqueueLog(logPath); - } + if (this.latestPaths.size() > 0) { + for (Path logPath : latestPaths) { + String name = logPath.getName(); + String walPrefix = AbstractFSWALProvider.getWALPrefixFromWALName(name); + SortedSet logs = new TreeSet<>(); + logs.add(name); + walsByGroup.put(walPrefix, logs); + // Abort RS and throw exception to make add peer failed + abortAndThrowIOExceptionWhenFail( + () -> this.queueStorage.addWAL(server.getServerName(), peerId, name)); + src.enqueueLog(logPath); } } } @@ -304,87 +365,217 @@ public class ReplicationSourceManager implements ReplicationListener { return src; } - @VisibleForTesting - int getSizeOfLatestPath() { - synchronized (latestPaths) { - return latestPaths.size(); - } - } - /** - * Delete a complete queue of wals associated with a peer cluster - * @param peerId Id of the peer cluster queue of wals to delete + * Close the previous replication sources of this peer id and open new sources to trigger the new + * replication state changes or new replication config changes. Here we don't need to change + * replication queue storage and only to enqueue all logs to the new replication source + * @param peerId the id of the replication peer + * @throws IOException */ - public void deleteSource(String peerId, boolean closeConnection) { - this.replicationQueues.removeQueue(peerId); - if (closeConnection) { - this.replicationPeers.peerDisconnected(peerId); + public void refreshSources(String peerId) throws IOException { + String terminateMessage = "Peer " + peerId + + " state or config changed. Will close the previous replication source and open a new one"; + ReplicationPeer peer = replicationPeers.getPeer(peerId); + ReplicationSourceInterface src = createSource(peerId, peer); + // synchronized on latestPaths to avoid missing the new log + synchronized (this.latestPaths) { + ReplicationSourceInterface toRemove = this.sources.put(peerId, src); + if (toRemove != null) { + LOG.info("Terminate replication source for " + toRemove.getPeerId()); + toRemove.terminate(terminateMessage); + } + for (SortedSet walsByGroup : walsById.get(peerId).values()) { + walsByGroup.forEach(wal -> src.enqueueLog(new Path(this.logDir, wal))); + } } - } + LOG.info("Startup replication source for " + src.getPeerId()); + src.startup(); - /** - * Terminate the replication on this region server - */ - public void join() { - this.executor.shutdown(); - for (ReplicationSourceInterface source : this.sources) { - source.terminate("Region server is closing"); + List toStartup = new ArrayList<>(); + // synchronized on oldsources to avoid race with NodeFailoverWorker + synchronized (this.oldsources) { + List previousQueueIds = new ArrayList<>(); + for (ReplicationSourceInterface oldSource : this.oldsources) { + if (oldSource.getPeerId().equals(peerId)) { + previousQueueIds.add(oldSource.getQueueId()); + oldSource.terminate(terminateMessage); + this.oldsources.remove(oldSource); + } + } + for (String queueId : previousQueueIds) { + ReplicationSourceInterface replicationSource = createSource(queueId, peer); + this.oldsources.add(replicationSource); + for (SortedSet walsByGroup : walsByIdRecoveredQueues.get(queueId).values()) { + walsByGroup.forEach(wal -> src.enqueueLog(new Path(wal))); + } + toStartup.add(replicationSource); + } + } + for (ReplicationSourceInterface replicationSource : oldsources) { + replicationSource.startup(); } } /** - * Get a copy of the wals of the first source on this rs - * @return a sorted set of wal names + * Clear the metrics and related replication queue of the specified old source + * @param src source to clear */ - @VisibleForTesting - Map>> getWALs() { - return Collections.unmodifiableMap(walsById); + void removeRecoveredSource(ReplicationSourceInterface src) { + LOG.info("Done with the recovered queue " + src.getQueueId()); + src.getSourceMetrics().clear(); + this.oldsources.remove(src); + // Delete queue from storage and memory + deleteQueue(src.getQueueId()); + this.walsByIdRecoveredQueues.remove(src.getQueueId()); } /** - * Get a copy of the wals of the recovered sources on this rs - * @return a sorted set of wal names + * Clear the metrics and related replication queue of the specified old source + * @param src source to clear */ - @VisibleForTesting - Map>> getWalsByIdRecoveredQueues() { - return Collections.unmodifiableMap(walsByIdRecoveredQueues); + void removeSource(ReplicationSourceInterface src) { + LOG.info("Done with the queue " + src.getQueueId()); + src.getSourceMetrics().clear(); + this.sources.remove(src.getPeerId()); + // Delete queue from storage and memory + deleteQueue(src.getQueueId()); + this.walsById.remove(src.getQueueId()); } /** - * Get a list of all the normal sources of this rs - * @return lis of all sources + * Delete a complete queue of wals associated with a replication source + * @param queueId the id of replication queue to delete */ - public List getSources() { - return this.sources; + private void deleteQueue(String queueId) { + abortWhenFail(() -> this.queueStorage.removeQueue(server.getServerName(), queueId)); + } + + @FunctionalInterface + private interface ReplicationQueueOperation { + void exec() throws ReplicationException; + } + + private void abortWhenFail(ReplicationQueueOperation op) { + try { + op.exec(); + } catch (ReplicationException e) { + server.abort("Failed to operate on replication queue", e); + } + } + + private void throwIOExceptionWhenFail(ReplicationQueueOperation op) throws IOException { + try { + op.exec(); + } catch (ReplicationException e) { + throw new IOException(e); + } + } + + private void abortAndThrowIOExceptionWhenFail(ReplicationQueueOperation op) throws IOException { + try { + op.exec(); + } catch (ReplicationException e) { + server.abort("Failed to operate on replication queue", e); + throw new IOException(e); + } } /** - * Get a list of all the old sources of this rs - * @return list of all old sources + * This method will log the current position to storage. And also clean old logs from the + * replication queue. + * @param log Path to the log currently being replicated + * @param queueId id of the replication queue + * @param position current location in the log + * @param queueRecovered indicates if this queue comes from another region server */ - public List getOldSources() { - return this.oldsources; + public void logPositionAndCleanOldLogs(Path log, String queueId, long position, + boolean queueRecovered) { + String fileName = log.getName(); + abortWhenFail( + () -> this.queueStorage.setWALPosition(server.getServerName(), queueId, fileName, position)); + cleanOldLogs(fileName, queueId, queueRecovered); } /** - * Get the normal source for a given peer - * @param peerId - * @return the normal source for the give peer if it exists, otherwise null. + * Cleans a log file and all older logs from replication queue. Called when we are sure that a log + * file is closed and has no more entries. + * @param log Path to the log + * @param queueId id of the replication queue + * @param queueRecovered Whether this is a recovered queue */ - public ReplicationSourceInterface getSource(String peerId) { - return getSources().stream().filter(s -> s.getPeerId().equals(peerId)).findFirst().orElse(null); + @VisibleForTesting + void cleanOldLogs(String log, String queueId, boolean queueRecovered) { + String logPrefix = AbstractFSWALProvider.getWALPrefixFromWALName(log); + if (queueRecovered) { + SortedSet wals = walsByIdRecoveredQueues.get(queueId).get(logPrefix); + if (wals != null && !wals.first().equals(log)) { + cleanOldLogs(wals, log, queueId); + } + } else { + // synchronized on walsById to avoid race with preLogRoll + synchronized (this.walsById) { + SortedSet wals = walsById.get(queueId).get(logPrefix); + if (wals != null && !wals.first().equals(log)) { + cleanOldLogs(wals, log, queueId); + } + } + } } - @VisibleForTesting - List getAllQueues() { - return replicationQueues.getAllQueues(); + private void cleanOldLogs(SortedSet wals, String key, String id) { + SortedSet walSet = wals.headSet(key); + if (LOG.isDebugEnabled()) { + LOG.debug("Removing " + walSet.size() + " logs in the list: " + walSet); + } + for (String wal : walSet) { + abortWhenFail(() -> this.queueStorage.removeWAL(server.getServerName(), id, wal)); + } + walSet.clear(); } void preLogRoll(Path newLog) throws IOException { - recordLog(newLog); String logName = newLog.getName(); String logPrefix = AbstractFSWALProvider.getWALPrefixFromWALName(logName); - synchronized (latestPaths) { + // synchronized on latestPaths to avoid the new open source miss the new log + synchronized (this.latestPaths) { + // Add log to queue storage + for (ReplicationSourceInterface source : this.sources.values()) { + // If record log to queue storage failed, abort RS and throw exception to make log roll + // failed + abortAndThrowIOExceptionWhenFail( + () -> this.queueStorage.addWAL(server.getServerName(), source.getQueueId(), logName)); + } + + // synchronized on walsById to avoid race with cleanOldLogs + synchronized (this.walsById) { + // Update walsById map + for (Map.Entry>> entry : this.walsById.entrySet()) { + String peerId = entry.getKey(); + Map> walsByPrefix = entry.getValue(); + boolean existingPrefix = false; + for (Map.Entry> walsEntry : walsByPrefix.entrySet()) { + SortedSet wals = walsEntry.getValue(); + if (this.sources.isEmpty()) { + // If there's no slaves, don't need to keep the old wals since + // we only consider the last one when a new slave comes in + wals.clear(); + } + if (logPrefix.equals(walsEntry.getKey())) { + wals.add(logName); + existingPrefix = true; + } + } + if (!existingPrefix) { + // The new log belongs to a new group, add it into this peer + LOG.debug("Start tracking logs for wal group " + logPrefix + " for peer " + peerId); + SortedSet wals = new TreeSet<>(); + wals.add(logName); + walsByPrefix.put(logPrefix, wals); + } + } + } + + // Add to latestPaths Iterator iterator = latestPaths.iterator(); while (iterator.hasNext()) { Path path = iterator.next(); @@ -397,287 +588,58 @@ public class ReplicationSourceManager implements ReplicationListener { } } - /** - * Check and enqueue the given log to the correct source. If there's still no source for the - * group to which the given log belongs, create one - * @param logPath the log path to check and enqueue - * @throws IOException - */ - private void recordLog(Path logPath) throws IOException { - String logName = logPath.getName(); - String logPrefix = AbstractFSWALProvider.getWALPrefixFromWALName(logName); - // update replication queues on ZK - // synchronize on replicationPeers to avoid adding source for the to-be-removed peer - synchronized (replicationPeers) { - for (String id : replicationPeers.getConnectedPeerIds()) { - try { - this.replicationQueues.addLog(id, logName); - } catch (ReplicationException e) { - throw new IOException("Cannot add log to replication queue" - + " when creating a new source, queueId=" + id + ", filename=" + logName, e); - } - } - } - // update walsById map - synchronized (walsById) { - for (Map.Entry>> entry : this.walsById.entrySet()) { - String peerId = entry.getKey(); - Map> walsByPrefix = entry.getValue(); - boolean existingPrefix = false; - for (Map.Entry> walsEntry : walsByPrefix.entrySet()) { - SortedSet wals = walsEntry.getValue(); - if (this.sources.isEmpty()) { - // If there's no slaves, don't need to keep the old wals since - // we only consider the last one when a new slave comes in - wals.clear(); - } - if (logPrefix.equals(walsEntry.getKey())) { - wals.add(logName); - existingPrefix = true; - } - } - if (!existingPrefix) { - // The new log belongs to a new group, add it into this peer - LOG.debug("Start tracking logs for wal group " + logPrefix + " for peer " + peerId); - SortedSet wals = new TreeSet<>(); - wals.add(logName); - walsByPrefix.put(logPrefix, wals); - } - } - } - } - void postLogRoll(Path newLog) throws IOException { // This only updates the sources we own, not the recovered ones - for (ReplicationSourceInterface source : this.sources) { + for (ReplicationSourceInterface source : this.sources.values()) { source.enqueueLog(newLog); } } - @VisibleForTesting - public AtomicLong getTotalBufferUsed() { - return totalBufferUsed; - } - - /** - * Factory method to create a replication source - * @param conf the configuration to use - * @param fs the file system to use - * @param manager the manager to use - * @param server the server object for this region server - * @param peerId the id of the peer cluster - * @return the created source - * @throws IOException - */ - private ReplicationSourceInterface getReplicationSource(Configuration conf, FileSystem fs, - ReplicationSourceManager manager, ReplicationQueues replicationQueues, - ReplicationPeers replicationPeers, Server server, String peerId, UUID clusterId, - ReplicationPeerConfig peerConfig, ReplicationPeer replicationPeer, - WALFileLengthProvider walFileLengthProvider) throws IOException { - RegionServerCoprocessorHost rsServerHost = null; - TableDescriptors tableDescriptors = null; - if (server instanceof HRegionServer) { - rsServerHost = ((HRegionServer) server).getRegionServerCoprocessorHost(); - tableDescriptors = ((HRegionServer) server).getTableDescriptors(); - } - - ReplicationSourceInterface src = ReplicationSourceFactory.create(conf, peerId); - - ReplicationEndpoint replicationEndpoint = null; - try { - String replicationEndpointImpl = peerConfig.getReplicationEndpointImpl(); - if (replicationEndpointImpl == null) { - // Default to HBase inter-cluster replication endpoint - replicationEndpointImpl = HBaseInterClusterReplicationEndpoint.class.getName(); - } - @SuppressWarnings("rawtypes") - Class c = Class.forName(replicationEndpointImpl); - replicationEndpoint = (ReplicationEndpoint) c.newInstance(); - 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); - throw new IOException(e); - } - - MetricsSource metrics = new MetricsSource(peerId); - // init replication source - src.init(conf, fs, manager, replicationQueues, replicationPeers, server, peerId, clusterId, - replicationEndpoint, walFileLengthProvider, metrics); - - // init replication endpoint - replicationEndpoint.init(new ReplicationEndpoint.Context(conf, replicationPeer.getConfiguration(), - fs, peerId, clusterId, replicationPeer, metrics, tableDescriptors, server)); - - return src; + @Override + public void regionServerRemoved(String regionserver) { + transferQueues(ServerName.valueOf(regionserver)); } /** - * Transfer all the queues of the specified to this region server. - * First it tries to grab a lock and if it works it will move the - * znodes and finally will delete the old znodes. - * + * Transfer all the queues of the specified to this region server. First it tries to grab a lock + * and if it works it will move the old queues and finally will delete the old queues. + *

    * It creates one old source for any type of source of the old rs. - * @param rsZnode */ - private void transferQueues(String rsZnode) { - NodeFailoverWorker transfer = - new NodeFailoverWorker(rsZnode, this.replicationQueues, this.replicationPeers, - this.clusterId); + private void transferQueues(ServerName deadRS) { + if (server.getServerName().equals(deadRS)) { + // it's just us, give up + return; + } + NodeFailoverWorker transfer = new NodeFailoverWorker(deadRS); try { this.executor.execute(transfer); } catch (RejectedExecutionException ex) { - LOG.info("Cancelling the transfer of " + rsZnode + " because of " + ex.getMessage()); - } - } - - /** - * Clear the references to the specified old source - * @param src source to clear - */ - public void closeRecoveredQueue(ReplicationSourceInterface src) { - LOG.info("Done with the recovered queue " + src.getPeerClusterZnode()); - if (src instanceof ReplicationSource) { - ((ReplicationSource) src).getSourceMetrics().clear(); + LOG.info("Cancelling the transfer of " + deadRS + " because of " + ex.getMessage()); } - this.oldsources.remove(src); - deleteSource(src.getPeerClusterZnode(), false); - this.walsByIdRecoveredQueues.remove(src.getPeerClusterZnode()); } /** - * Clear the references to the specified old source - * @param src source to clear - */ - public void closeQueue(ReplicationSourceInterface src) { - LOG.info("Done with the queue " + src.getPeerClusterZnode()); - src.getSourceMetrics().clear(); - this.sources.remove(src); - deleteSource(src.getPeerClusterZnode(), true); - this.walsById.remove(src.getPeerClusterZnode()); - } - - /** - * Thie method first deletes all the recovered sources for the specified - * id, then deletes the normal source (deleting all related data in ZK). - * @param id The id of the peer cluster - */ - public void removePeer(String id) { - LOG.info("Closing the following queue " + id + ", currently have " - + sources.size() + " and another " - + oldsources.size() + " that were recovered"); - String terminateMessage = "Replication stream was removed by a user"; - List oldSourcesToDelete = new ArrayList<>(); - // synchronized on oldsources to avoid adding recovered source for the to-be-removed peer - // see NodeFailoverWorker.run - synchronized (oldsources) { - // First close all the recovered sources for this peer - for (ReplicationSourceInterface src : oldsources) { - if (id.equals(src.getPeerId())) { - oldSourcesToDelete.add(src); - } - } - for (ReplicationSourceInterface src : oldSourcesToDelete) { - src.terminate(terminateMessage); - closeRecoveredQueue(src); - } - } - LOG.info("Number of deleted recovered sources for " + id + ": " - + oldSourcesToDelete.size()); - // Now look for the one on this cluster - List srcToRemove = new ArrayList<>(); - // synchronize on replicationPeers to avoid adding source for the to-be-removed peer - synchronized (this.replicationPeers) { - for (ReplicationSourceInterface src : this.sources) { - if (id.equals(src.getPeerId())) { - srcToRemove.add(src); - } - } - if (srcToRemove.isEmpty()) { - LOG.error("The peer we wanted to remove is missing a ReplicationSourceInterface. " + - "This could mean that ReplicationSourceInterface initialization failed for this peer " + - "and that replication on this peer may not be caught up. peerId=" + id); - } - for (ReplicationSourceInterface toRemove : srcToRemove) { - toRemove.terminate(terminateMessage); - closeQueue(toRemove); - } - deleteSource(id, true); - } - } - - @Override - public void regionServerRemoved(String regionserver) { - transferQueues(regionserver); - } - - @Override - public void peerRemoved(String peerId) { - removePeer(peerId); - this.replicationQueues.removePeerFromHFileRefs(peerId); - } - - @Override - public void peerListChanged(List peerIds) { - for (String id : peerIds) { - try { - boolean added = this.replicationPeers.peerConnected(id); - if (added) { - addSource(id); - if (replicationForBulkLoadDataEnabled) { - this.replicationQueues.addPeerToHFileRefs(id); - } - } - } catch (Exception e) { - LOG.error("Error while adding a new peer", e); - } - } - } - - /** - * Class responsible to setup new ReplicationSources to take care of the - * queues from dead region servers. + * Class responsible to setup new ReplicationSources to take care of the queues from dead region + * servers. */ class NodeFailoverWorker extends Thread { - private String rsZnode; - private final ReplicationQueues rq; - private final ReplicationPeers rp; - private final UUID clusterId; + private final ServerName deadRS; - /** - * @param rsZnode - */ - public NodeFailoverWorker(String rsZnode) { - this(rsZnode, replicationQueues, replicationPeers, ReplicationSourceManager.this.clusterId); - } - - public NodeFailoverWorker(String rsZnode, final ReplicationQueues replicationQueues, - final ReplicationPeers replicationPeers, final UUID clusterId) { - super("Failover-for-"+rsZnode); - this.rsZnode = rsZnode; - this.rq = replicationQueues; - this.rp = replicationPeers; - this.clusterId = clusterId; + @VisibleForTesting + public NodeFailoverWorker(ServerName deadRS) { + super("Failover-for-" + deadRS); + this.deadRS = deadRS; } @Override public void run() { - if (this.rq.isThisOurRegionServer(rsZnode)) { - return; - } // Wait a bit before transferring the queues, we may be shutting down. // This sleep may not be enough in some cases. try { Thread.sleep(sleepBeforeFailover + - (long) (ThreadLocalRandom.current().nextFloat() * sleepBeforeFailover)); + (long) (ThreadLocalRandom.current().nextFloat() * sleepBeforeFailover)); } catch (InterruptedException e) { LOG.warn("Interrupted while waiting before transferring a queue."); Thread.currentThread().interrupt(); @@ -688,25 +650,30 @@ public class ReplicationSourceManager implements ReplicationListener { return; } Map> newQueues = new HashMap<>(); - List peers = rq.getUnClaimedQueueIds(rsZnode); - while (peers != null && !peers.isEmpty()) { - Pair> peer = this.rq.claimQueue(rsZnode, - peers.get(ThreadLocalRandom.current().nextInt(peers.size()))); - long sleep = sleepBeforeFailover/2; - if (peer != null) { - newQueues.put(peer.getFirst(), peer.getSecond()); - sleep = sleepBeforeFailover; + try { + List queues = queueStorage.getAllQueues(deadRS); + while (!queues.isEmpty()) { + Pair> peer = queueStorage.claimQueue(deadRS, + queues.get(ThreadLocalRandom.current().nextInt(queues.size())), server.getServerName()); + long sleep = sleepBeforeFailover / 2; + if (!peer.getSecond().isEmpty()) { + newQueues.put(peer.getFirst(), peer.getSecond()); + sleep = sleepBeforeFailover; + } + try { + Thread.sleep(sleep); + } catch (InterruptedException e) { + LOG.warn("Interrupted while waiting before transferring a queue."); + Thread.currentThread().interrupt(); + } + queues = queueStorage.getAllQueues(deadRS); } - try { - Thread.sleep(sleep); - } catch (InterruptedException e) { - LOG.warn("Interrupted while waiting before transferring a queue."); - Thread.currentThread().interrupt(); + if (queues.isEmpty()) { + queueStorage.removeReplicatorIfQueueIsEmpty(deadRS); } - peers = rq.getUnClaimedQueueIds(rsZnode); - } - if (peers != null) { - rq.removeReplicatorIfQueueIsEmpty(rsZnode); + } catch (ReplicationException e) { + server.abort("Failed to claim queue from dead regionserver", e); + return; } // Copying over the failed queue is completed. if (newQueues.isEmpty()) { @@ -716,28 +683,23 @@ public class ReplicationSourceManager implements ReplicationListener { } for (Map.Entry> entry : newQueues.entrySet()) { - String peerId = entry.getKey(); + String queueId = entry.getKey(); Set walsSet = entry.getValue(); try { // there is not an actual peer defined corresponding to peerId for the failover. - ReplicationQueueInfo replicationQueueInfo = new ReplicationQueueInfo(peerId); + ReplicationQueueInfo replicationQueueInfo = new ReplicationQueueInfo(queueId); String actualPeerId = replicationQueueInfo.getPeerId(); - ReplicationPeer peer = replicationPeers.getConnectedPeer(actualPeerId); - ReplicationPeerConfig peerConfig = null; - try { - peerConfig = replicationPeers.getReplicationPeerConfig(actualPeerId); - } catch (ReplicationException ex) { - LOG.warn("Received exception while getting replication peer config, skipping replay" - + ex); - } - if (peer == null || peerConfig == null) { - LOG.warn("Skipping failover for peer:" + actualPeerId + " of node" + rsZnode); - replicationQueues.removeQueue(peerId); + + ReplicationPeer peer = replicationPeers.getPeer(actualPeerId); + if (peer == null) { + LOG.warn("Skipping failover for peer:" + actualPeerId + " of node " + deadRS + + ", peer is null"); + abortWhenFail(() -> queueStorage.removeQueue(server.getServerName(), queueId)); continue; } // track sources in walsByIdRecoveredQueues Map> walsByGroup = new HashMap<>(); - walsByIdRecoveredQueues.put(peerId, walsByGroup); + walsByIdRecoveredQueues.put(queueId, walsByGroup); for (String wal : walsSet) { String walPrefix = AbstractFSWALProvider.getWALPrefixFromWALName(wal); SortedSet wals = walsByGroup.get(walPrefix); @@ -748,16 +710,12 @@ public class ReplicationSourceManager implements ReplicationListener { wals.add(wal); } - // enqueue sources - ReplicationSourceInterface src = - getReplicationSource(conf, fs, ReplicationSourceManager.this, this.rq, this.rp, - server, peerId, this.clusterId, peerConfig, peer, walFileLengthProvider); + ReplicationSourceInterface src = createSource(queueId, peer); // synchronized on oldsources to avoid adding recovered source for the to-be-removed peer - // see removePeer synchronized (oldsources) { - if (!this.rp.getConnectedPeerIds().contains(src.getPeerId())) { + if (!replicationPeers.getAllPeerIds().contains(src.getPeerId())) { src.terminate("Recovered queue doesn't belong to any current peer"); - closeRecoveredQueue(src); + removeRecoveredSource(src); continue; } oldsources.add(src); @@ -774,27 +732,80 @@ public class ReplicationSourceManager implements ReplicationListener { } } - class AdoptAbandonedQueuesWorker extends Thread{ + /** + * Terminate the replication on this region server + */ + public void join() { + this.executor.shutdown(); + for (ReplicationSourceInterface source : this.sources.values()) { + source.terminate("Region server is closing"); + } + } - public AdoptAbandonedQueuesWorker() {} + /** + * Get a copy of the wals of the normal sources on this rs + * @return a sorted set of wal names + */ + @VisibleForTesting + Map>> getWALs() { + return Collections.unmodifiableMap(walsById); + } - @Override - public void run() { - List currentReplicators = replicationQueues.getListOfReplicators(); - if (currentReplicators == null || currentReplicators.isEmpty()) { - return; - } - List otherRegionServers = replicationTracker.getListOfRegionServers(); - LOG.info("Current list of replicators: " + currentReplicators + " other RSs: " - + otherRegionServers); - - // Look if there's anything to process after a restart - for (String rs : currentReplicators) { - if (!otherRegionServers.contains(rs)) { - transferQueues(rs); - } - } + /** + * Get a copy of the wals of the recovered sources on this rs + * @return a sorted set of wal names + */ + @VisibleForTesting + Map>> getWalsByIdRecoveredQueues() { + return Collections.unmodifiableMap(walsByIdRecoveredQueues); + } + + /** + * Get a list of all the normal sources of this rs + * @return list of all normal sources + */ + public List getSources() { + return new ArrayList<>(this.sources.values()); + } + + /** + * Get a list of all the recovered sources of this rs + * @return list of all recovered sources + */ + public List getOldSources() { + return this.oldsources; + } + + /** + * Get the normal source for a given peer + * @return the normal source for the give peer if it exists, otherwise null. + */ + @VisibleForTesting + public ReplicationSourceInterface getSource(String peerId) { + return this.sources.get(peerId); + } + + @VisibleForTesting + List getAllQueues() throws IOException { + List allQueues = Collections.emptyList(); + try { + allQueues = queueStorage.getAllQueues(server.getServerName()); + } catch (ReplicationException e) { + throw new IOException(e); } + return allQueues; + } + + @VisibleForTesting + int getSizeOfLatestPath() { + synchronized (latestPaths) { + return latestPaths.size(); + } + } + + @VisibleForTesting + public AtomicLong getTotalBufferUsed() { + return totalBufferUsed; } /** @@ -829,41 +840,46 @@ public class ReplicationSourceManager implements ReplicationListener { * Get the ReplicationPeers used by this ReplicationSourceManager * @return the ReplicationPeers used by this ReplicationSourceManager */ - public ReplicationPeers getReplicationPeers() {return this.replicationPeers;} + public ReplicationPeers getReplicationPeers() { + return this.replicationPeers; + } /** * Get a string representation of all the sources' metrics */ public String getStats() { StringBuilder stats = new StringBuilder(); - for (ReplicationSourceInterface source : sources) { + for (ReplicationSourceInterface source : this.sources.values()) { stats.append("Normal source for cluster " + source.getPeerId() + ": "); stats.append(source.getStats() + "\n"); } for (ReplicationSourceInterface oldSource : oldsources) { - stats.append("Recovered source for cluster/machine(s) " + oldSource.getPeerId()+": "); - stats.append(oldSource.getStats()+ "\n"); + stats.append("Recovered source for cluster/machine(s) " + oldSource.getPeerId() + ": "); + stats.append(oldSource.getStats() + "\n"); } return stats.toString(); } public void addHFileRefs(TableName tableName, byte[] family, List> pairs) - throws ReplicationException { - for (ReplicationSourceInterface source : this.sources) { - source.addHFileRefs(tableName, family, pairs); + throws IOException { + for (ReplicationSourceInterface source : this.sources.values()) { + throwIOExceptionWhenFail(() -> source.addHFileRefs(tableName, family, pairs)); } } public void cleanUpHFileRefs(String peerId, List files) { - this.replicationQueues.removeHFileRefs(peerId, files); + abortWhenFail(() -> this.queueStorage.removeHFileRefs(peerId, files)); + } + + int activeFailoverTaskCount() { + return executor.getActiveCount(); } /** - * Whether an entry can be pushed to the peer or not right now. - * If we enable serial replication, we can not push the entry until all entries in its region - * whose sequence numbers are smaller than this entry have been pushed. - * For each ReplicationSource, we need only check the first entry in each region, as long as it - * can be pushed, we can push all in this ReplicationSource. + * Whether an entry can be pushed to the peer or not right now. If we enable serial replication, + * we can not push the entry until all entries in its region whose sequence numbers are smaller + * than this entry have been pushed. For each ReplicationSource, we need only check the first + * entry in each region, as long as it can be pushed, we can push all in this ReplicationSource. * This method will be blocked until we can push. * @return the first barrier of entry's region, or -1 if there is no barrier. It is used to * prevent saving positions in the region of no barrier. @@ -874,22 +890,18 @@ public class ReplicationSourceManager implements ReplicationListener { /** * There are barriers for this region and position for this peer. N barriers form N intervals, * (b1,b2) (b2,b3) ... (bn,max). Generally, there is no logs whose seq id is not greater than - * the first barrier and the last interval is start from the last barrier. - * - * There are several conditions that we can push now, otherwise we should block: - * 1) "Serial replication" is not enabled, we can push all logs just like before. This case - * should not call this method. - * 2) There is no barriers for this region, or the seq id is smaller than the first barrier. - * It is mainly because we alter REPLICATION_SCOPE = 2. We can not guarantee the - * order of logs that is written before altering. - * 3) This entry is in the first interval of barriers. We can push them because it is the - * start of a region. But if the region is created by region split, we should check - * if the parent regions are fully pushed. - * 4) If the entry's seq id and the position are in same section, or the pos is the last - * number of previous section. Because when open a region we put a barrier the number - * is the last log's id + 1. - * 5) Log's seq is smaller than pos in meta, we are retrying. It may happen when a RS crashes - * after save replication meta and before save zk offset. + * the first barrier and the last interval is start from the last barrier. There are several + * conditions that we can push now, otherwise we should block: 1) "Serial replication" is not + * enabled, we can push all logs just like before. This case should not call this method. 2) + * There is no barriers for this region, or the seq id is smaller than the first barrier. It is + * mainly because we alter REPLICATION_SCOPE = 2. We can not guarantee the order of logs that is + * written before altering. 3) This entry is in the first interval of barriers. We can push them + * because it is the start of a region. But if the region is created by region split, we should + * check if the parent regions are fully pushed. 4) If the entry's seq id and the position are + * in same section, or the pos is the last number of previous section. Because when open a + * region we put a barrier the number is the last log's id + 1. 5) Log's seq is smaller than pos + * in meta, we are retrying. It may happen when a RS crashes after save replication meta and + * before save zk offset. */ List barriers = MetaTableAccessor.getReplicationBarriers(connection, encodedName); if (barriers.isEmpty() || seq <= barriers.get(0)) { @@ -903,8 +915,8 @@ public class ReplicationSourceManager implements ReplicationListener { if (interval == 1) { // Case 3 // Check if there are parent regions - String parentValue = MetaTableAccessor.getSerialReplicationParentRegion(connection, - encodedName); + String parentValue = + MetaTableAccessor.getSerialReplicationParentRegion(connection, encodedName); if (parentValue == null) { // This region has no parent or the parent's log entries are fully pushed. return; @@ -916,16 +928,17 @@ public class ReplicationSourceManager implements ReplicationListener { byte[] region = Bytes.toBytes(parent); long pos = MetaTableAccessor.getReplicationPositionForOnePeer(connection, region, peerId); List parentBarriers = MetaTableAccessor.getReplicationBarriers(connection, region); - if (parentBarriers.size() > 0 - && parentBarriers.get(parentBarriers.size() - 1) - 1 > pos) { + if (parentBarriers.size() > 0 && + parentBarriers.get(parentBarriers.size() - 1) - 1 > pos) { allParentDone = false; // For a closed region, we will write a close event marker to WAL whose sequence id is // larger than final barrier but still smaller than next region's openSeqNum. // So if the pos is larger than last barrier, we can say we have read the event marker // which means the parent region has been fully pushed. - LOG.info(Bytes.toString(encodedName) + " can not start pushing because parent region's" - + " log has not been fully pushed: parent=" + Bytes.toString(region) + " pos=" + pos - + " barriers=" + Arrays.toString(barriers.toArray())); + LOG.info( + Bytes.toString(encodedName) + " can not start pushing because parent region's" + + " log has not been fully pushed: parent=" + Bytes.toString(region) + " pos=" + pos + + " barriers=" + Arrays.toString(barriers.toArray())); break; } } @@ -939,7 +952,8 @@ public class ReplicationSourceManager implements ReplicationListener { } while (true) { - long pos = MetaTableAccessor.getReplicationPositionForOnePeer(connection, encodedName, peerId); + long pos = + MetaTableAccessor.getReplicationPositionForOnePeer(connection, encodedName, peerId); if (seq <= pos) { // Case 5 } @@ -954,9 +968,9 @@ public class ReplicationSourceManager implements ReplicationListener { } } - LOG.info(Bytes.toString(encodedName) + " can not start pushing to peer " + peerId - + " because previous log has not been pushed: sequence=" + seq + " pos=" + pos - + " barriers=" + Arrays.toString(barriers.toArray())); + LOG.info(Bytes.toString(encodedName) + " can not start pushing to peer " + peerId + + " because previous log has not been pushed: sequence=" + seq + " pos=" + pos + + " barriers=" + Arrays.toString(barriers.toArray())); Thread.sleep(replicationWaitTime); } } 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 ea98cda..808f738 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 @@ -277,8 +277,8 @@ public class ReplicationSourceShipper extends Thread { } protected void updateLogPosition(long lastReadPosition) { - source.getSourceManager().logPositionAndCleanOldLogs(currentPath, source.getPeerClusterZnode(), - lastReadPosition, false, false); + source.getSourceManager().logPositionAndCleanOldLogs(currentPath, source.getQueueId(), + lastReadPosition, false); lastLoggedPosition = lastReadPosition; } @@ -295,7 +295,7 @@ public class ReplicationSourceShipper extends Thread { public void startup(UncaughtExceptionHandler handler) { String name = Thread.currentThread().getName(); Threads.setDaemonThreadRunning(this, name + ".replicationSource." + walGroupId + "," - + source.getPeerClusterZnode(), handler); + + source.getQueueId(), handler); } public PriorityBlockingQueue getLogQueue() { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReader.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReader.java index 1ec797f..e2201e5 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReader.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReader.java @@ -115,7 +115,7 @@ public class ReplicationSourceWALReader extends Thread { this.conf.getInt("replication.source.maxretriesmultiplier", 300); // 5 minutes @ 1 sec per this.eofAutoRecovery = conf.getBoolean("replication.source.eof.autorecovery", false); this.entryBatchQueue = new LinkedBlockingQueue<>(batchCount); - LOG.info("peerClusterZnode=" + source.getPeerClusterZnode() + LOG.info("peerClusterZnode=" + source.getQueueId() + ", ReplicationSourceWALReaderThread : " + source.getPeerId() + " inited, replicationBatchSizeCapacity=" + replicationBatchSizeCapacity + ", replicationBatchCountCapacity=" + replicationBatchCountCapacity diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java index 21b8ac5..9ec244a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -18,7 +18,6 @@ package org.apache.hadoop.hbase.replication.regionserver; import java.io.IOException; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configured; import org.apache.hadoop.fs.FileSystem; @@ -37,22 +36,19 @@ import org.apache.hadoop.hbase.zookeeper.MetaTableLocator; import org.apache.hadoop.hbase.zookeeper.ZKWatcher; import org.apache.hadoop.util.Tool; import org.apache.hadoop.util.ToolRunner; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** - * In a scenario of Replication based Disaster/Recovery, when hbase - * Master-Cluster crashes, this tool is used to sync-up the delta from Master to - * Slave using the info from ZooKeeper. The tool will run on Master-Cluser, and - * assume ZK, Filesystem and NetWork still available after hbase crashes + * In a scenario of Replication based Disaster/Recovery, when hbase Master-Cluster crashes, this + * tool is used to sync-up the delta from Master to Slave using the info from ZooKeeper. The tool + * will run on Master-Cluser, and assume ZK, Filesystem and NetWork still available after hbase + * crashes * + *

      * hbase org.apache.hadoop.hbase.replication.regionserver.ReplicationSyncUp
    + * 
    */ - public class ReplicationSyncUp extends Configured implements Tool { - private static final Logger LOG = LoggerFactory.getLogger(ReplicationSyncUp.class.getName()); - private static Configuration conf; private static final long SLEEP_TIME = 10000; @@ -105,13 +101,14 @@ public class ReplicationSyncUp extends Configured implements Tool { System.out.println("Start Replication Server start"); replication = new Replication(new DummyServer(zkw), fs, logDir, oldLogDir); manager = replication.getReplicationManager(); - manager.init(); + manager.init().get(); try { - int numberOfOldSource = 1; // default wait once - while (numberOfOldSource > 0) { + while (manager.activeFailoverTaskCount() > 0) { + Thread.sleep(SLEEP_TIME); + } + while (manager.getOldSources().size() > 0) { Thread.sleep(SLEEP_TIME); - numberOfOldSource = manager.getOldSources().size(); } } catch (InterruptedException e) { System.err.println("didn't wait long enough:" + e); @@ -121,7 +118,7 @@ public class ReplicationSyncUp extends Configured implements Tool { manager.join(); zkw.close(); - return (0); + return 0; } static class DummyServer implements Server { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java index 602af91..e3a1bb2 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -119,13 +119,6 @@ import org.apache.hadoop.hbase.security.Superusers; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.security.UserProvider; import org.apache.hadoop.hbase.security.access.Permission.Action; -import org.apache.hbase.thirdparty.com.google.common.collect.ArrayListMultimap; -import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableSet; -import org.apache.hbase.thirdparty.com.google.common.collect.ListMultimap; -import org.apache.hbase.thirdparty.com.google.common.collect.Lists; -import org.apache.hbase.thirdparty.com.google.common.collect.MapMaker; -import org.apache.hbase.thirdparty.com.google.common.collect.Maps; -import org.apache.hbase.thirdparty.com.google.common.collect.Sets; import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils; import org.apache.hadoop.hbase.util.ByteRange; import org.apache.hadoop.hbase.util.Bytes; @@ -138,6 +131,14 @@ import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hbase.thirdparty.com.google.common.collect.ArrayListMultimap; +import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableSet; +import org.apache.hbase.thirdparty.com.google.common.collect.ListMultimap; +import org.apache.hbase.thirdparty.com.google.common.collect.Lists; +import org.apache.hbase.thirdparty.com.google.common.collect.MapMaker; +import org.apache.hbase.thirdparty.com.google.common.collect.Maps; +import org.apache.hbase.thirdparty.com.google.common.collect.Sets; + /** * Provides basic authorization checks for data access and administrative * operations. @@ -428,7 +429,6 @@ public class AccessController implements MasterCoprocessor, RegionCoprocessor, private User getActiveUser(ObserverContext ctx) throws IOException { // for non-rpc handling, fallback to system user Optional optionalUser = ctx.getCaller(); - User user; if (optionalUser.isPresent()) { return optionalUser.get(); } @@ -2649,6 +2649,12 @@ public class AccessController implements MasterCoprocessor, RegionCoprocessor, } @Override + public void preExecuteProcedures(ObserverContext ctx) + throws IOException { + checkSystemOrSuperUser(getActiveUser(ctx)); + } + + @Override public void preMoveServersAndTables(ObserverContext ctx, Set
    servers, Set tables, String targetGroup) throws IOException { requirePermission(getActiveUser(ctx), "moveServersAndTables", Action.ADMIN); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java index 5ca1ed6..0213fa3 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java @@ -116,6 +116,7 @@ import org.apache.hadoop.hbase.regionserver.HRegionFileSystem; import org.apache.hadoop.hbase.regionserver.StoreFileInfo; import org.apache.hadoop.hbase.regionserver.wal.MetricsWAL; import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener; +import org.apache.hadoop.hbase.replication.ReplicationException; import org.apache.hadoop.hbase.security.AccessDeniedException; import org.apache.hadoop.hbase.security.UserProvider; import org.apache.hadoop.hbase.util.Bytes.ByteArrayComparator; @@ -142,6 +143,7 @@ import org.apache.yetus.audience.InterfaceStability; import org.apache.zookeeper.KeeperException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.hbase.thirdparty.com.google.common.base.Joiner; import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList; @@ -149,6 +151,7 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Lists; import org.apache.hbase.thirdparty.com.google.common.collect.Multimap; import org.apache.hbase.thirdparty.com.google.common.collect.Ordering; import org.apache.hbase.thirdparty.com.google.common.collect.TreeMultimap; + import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface; @@ -327,8 +330,7 @@ public class HBaseFsck extends Configured implements Closeable { * @throws MasterNotRunningException if the master is not running * @throws ZooKeeperConnectionException if unable to connect to ZooKeeper */ - public HBaseFsck(Configuration conf) throws MasterNotRunningException, - ZooKeeperConnectionException, IOException, ClassNotFoundException { + public HBaseFsck(Configuration conf) throws IOException, ClassNotFoundException { this(conf, createThreadPool(conf)); } @@ -752,7 +754,7 @@ public class HBaseFsck extends Configured implements Closeable { * @return 0 on success, non-zero on failure */ public int onlineHbck() - throws IOException, KeeperException, InterruptedException { + throws IOException, KeeperException, InterruptedException, ReplicationException { // print hbase server version errors.print("Version: " + status.getHBaseVersion()); @@ -3572,8 +3574,8 @@ public class HBaseFsck extends Configured implements Closeable { return hbi; } - private void checkAndFixReplication() throws IOException { - ReplicationChecker checker = new ReplicationChecker(getConf(), zkw, connection, errors); + private void checkAndFixReplication() throws ReplicationException { + ReplicationChecker checker = new ReplicationChecker(getConf(), zkw, errors); checker.checkUnDeletedQueues(); if (checker.hasUnDeletedQueues() && this.fixReplication) { @@ -4861,8 +4863,8 @@ public class HBaseFsck extends Configured implements Closeable { }; - public HBaseFsck exec(ExecutorService exec, String[] args) throws KeeperException, IOException, - InterruptedException { + public HBaseFsck exec(ExecutorService exec, String[] args) + throws KeeperException, IOException, InterruptedException, ReplicationException { long sleepBeforeRerun = DEFAULT_SLEEP_BEFORE_RERUN; boolean checkCorruptHFiles = false; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/hbck/ReplicationChecker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/hbck/ReplicationChecker.java index 839b5ad..c08c654 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/hbck/ReplicationChecker.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/hbck/ReplicationChecker.java @@ -15,86 +15,117 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.util.hbck; -import java.io.IOException; +import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Map.Entry; import java.util.Set; - import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.zookeeper.ZKWatcher; -import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hadoop.hbase.client.ClusterConnection; -import org.apache.hadoop.hbase.master.cleaner.ReplicationZKNodeCleaner; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.replication.ReplicationException; +import org.apache.hadoop.hbase.replication.ReplicationPeerStorage; import org.apache.hadoop.hbase.replication.ReplicationQueueInfo; +import org.apache.hadoop.hbase.replication.ReplicationQueueStorage; +import org.apache.hadoop.hbase.replication.ReplicationStorageFactory; import org.apache.hadoop.hbase.util.HBaseFsck; import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter; +import org.apache.hadoop.hbase.zookeeper.ZKWatcher; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; -/* +/** * Check and fix undeleted replication queues for removed peerId. */ @InterfaceAudience.Private public class ReplicationChecker { + + private static final Logger LOG = LoggerFactory.getLogger(ReplicationChecker.class); + private final ErrorReporter errorReporter; // replicator with its queueIds for removed peers - private Map> undeletedQueueIds = new HashMap<>(); + private Map> undeletedQueueIds = new HashMap<>(); // replicator with its undeleted queueIds for removed peers in hfile-refs queue - private Set undeletedHFileRefsQueueIds = new HashSet<>(); - private final ReplicationZKNodeCleaner cleaner; + private Set undeletedHFileRefsPeerIds = new HashSet<>(); - public ReplicationChecker(Configuration conf, ZKWatcher zkw, ClusterConnection connection, - ErrorReporter errorReporter) throws IOException { - this.cleaner = new ReplicationZKNodeCleaner(conf, zkw, connection); + private final ReplicationPeerStorage peerStorage; + private final ReplicationQueueStorage queueStorage; + + public ReplicationChecker(Configuration conf, ZKWatcher zkw, ErrorReporter errorReporter) { + this.peerStorage = ReplicationStorageFactory.getReplicationPeerStorage(zkw, conf); + this.queueStorage = ReplicationStorageFactory.getReplicationQueueStorage(zkw, conf); this.errorReporter = errorReporter; } public boolean hasUnDeletedQueues() { - return errorReporter.getErrorList().contains( - HBaseFsck.ErrorReporter.ERROR_CODE.UNDELETED_REPLICATION_QUEUE); + return errorReporter.getErrorList() + .contains(HBaseFsck.ErrorReporter.ERROR_CODE.UNDELETED_REPLICATION_QUEUE); } - public void checkUnDeletedQueues() throws IOException { - undeletedQueueIds = cleaner.getUnDeletedQueues(); - for (Entry> replicatorAndQueueIds : undeletedQueueIds.entrySet()) { - String replicator = replicatorAndQueueIds.getKey(); - for (String queueId : replicatorAndQueueIds.getValue()) { + private Map> getUnDeletedQueues() throws ReplicationException { + Map> undeletedQueues = new HashMap<>(); + Set peerIds = new HashSet<>(peerStorage.listPeerIds()); + for (ServerName replicator : queueStorage.getListOfReplicators()) { + for (String queueId : queueStorage.getAllQueues(replicator)) { ReplicationQueueInfo queueInfo = new ReplicationQueueInfo(queueId); - String msg = "Undeleted replication queue for removed peer found: " - + String.format("[removedPeerId=%s, replicator=%s, queueId=%s]", queueInfo.getPeerId(), - replicator, queueId); - errorReporter.reportError(HBaseFsck.ErrorReporter.ERROR_CODE.UNDELETED_REPLICATION_QUEUE, - msg); + if (!peerIds.contains(queueInfo.getPeerId())) { + undeletedQueues.computeIfAbsent(replicator, key -> new ArrayList<>()).add(queueId); + LOG.debug( + "Undeleted replication queue for removed peer found: " + + "[removedPeerId={}, replicator={}, queueId={}]", + queueInfo.getPeerId(), replicator, queueId); + } } } - - checkUnDeletedHFileRefsQueues(); + return undeletedQueues; } - private void checkUnDeletedHFileRefsQueues() throws IOException { - undeletedHFileRefsQueueIds = cleaner.getUnDeletedHFileRefsQueues(); - if (undeletedHFileRefsQueueIds != null && !undeletedHFileRefsQueueIds.isEmpty()) { - String msg = "Undeleted replication hfile-refs queue for removed peer found: " - + undeletedHFileRefsQueueIds + " under hfile-refs node"; - errorReporter - .reportError(HBaseFsck.ErrorReporter.ERROR_CODE.UNDELETED_REPLICATION_QUEUE, msg); + private Set getUndeletedHFileRefsPeers() throws ReplicationException { + Set undeletedHFileRefsPeerIds = + new HashSet<>(queueStorage.getAllPeersFromHFileRefsQueue()); + Set peerIds = new HashSet<>(peerStorage.listPeerIds()); + undeletedHFileRefsPeerIds.removeAll(peerIds); + if (LOG.isDebugEnabled()) { + for (String peerId : undeletedHFileRefsPeerIds) { + LOG.debug("Undeleted replication hfile-refs queue for removed peer {} found", peerId); + } } + return undeletedHFileRefsPeerIds; } - public void fixUnDeletedQueues() throws IOException { - if (!undeletedQueueIds.isEmpty()) { - cleaner.removeQueues(undeletedQueueIds); - } - fixUnDeletedHFileRefsQueue(); + public void checkUnDeletedQueues() throws ReplicationException { + undeletedQueueIds = getUnDeletedQueues(); + undeletedQueueIds.forEach((replicator, queueIds) -> { + queueIds.forEach(queueId -> { + ReplicationQueueInfo queueInfo = new ReplicationQueueInfo(queueId); + String msg = "Undeleted replication queue for removed peer found: " + + String.format("[removedPeerId=%s, replicator=%s, queueId=%s]", queueInfo.getPeerId(), + replicator, queueId); + errorReporter.reportError(HBaseFsck.ErrorReporter.ERROR_CODE.UNDELETED_REPLICATION_QUEUE, + msg); + }); + }); + undeletedHFileRefsPeerIds = getUndeletedHFileRefsPeers(); + undeletedHFileRefsPeerIds.stream() + .map( + peerId -> "Undeleted replication hfile-refs queue for removed peer " + peerId + " found") + .forEach(msg -> errorReporter + .reportError(HBaseFsck.ErrorReporter.ERROR_CODE.UNDELETED_REPLICATION_QUEUE, msg)); } - private void fixUnDeletedHFileRefsQueue() throws IOException { - if (undeletedHFileRefsQueueIds != null && !undeletedHFileRefsQueueIds.isEmpty()) { - cleaner.removeHFileRefsQueues(undeletedHFileRefsQueueIds); + public void fixUnDeletedQueues() throws ReplicationException { + for (Map.Entry> replicatorAndQueueIds : undeletedQueueIds.entrySet()) { + ServerName replicator = replicatorAndQueueIds.getKey(); + for (String queueId : replicatorAndQueueIds.getValue()) { + queueStorage.removeQueue(replicator, queueId); + } + queueStorage.removeReplicatorIfQueueIsEmpty(replicator); + } + for (String peerId : undeletedHFileRefsPeerIds) { + queueStorage.removePeerFromHFileRefs(peerId); } } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestJMXConnectorServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestJMXConnectorServer.java index 444db64..9f68252 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestJMXConnectorServer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestJMXConnectorServer.java @@ -202,5 +202,12 @@ public class TestJMXConnectorServer { throw new AccessDeniedException("Insufficient permissions to shut down cluster."); } } + + @Override + public void preExecuteProcedures(ObserverContext ctx) + throws IOException { + // FIXME: ignore the procedure permission check since in our UT framework master is neither + // the systemuser nor the superuser so we can not call executeProcedures... + } } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncReplicationAdminApi.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncReplicationAdminApi.java index 28a7562..5225add 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncReplicationAdminApi.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncReplicationAdminApi.java @@ -25,6 +25,7 @@ import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import java.io.IOException; import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; @@ -34,10 +35,13 @@ import java.util.Set; import java.util.concurrent.CompletionException; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.replication.ReplicationException; import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; import org.apache.hadoop.hbase.replication.ReplicationPeerDescription; +import org.apache.hadoop.hbase.replication.ReplicationQueueStorage; +import org.apache.hadoop.hbase.replication.ReplicationStorageFactory; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.junit.After; @@ -56,8 +60,8 @@ public class TestAsyncReplicationAdminApi extends TestAsyncAdminBase { private final String ID_ONE = "1"; private final String KEY_ONE = "127.0.0.1:2181:/hbase"; - private final String ID_SECOND = "2"; - private final String KEY_SECOND = "127.0.0.1:2181:/hbase2"; + private final String ID_TWO = "2"; + private final String KEY_TWO = "127.0.0.1:2181:/hbase2"; @BeforeClass public static void setUpBeforeClass() throws Exception { @@ -70,16 +74,21 @@ public class TestAsyncReplicationAdminApi extends TestAsyncAdminBase { } @After - public void cleanupPeer() { + public void clearPeerAndQueues() throws IOException, ReplicationException { try { admin.removeReplicationPeer(ID_ONE).join(); } catch (Exception e) { - LOG.debug("Replication peer " + ID_ONE + " may already be removed"); } try { - admin.removeReplicationPeer(ID_SECOND).join(); + admin.removeReplicationPeer(ID_TWO).join(); } catch (Exception e) { - LOG.debug("Replication peer " + ID_SECOND + " may already be removed"); + } + ReplicationQueueStorage queueStorage = ReplicationStorageFactory + .getReplicationQueueStorage(TEST_UTIL.getZooKeeperWatcher(), TEST_UTIL.getConfiguration()); + for (ServerName serverName : queueStorage.getListOfReplicators()) { + for (String queue : queueStorage.getAllQueues(serverName)) { + queueStorage.removeQueue(serverName, queue); + } } } @@ -88,7 +97,7 @@ public class TestAsyncReplicationAdminApi extends TestAsyncAdminBase { ReplicationPeerConfig rpc1 = new ReplicationPeerConfig(); rpc1.setClusterKey(KEY_ONE); ReplicationPeerConfig rpc2 = new ReplicationPeerConfig(); - rpc2.setClusterKey(KEY_SECOND); + rpc2.setClusterKey(KEY_TWO); // Add a valid peer admin.addReplicationPeer(ID_ONE, rpc1).join(); // try adding the same (fails) @@ -101,19 +110,19 @@ public class TestAsyncReplicationAdminApi extends TestAsyncAdminBase { assertEquals(1, admin.listReplicationPeers().get().size()); // Try to remove an inexisting peer try { - admin.removeReplicationPeer(ID_SECOND).join(); + admin.removeReplicationPeer(ID_TWO).join(); fail("Test case should fail as removing a inexisting peer."); } catch (CompletionException e) { // OK! } assertEquals(1, admin.listReplicationPeers().get().size()); // Add a second since multi-slave is supported - admin.addReplicationPeer(ID_SECOND, rpc2).join(); + admin.addReplicationPeer(ID_TWO, rpc2).join(); assertEquals(2, admin.listReplicationPeers().get().size()); // Remove the first peer we added admin.removeReplicationPeer(ID_ONE).join(); assertEquals(1, admin.listReplicationPeers().get().size()); - admin.removeReplicationPeer(ID_SECOND).join(); + admin.removeReplicationPeer(ID_TWO).join(); assertEquals(0, admin.listReplicationPeers().get().size()); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java index fb29e9e..a6091e1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java @@ -17,6 +17,12 @@ */ package org.apache.hadoop.hbase.client.replication; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; import java.io.IOException; import java.util.ArrayList; @@ -25,24 +31,23 @@ import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.atomic.AtomicLong; import java.util.regex.Pattern; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.ReplicationPeerNotFoundException; +import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; 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.ReplicationPeerConfigBuilder; import org.apache.hadoop.hbase.replication.ReplicationPeerDescription; -import org.apache.hadoop.hbase.replication.ReplicationQueues; -import org.apache.hadoop.hbase.replication.ReplicationQueuesArguments; +import org.apache.hadoop.hbase.replication.ReplicationQueueStorage; +import org.apache.hadoop.hbase.replication.ReplicationStorageFactory; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.MediumTests; -import org.apache.hadoop.hbase.zookeeper.ZKWatcher; import org.junit.After; import org.junit.AfterClass; import org.junit.BeforeClass; @@ -53,21 +58,14 @@ import org.junit.rules.TestName; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - /** * Unit testing of ReplicationAdmin */ @Category({MediumTests.class, ClientTests.class}) public class TestReplicationAdmin { - private static final Logger LOG = - LoggerFactory.getLogger(TestReplicationAdmin.class); + private static final Logger LOG = LoggerFactory.getLogger(TestReplicationAdmin.class); + private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); @@ -102,16 +100,103 @@ public class TestReplicationAdmin { } @After - public void cleanupPeer() { + public void tearDown() throws Exception { + for (ReplicationPeerDescription desc : hbaseAdmin.listReplicationPeers()) { + hbaseAdmin.removeReplicationPeer(desc.getPeerId()); + } + ReplicationQueueStorage queueStorage = ReplicationStorageFactory + .getReplicationQueueStorage(TEST_UTIL.getZooKeeperWatcher(), TEST_UTIL.getConfiguration()); + for (ServerName serverName : queueStorage.getListOfReplicators()) { + for (String queue : queueStorage.getAllQueues(serverName)) { + queueStorage.removeQueue(serverName, queue); + } + queueStorage.removeReplicatorIfQueueIsEmpty(serverName); + } + } + + @Test + public void testConcurrentPeerOperations() throws Exception { + int threadNum = 5; + AtomicLong successCount = new AtomicLong(0); + + // Test concurrent add peer operation + Thread[] addPeers = new Thread[threadNum]; + for (int i = 0; i < threadNum; i++) { + addPeers[i] = new Thread(() -> { + try { + hbaseAdmin.addReplicationPeer(ID_ONE, + ReplicationPeerConfig.newBuilder().setClusterKey(KEY_ONE).build()); + successCount.incrementAndGet(); + } catch (Exception e) { + LOG.debug("Got exception when add replication peer", e); + } + }); + addPeers[i].start(); + } + for (Thread addPeer : addPeers) { + addPeer.join(); + } + assertEquals(1, successCount.get()); + + // Test concurrent remove peer operation + successCount.set(0); + Thread[] removePeers = new Thread[threadNum]; + for (int i = 0; i < threadNum; i++) { + removePeers[i] = new Thread(() -> { + try { + hbaseAdmin.removeReplicationPeer(ID_ONE); + successCount.incrementAndGet(); + } catch (Exception e) { + LOG.debug("Got exception when remove replication peer", e); + } + }); + removePeers[i].start(); + } + for (Thread removePeer : removePeers) { + removePeer.join(); + } + assertEquals(1, successCount.get()); + + // Test concurrent add peer operation again + successCount.set(0); + addPeers = new Thread[threadNum]; + for (int i = 0; i < threadNum; i++) { + addPeers[i] = new Thread(() -> { + try { + hbaseAdmin.addReplicationPeer(ID_ONE, + ReplicationPeerConfig.newBuilder().setClusterKey(KEY_ONE).build()); + successCount.incrementAndGet(); + } catch (Exception e) { + LOG.debug("Got exception when add replication peer", e); + } + }); + addPeers[i].start(); + } + for (Thread addPeer : addPeers) { + addPeer.join(); + } + assertEquals(1, successCount.get()); + } + + @Test + public void testAddInvalidPeer() { + ReplicationPeerConfigBuilder builder = ReplicationPeerConfig.newBuilder(); + builder.setClusterKey(KEY_ONE); try { - hbaseAdmin.removeReplicationPeer(ID_ONE); + String invalidPeerId = "1-2"; + hbaseAdmin.addReplicationPeer(invalidPeerId, builder.build()); + fail("Should fail as the peer id: " + invalidPeerId + " is invalid"); } catch (Exception e) { - LOG.debug("Replication peer " + ID_ONE + " may already be removed"); + // OK } + try { - hbaseAdmin.removeReplicationPeer(ID_SECOND); + String invalidClusterKey = "2181:/hbase"; + builder.setClusterKey(invalidClusterKey); + hbaseAdmin.addReplicationPeer(ID_ONE, builder.build()); + fail("Should fail as the peer cluster key: " + invalidClusterKey + " is invalid"); } catch (Exception e) { - LOG.debug("Replication peer " + ID_SECOND + " may already be removed"); + // OK } } @@ -201,32 +286,29 @@ public class TestReplicationAdmin { ReplicationPeerConfig rpc2 = new ReplicationPeerConfig(); rpc2.setClusterKey(KEY_SECOND); Configuration conf = TEST_UTIL.getConfiguration(); - ZKWatcher zkw = new ZKWatcher(conf, "Test HBaseAdmin", null); - ReplicationQueues repQueues = - ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(conf, null, zkw)); - repQueues.init("server1"); + ReplicationQueueStorage queueStorage = + ReplicationStorageFactory.getReplicationQueueStorage(TEST_UTIL.getZooKeeperWatcher(), conf); + ServerName serverName = ServerName.valueOf("server1", 8000, 1234); // add queue for ID_ONE - repQueues.addLog(ID_ONE, "file1"); + queueStorage.addWAL(serverName, ID_ONE, "file1"); try { admin.addPeer(ID_ONE, rpc1, null); fail(); } catch (Exception e) { // OK! } - repQueues.removeQueue(ID_ONE); - assertEquals(0, repQueues.getAllQueues().size()); + queueStorage.removeQueue(serverName, ID_ONE); + assertEquals(0, queueStorage.getAllQueues(serverName).size()); // add recovered queue for ID_ONE - repQueues.addLog(ID_ONE + "-server2", "file1"); + queueStorage.addWAL(serverName, ID_ONE + "-server2", "file1"); try { admin.addPeer(ID_ONE, rpc2, null); fail(); } catch (Exception e) { // OK! } - repQueues.removeAllQueues(); - zkw.close(); } /** @@ -422,7 +504,7 @@ public class TestReplicationAdmin { tableCFs.clear(); tableCFs.put(tableName2, null); admin.removePeerTableCFs(ID_ONE, tableCFs); - assertTrue(false); + fail(); } catch (ReplicationException e) { } tableCFs.clear(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java index 7b6af0e..7196b7c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java @@ -19,6 +19,8 @@ package org.apache.hadoop.hbase.master; import static org.mockito.Mockito.mock; +import com.google.protobuf.Service; + import java.io.IOException; import java.util.List; @@ -42,6 +44,7 @@ import org.apache.hadoop.hbase.master.assignment.AssignmentManager; import org.apache.hadoop.hbase.master.locking.LockManager; import org.apache.hadoop.hbase.master.normalizer.RegionNormalizer; import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv; +import org.apache.hadoop.hbase.master.replication.ReplicationPeerManager; import org.apache.hadoop.hbase.master.snapshot.SnapshotManager; import org.apache.hadoop.hbase.procedure.MasterProcedureManagerHost; import org.apache.hadoop.hbase.procedure2.LockedResource; @@ -55,8 +58,6 @@ import org.apache.hadoop.hbase.replication.ReplicationPeerDescription; import org.apache.hadoop.hbase.zookeeper.MetaTableLocator; import org.apache.hadoop.hbase.zookeeper.ZKWatcher; -import com.google.protobuf.Service; - public class MockNoopMasterServices implements MasterServices, Server { private final Configuration conf; private final MetricsMaster metricsMaster; @@ -369,7 +370,6 @@ public class MockNoopMasterServices implements MasterServices, Server { @Override public ClusterConnection getClusterConnection() { - // TODO Auto-generated method stub return null; } @@ -399,20 +399,24 @@ public class MockNoopMasterServices implements MasterServices, Server { } @Override - public void addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled) + public long addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled) throws ReplicationException { + return 0; } @Override - public void removeReplicationPeer(String peerId) throws ReplicationException { + public long removeReplicationPeer(String peerId) throws ReplicationException { + return 0; } @Override - public void enableReplicationPeer(String peerId) throws ReplicationException, IOException { + public long enableReplicationPeer(String peerId) throws ReplicationException, IOException { + return 0; } @Override - public void disableReplicationPeer(String peerId) throws ReplicationException, IOException { + public long disableReplicationPeer(String peerId) throws ReplicationException, IOException { + return 0; } @Override @@ -422,8 +426,9 @@ public class MockNoopMasterServices implements MasterServices, Server { } @Override - public void updateReplicationPeerConfig(String peerId, ReplicationPeerConfig peerConfig) + public long updateReplicationPeerConfig(String peerId, ReplicationPeerConfig peerConfig) throws ReplicationException, IOException { + return 0; } @Override @@ -457,8 +462,7 @@ public class MockNoopMasterServices implements MasterServices, Server { } @Override - public ProcedureEvent getInitializedEvent() { - // TODO Auto-generated method stub + public ProcedureEvent getInitializedEvent() { return null; } @@ -471,4 +475,9 @@ public class MockNoopMasterServices implements MasterServices, Server { public Connection createConnection(Configuration conf) throws IOException { return null; } -} + + @Override + public ReplicationPeerManager getReplicationPeerManager() { + return null; + } +} \ No newline at end of file diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java index fd44c89..f5d36bf 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java @@ -44,6 +44,8 @@ import org.apache.hadoop.hbase.client.ClusterConnection; import org.apache.hadoop.hbase.client.HConnectionTestingUtility; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.monitoring.MonitoredTask; +import org.apache.hadoop.hbase.replication.ReplicationException; + import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest; import org.apache.hadoop.hbase.testclassification.MasterTests; @@ -270,7 +272,7 @@ public class TestMasterNoCluster { @Override void initializeZKBasedSystemTrackers() throws IOException, InterruptedException, - KeeperException, CoordinatedStateException { + KeeperException, CoordinatedStateException, ReplicationException { super.initializeZKBasedSystemTrackers(); // Record a newer server in server manager at first getServerManager().recordNewServerWithLock(newServer, diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManager.java index 3c453bc..3ab915b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManager.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManager.java @@ -531,26 +531,16 @@ public class TestAssignmentManager { @Override public ExecuteProceduresResponse sendRequest(ServerName server, ExecuteProceduresRequest request) throws IOException { - ExecuteProceduresResponse.Builder builder = ExecuteProceduresResponse.newBuilder(); if (request.getOpenRegionCount() > 0) { - for (OpenRegionRequest req: request.getOpenRegionList()) { - OpenRegionResponse.Builder resp = OpenRegionResponse.newBuilder(); - for (RegionOpenInfo openReq: req.getOpenInfoList()) { - RegionOpeningState state = execOpenRegion(server, openReq); - if (state != null) { - resp.addOpeningState(state); - } + for (OpenRegionRequest req : request.getOpenRegionList()) { + for (RegionOpenInfo openReq : req.getOpenInfoList()) { + execOpenRegion(server, openReq); } - builder.addOpenRegion(resp.build()); } } if (request.getCloseRegionCount() > 0) { - for (CloseRegionRequest req: request.getCloseRegionList()) { - CloseRegionResponse resp = execCloseRegion(server, - req.getRegion().getValue().toByteArray()); - if (resp != null) { - builder.addCloseRegion(resp); - } + for (CloseRegionRequest req : request.getCloseRegionList()) { + execCloseRegion(server, req.getRegion().getValue().toByteArray()); } } return ExecuteProceduresResponse.newBuilder().build(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java index 08b27ec..2de6608 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -24,16 +24,12 @@ import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.spy; import java.io.IOException; -import java.lang.reflect.Field; import java.net.URLEncoder; import java.util.Iterator; -import java.util.LinkedList; import java.util.List; import java.util.Random; - -import org.apache.hadoop.fs.FSDataOutputStream; -import org.apache.hbase.thirdparty.com.google.common.collect.Lists; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -48,10 +44,8 @@ import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.ZooKeeperConnectionException; import org.apache.hadoop.hbase.client.ClusterConnection; import org.apache.hadoop.hbase.client.Connection; -import org.apache.hadoop.hbase.replication.ReplicationFactory; -import org.apache.hadoop.hbase.replication.ReplicationQueues; -import org.apache.hadoop.hbase.replication.ReplicationQueuesArguments; -import org.apache.hadoop.hbase.replication.ReplicationQueuesClientZKImpl; +import org.apache.hadoop.hbase.replication.ReplicationQueueStorage; +import org.apache.hadoop.hbase.replication.ReplicationStorageFactory; import org.apache.hadoop.hbase.replication.master.ReplicationLogCleaner; import org.apache.hadoop.hbase.replication.regionserver.Replication; import org.apache.hadoop.hbase.testclassification.MasterTests; @@ -65,10 +59,11 @@ import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; import org.junit.experimental.categories.Category; -import org.mockito.Mockito; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hbase.thirdparty.com.google.common.collect.Lists; + @Category({MasterTests.class, MediumTests.class}) public class TestLogsCleaner { @@ -115,9 +110,8 @@ public class TestLogsCleaner { Replication.decorateMasterConfiguration(conf); Server server = new DummyServer(); - ReplicationQueues repQueues = ReplicationFactory.getReplicationQueues( - new ReplicationQueuesArguments(conf, server, server.getZooKeeper())); - repQueues.init(server.getServerName().toString()); + ReplicationQueueStorage queueStorage = + ReplicationStorageFactory.getReplicationQueueStorage(server.getZooKeeper(), conf); final Path oldLogDir = new Path(TEST_UTIL.getDataTestDir(), HConstants.HREGION_OLDLOGDIR_NAME); final Path oldProcedureWALDir = new Path(oldLogDir, "masterProcedureWALs"); String fakeMachineName = URLEncoder.encode(server.getServerName().toString(), "UTF8"); @@ -148,7 +142,7 @@ public class TestLogsCleaner { // Case 4: put 3 WALs in ZK indicating that they are scheduled for replication so these // files would pass TimeToLiveLogCleaner but would be rejected by ReplicationLogCleaner if (i % (30 / 3) == 1) { - repQueues.addLog(fakeMachineName, fileName.getName()); + queueStorage.addWAL(server.getServerName(), fakeMachineName, fileName.getName()); LOG.info("Replication log file: " + fileName); } } @@ -195,24 +189,6 @@ public class TestLogsCleaner { } } - @Test(timeout=5000) - public void testZnodeCversionChange() throws Exception { - Configuration conf = TEST_UTIL.getConfiguration(); - ReplicationLogCleaner cleaner = new ReplicationLogCleaner(); - cleaner.setConf(conf); - - ReplicationQueuesClientZKImpl rqcMock = Mockito.mock(ReplicationQueuesClientZKImpl.class); - Mockito.when(rqcMock.getQueuesZNodeCversion()).thenReturn(1, 2, 3, 4); - - Field rqc = ReplicationLogCleaner.class.getDeclaredField("replicationQueues"); - rqc.setAccessible(true); - - rqc.set(cleaner, rqcMock); - - // This should return eventually when cversion stabilizes - cleaner.getDeletableFiles(new LinkedList<>()); - } - /** * ReplicationLogCleaner should be able to ride over ZooKeeper errors without aborting. */ diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java index 2948701..905ade4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java @@ -1,12 +1,19 @@ -/* - * 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. +/** + * 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.master.cleaner; @@ -17,14 +24,10 @@ import static org.junit.Assert.fail; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.spy; -import org.apache.hbase.thirdparty.com.google.common.collect.Lists; - import java.io.IOException; -import java.lang.reflect.Field; import java.util.ArrayList; import java.util.Iterator; import java.util.List; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; @@ -43,10 +46,8 @@ 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.ReplicationPeers; -import org.apache.hadoop.hbase.replication.ReplicationQueues; -import org.apache.hadoop.hbase.replication.ReplicationQueuesArguments; -import org.apache.hadoop.hbase.replication.ReplicationQueuesClient; -import org.apache.hadoop.hbase.replication.ReplicationQueuesZKImpl; +import org.apache.hadoop.hbase.replication.ReplicationQueueStorage; +import org.apache.hadoop.hbase.replication.ReplicationStorageFactory; import org.apache.hadoop.hbase.replication.master.ReplicationHFileCleaner; import org.apache.hadoop.hbase.replication.regionserver.Replication; import org.apache.hadoop.hbase.testclassification.MasterTests; @@ -63,47 +64,35 @@ import org.junit.Before; import org.junit.BeforeClass; import org.junit.Test; import org.junit.experimental.categories.Category; -import org.mockito.Mockito; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hbase.thirdparty.com.google.common.collect.Lists; + @Category({ MasterTests.class, SmallTests.class }) public class TestReplicationHFileCleaner { - private static final Logger LOG = LoggerFactory.getLogger(ReplicationQueuesZKImpl.class); + private static final Logger LOG = LoggerFactory.getLogger(TestReplicationHFileCleaner.class); private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static Server server; - private static ReplicationQueues rq; + private static ReplicationQueueStorage rq; private static ReplicationPeers rp; private static final String peerId = "TestReplicationHFileCleaner"; private static Configuration conf = TEST_UTIL.getConfiguration(); static FileSystem fs = null; Path root; - /** - * @throws java.lang.Exception - */ @BeforeClass public static void setUpBeforeClass() throws Exception { TEST_UTIL.startMiniZKCluster(); server = new DummyServer(); conf.setBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY, true); Replication.decorateMasterConfiguration(conf); - rp = ReplicationFactory.getReplicationPeers(server.getZooKeeper(), conf, server); + rp = ReplicationFactory.getReplicationPeers(server.getZooKeeper(), conf); rp.init(); - rq = ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(conf, server, server.getZooKeeper())); - rq.init(server.getServerName().toString()); - try { - fs = FileSystem.get(conf); - } finally { - if (fs != null) { - fs.close(); - } - } + rq = ReplicationStorageFactory.getReplicationQueueStorage(server.getZooKeeper(), conf); + fs = FileSystem.get(conf); } - /** - * @throws java.lang.Exception - */ @AfterClass public static void tearDownAfterClass() throws Exception { TEST_UTIL.shutdownMiniZKCluster(); @@ -112,7 +101,8 @@ public class TestReplicationHFileCleaner { @Before public void setup() throws ReplicationException, IOException { root = TEST_UTIL.getDataTestDirOnTestFS(); - rp.registerPeer(peerId, new ReplicationPeerConfig().setClusterKey(TEST_UTIL.getClusterKey())); + rp.getPeerStorage().addPeer(peerId, + ReplicationPeerConfig.newBuilder().setClusterKey(TEST_UTIL.getClusterKey()).build(), true); rq.addPeerToHFileRefs(peerId); } @@ -123,7 +113,7 @@ public class TestReplicationHFileCleaner { } catch (IOException e) { LOG.warn("Failed to delete files recursively from path " + root); } - rp.unregisterPeer(peerId); + rp.getPeerStorage().removePeer(peerId); } @Test @@ -188,32 +178,6 @@ public class TestReplicationHFileCleaner { assertTrue(deletableFilesIterator.next().getPath().equals(deletablefile)); } - /* - * Test for HBASE-14621. This test will not assert directly anything. Without the fix the test - * will end up in a infinite loop, so it will timeout. - */ - @Test(timeout = 15000) - public void testForDifferntHFileRefsZnodeVersion() throws Exception { - // 1. Create a file - Path file = new Path(root, "testForDifferntHFileRefsZnodeVersion"); - fs.createNewFile(file); - // 2. Assert file is successfully created - assertTrue("Test file not created!", fs.exists(file)); - ReplicationHFileCleaner cleaner = new ReplicationHFileCleaner(); - cleaner.setConf(conf); - - ReplicationQueuesClient replicationQueuesClient = Mockito.mock(ReplicationQueuesClient.class); - //Return different znode version for each call - Mockito.when(replicationQueuesClient.getHFileRefsNodeChangeVersion()).thenReturn(1, 2); - - Class cleanerClass = cleaner.getClass(); - Field rqc = cleanerClass.getDeclaredField("rqc"); - rqc.setAccessible(true); - rqc.set(cleaner, replicationQueuesClient); - - cleaner.isFileDeletable(fs.getFileStatus(file)); - } - /** * ReplicationHFileCleaner should be able to ride over ZooKeeper errors without aborting. */ diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationZKNodeCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationZKNodeCleaner.java deleted file mode 100644 index 6aa59cb..0000000 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationZKNodeCleaner.java +++ /dev/null @@ -1,115 +0,0 @@ -/** - * 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.master.cleaner; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - -import java.util.List; -import java.util.Map; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.replication.ReplicationFactory; -import org.apache.hadoop.hbase.replication.ReplicationQueues; -import org.apache.hadoop.hbase.replication.ReplicationQueuesArguments; -import org.apache.hadoop.hbase.replication.ReplicationQueuesZKImpl; -import org.apache.hadoop.hbase.testclassification.MasterTests; -import org.apache.hadoop.hbase.testclassification.MediumTests; -import org.apache.hadoop.hbase.zookeeper.ZKWatcher; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.Test; -import org.junit.experimental.categories.Category; - -@Category({ MasterTests.class, MediumTests.class }) -public class TestReplicationZKNodeCleaner { - private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); - - private final String ID_ONE = "1"; - private final String SERVER_ONE = "server1"; - private final String ID_TWO = "2"; - private final String SERVER_TWO = "server2"; - - private final Configuration conf; - private final ZKWatcher zkw; - private final ReplicationQueues repQueues; - - public TestReplicationZKNodeCleaner() throws Exception { - conf = TEST_UTIL.getConfiguration(); - zkw = new ZKWatcher(conf, "TestReplicationZKNodeCleaner", null); - repQueues = ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(conf, null, - zkw)); - assertTrue(repQueues instanceof ReplicationQueuesZKImpl); - } - - @BeforeClass - public static void setUpBeforeClass() throws Exception { - TEST_UTIL.getConfiguration().setInt("hbase.master.cleaner.interval", 10000); - TEST_UTIL.startMiniCluster(); - } - - @AfterClass - public static void tearDownAfterClass() throws Exception { - TEST_UTIL.shutdownMiniCluster(); - } - - @Test - public void testReplicationZKNodeCleaner() throws Exception { - repQueues.init(SERVER_ONE); - // add queue for ID_ONE which isn't exist - repQueues.addLog(ID_ONE, "file1"); - - ReplicationZKNodeCleaner cleaner = new ReplicationZKNodeCleaner(conf, zkw, null); - Map> undeletedQueues = cleaner.getUnDeletedQueues(); - assertEquals(1, undeletedQueues.size()); - assertTrue(undeletedQueues.containsKey(SERVER_ONE)); - assertEquals(1, undeletedQueues.get(SERVER_ONE).size()); - assertTrue(undeletedQueues.get(SERVER_ONE).contains(ID_ONE)); - - // add a recovery queue for ID_TWO which isn't exist - repQueues.addLog(ID_TWO + "-" + SERVER_TWO, "file2"); - - undeletedQueues = cleaner.getUnDeletedQueues(); - assertEquals(1, undeletedQueues.size()); - assertTrue(undeletedQueues.containsKey(SERVER_ONE)); - assertEquals(2, undeletedQueues.get(SERVER_ONE).size()); - assertTrue(undeletedQueues.get(SERVER_ONE).contains(ID_ONE)); - assertTrue(undeletedQueues.get(SERVER_ONE).contains(ID_TWO + "-" + SERVER_TWO)); - - cleaner.removeQueues(undeletedQueues); - undeletedQueues = cleaner.getUnDeletedQueues(); - assertEquals(0, undeletedQueues.size()); - } - - @Test - public void testReplicationZKNodeCleanerChore() throws Exception { - repQueues.init(SERVER_ONE); - // add queue for ID_ONE which isn't exist - repQueues.addLog(ID_ONE, "file1"); - // add a recovery queue for ID_TWO which isn't exist - repQueues.addLog(ID_TWO + "-" + SERVER_TWO, "file2"); - - // Wait the cleaner chore to run - Thread.sleep(20000); - - ReplicationZKNodeCleaner cleaner = new ReplicationZKNodeCleaner(conf, zkw, null); - assertEquals(0, cleaner.getUnDeletedQueues().size()); - } -} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureScheduler.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureScheduler.java index 0291165..fd77e1f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureScheduler.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureScheduler.java @@ -905,6 +905,27 @@ public class TestMasterProcedureScheduler { } } + public static class TestPeerProcedure extends TestProcedure implements PeerProcedureInterface { + private final String peerId; + private final PeerOperationType opType; + + public TestPeerProcedure(long procId, String peerId, PeerOperationType opType) { + super(procId); + this.peerId = peerId; + this.opType = opType; + } + + @Override + public String getPeerId() { + return peerId; + } + + @Override + public PeerOperationType getPeerOperationType() { + return opType; + } + } + private static LockProcedure createLockProcedure(LockType lockType, long procId) throws Exception { LockProcedure procedure = new LockProcedure(); @@ -927,22 +948,19 @@ public class TestMasterProcedureScheduler { return createLockProcedure(LockType.SHARED, procId); } - private static void assertLockResource(LockedResource resource, - LockedResourceType resourceType, String resourceName) - { + private static void assertLockResource(LockedResource resource, LockedResourceType resourceType, + String resourceName) { assertEquals(resourceType, resource.getResourceType()); assertEquals(resourceName, resource.getResourceName()); } - private static void assertExclusiveLock(LockedResource resource, Procedure procedure) - { + private static void assertExclusiveLock(LockedResource resource, Procedure procedure) { assertEquals(LockType.EXCLUSIVE, resource.getLockType()); assertEquals(procedure, resource.getExclusiveLockOwnerProcedure()); assertEquals(0, resource.getSharedLockCount()); } - private static void assertSharedLock(LockedResource resource, int lockCount) - { + private static void assertSharedLock(LockedResource resource, int lockCount) { assertEquals(LockType.SHARED, resource.getLockType()); assertEquals(lockCount, resource.getSharedLockCount()); } @@ -1027,6 +1045,39 @@ public class TestMasterProcedureScheduler { } @Test + public void testListLocksPeer() throws Exception { + String peerId = "1"; + LockProcedure procedure = createExclusiveLockProcedure(4); + queue.waitPeerExclusiveLock(procedure, peerId); + + List locks = queue.getLocks(); + assertEquals(1, locks.size()); + + LockedResource resource = locks.get(0); + assertLockResource(resource, LockedResourceType.PEER, peerId); + assertExclusiveLock(resource, procedure); + assertTrue(resource.getWaitingProcedures().isEmpty()); + + // Try to acquire the exclusive lock again with same procedure + assertFalse(queue.waitPeerExclusiveLock(procedure, peerId)); + + // Try to acquire the exclusive lock again with new procedure + LockProcedure procedure2 = createExclusiveLockProcedure(5); + assertTrue(queue.waitPeerExclusiveLock(procedure2, peerId)); + + // Same peerId, still only has 1 LockedResource + locks = queue.getLocks(); + assertEquals(1, locks.size()); + + resource = locks.get(0); + assertLockResource(resource, LockedResourceType.PEER, peerId); + // LockedResource owner still is the origin procedure + assertExclusiveLock(resource, procedure); + // The new procedure should in the waiting list + assertEquals(1, resource.getWaitingProcedures().size()); + } + + @Test public void testListLocksWaiting() throws Exception { LockProcedure procedure1 = createExclusiveLockProcedure(1); queue.waitTableExclusiveLock(procedure1, TableName.valueOf("ns4", "table4")); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureSchedulerConcurrency.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureSchedulerConcurrency.java index 2e8e52a..4e67a63 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureSchedulerConcurrency.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureSchedulerConcurrency.java @@ -26,6 +26,8 @@ import java.util.concurrent.atomic.AtomicInteger; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.master.procedure.PeerProcedureInterface.PeerOperationType; +import org.apache.hadoop.hbase.master.procedure.TestMasterProcedureScheduler.TestPeerProcedure; import org.apache.hadoop.hbase.master.procedure.TestMasterProcedureScheduler.TestTableProcedure; import org.apache.hadoop.hbase.procedure2.Procedure; import org.apache.hadoop.hbase.testclassification.MediumTests; @@ -63,6 +65,85 @@ public class TestMasterProcedureSchedulerConcurrency { queue.clear(); } + @Test + public void testConcurrentPeerOperations() throws Exception { + TestPeerProcedureSet procSet = new TestPeerProcedureSet(queue); + + int NUM_ITEMS = 10; + int NUM_PEERS = 5; + AtomicInteger opsCount = new AtomicInteger(0); + for (int i = 0; i < NUM_PEERS; ++i) { + String peerId = String.format("test-peer-%04d", i); + for (int j = 1; j < NUM_ITEMS; ++j) { + procSet.addBack(new TestPeerProcedure(i * 100 + j, peerId, PeerOperationType.ADD)); + opsCount.incrementAndGet(); + } + } + assertEquals(opsCount.get(), queue.size()); + + Thread[] threads = new Thread[NUM_PEERS * 2]; + HashSet concurrentPeers = new HashSet<>(); + ArrayList failures = new ArrayList<>(); + AtomicInteger concurrentCount = new AtomicInteger(0); + for (int i = 0; i < threads.length; ++i) { + threads[i] = new Thread() { + @Override + public void run() { + while (opsCount.get() > 0) { + try { + TestPeerProcedure proc = procSet.acquire(); + if (proc == null) { + queue.signalAll(); + if (opsCount.get() > 0) { + continue; + } + break; + } + + String peerId = proc.getPeerId(); + synchronized (concurrentPeers) { + assertTrue("unexpected concurrency on " + peerId, concurrentPeers.add(peerId)); + } + assertTrue(opsCount.decrementAndGet() >= 0); + + try { + long procId = proc.getProcId(); + int concurrent = concurrentCount.incrementAndGet(); + assertTrue("inc-concurrent="+ concurrent +" 1 <= concurrent <= "+ NUM_PEERS, + concurrent >= 1 && concurrent <= NUM_PEERS); + LOG.debug("[S] peerId="+ peerId +" procId="+ procId +" concurrent="+ concurrent); + Thread.sleep(2000); + concurrent = concurrentCount.decrementAndGet(); + LOG.debug("[E] peerId="+ peerId +" procId="+ procId +" concurrent="+ concurrent); + assertTrue("dec-concurrent=" + concurrent, concurrent < NUM_PEERS); + } finally { + synchronized (concurrentPeers) { + assertTrue(concurrentPeers.remove(peerId)); + } + procSet.release(proc); + } + } catch (Throwable e) { + LOG.error("Failed " + e.getMessage(), e); + synchronized (failures) { + failures.add(e.getMessage()); + } + } finally { + queue.signalAll(); + } + } + } + }; + threads[i].start(); + } + + for (int i = 0; i < threads.length; ++i) { + threads[i].join(); + } + assertTrue(failures.toString(), failures.isEmpty()); + assertEquals(0, opsCount.get()); + assertEquals(0, queue.size()); + } + /** * Verify that "write" operations for a single table are serialized, * but different tables can be executed in parallel. @@ -221,4 +302,58 @@ public class TestMasterProcedureSchedulerConcurrency { return ((TableProcedureInterface)proc).getTableOperationType(); } } + + public static class TestPeerProcedureSet { + private final MasterProcedureScheduler queue; + + public TestPeerProcedureSet(final MasterProcedureScheduler queue) { + this.queue = queue; + } + + public void addBack(TestPeerProcedure proc) { + queue.addBack(proc); + } + + public TestPeerProcedure acquire() { + TestPeerProcedure proc = null; + boolean waiting = true; + while (waiting && queue.size() > 0) { + proc = (TestPeerProcedure) queue.poll(100000000L); + if (proc == null) { + continue; + } + switch (proc.getPeerOperationType()) { + case ADD: + case REMOVE: + case ENABLE: + case DISABLE: + case UPDATE_CONFIG: + waiting = queue.waitPeerExclusiveLock(proc, proc.getPeerId()); + break; + case REFRESH: + waiting = false; + break; + default: + throw new UnsupportedOperationException(); + } + } + return proc; + } + + public void release(TestPeerProcedure proc) { + switch (proc.getPeerOperationType()) { + case ADD: + case REMOVE: + case ENABLE: + case DISABLE: + case UPDATE_CONFIG: + queue.wakePeerExclusiveLock(proc, proc.getPeerId()); + break; + case REFRESH: + break; + default: + throw new UnsupportedOperationException(); + } + } + } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java index 7ea79f9..ff20ddc 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -22,6 +21,7 @@ import java.io.IOException; import java.util.List; import java.util.UUID; import java.util.concurrent.atomic.AtomicBoolean; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -46,11 +46,12 @@ public class ReplicationSourceDummy implements ReplicationSourceInterface { MetricsSource metrics; WALFileLengthProvider walFileLengthProvider; AtomicBoolean startup = new AtomicBoolean(false); + @Override public void init(Configuration conf, FileSystem fs, ReplicationSourceManager manager, - ReplicationQueues rq, ReplicationPeers rp, Server server, String peerClusterId, - UUID clusterId, ReplicationEndpoint replicationEndpoint, - WALFileLengthProvider walFileLengthProvider, MetricsSource metrics) throws IOException { + ReplicationQueueStorage rq, ReplicationPeer rp, Server server, String peerClusterId, + UUID clusterId, WALFileLengthProvider walFileLengthProvider, MetricsSource metrics) + throws IOException { this.manager = manager; this.peerClusterId = peerClusterId; this.metrics = metrics; @@ -88,7 +89,7 @@ public class ReplicationSourceDummy implements ReplicationSourceInterface { } @Override - public String getPeerClusterZnode() { + public String getQueueId() { return peerClusterId; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java index c57d9bb..ca4369e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java @@ -21,9 +21,7 @@ package org.apache.hadoop.hbase.replication; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import java.io.IOException; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestNamespaceReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestNamespaceReplication.java index ed71123..40a955e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestNamespaceReplication.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestNamespaceReplication.java @@ -30,12 +30,11 @@ import java.util.List; import java.util.Map; import java.util.Set; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.Delete; @@ -43,6 +42,8 @@ import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; @@ -68,9 +69,6 @@ public class TestNamespaceReplication extends TestReplicationBase { private static final byte[] val = Bytes.toBytes("myval"); - private static HTableDescriptor tabA; - private static HTableDescriptor tabB; - private static Connection connection1; private static Connection connection2; private static Admin admin1; @@ -90,23 +88,21 @@ public class TestNamespaceReplication extends TestReplicationBase { admin2.createNamespace(NamespaceDescriptor.create(ns1).build()); admin2.createNamespace(NamespaceDescriptor.create(ns2).build()); - tabA = new HTableDescriptor(tabAName); - HColumnDescriptor fam = new HColumnDescriptor(f1Name); - fam.setScope(HConstants.REPLICATION_SCOPE_GLOBAL); - tabA.addFamily(fam); - fam = new HColumnDescriptor(f2Name); - fam.setScope(HConstants.REPLICATION_SCOPE_GLOBAL); - tabA.addFamily(fam); + TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tabAName); + builder.addColumnFamily(ColumnFamilyDescriptorBuilder + .newBuilder(f1Name).setScope(HConstants.REPLICATION_SCOPE_GLOBAL).build()); + builder.addColumnFamily(ColumnFamilyDescriptorBuilder + .newBuilder(f2Name).setScope(HConstants.REPLICATION_SCOPE_GLOBAL).build()); + TableDescriptor tabA = builder.build(); admin1.createTable(tabA); admin2.createTable(tabA); - tabB = new HTableDescriptor(tabBName); - fam = new HColumnDescriptor(f1Name); - fam.setScope(HConstants.REPLICATION_SCOPE_GLOBAL); - tabB.addFamily(fam); - fam = new HColumnDescriptor(f2Name); - fam.setScope(HConstants.REPLICATION_SCOPE_GLOBAL); - tabB.addFamily(fam); + builder = TableDescriptorBuilder.newBuilder(tabBName); + builder.addColumnFamily(ColumnFamilyDescriptorBuilder + .newBuilder(f1Name).setScope(HConstants.REPLICATION_SCOPE_GLOBAL).build()); + builder.addColumnFamily(ColumnFamilyDescriptorBuilder + .newBuilder(f2Name).setScope(HConstants.REPLICATION_SCOPE_GLOBAL).build()); + TableDescriptor tabB = builder.build(); admin1.createTable(tabB); admin2.createTable(tabB); } @@ -134,22 +130,24 @@ public class TestNamespaceReplication extends TestReplicationBase { @Test public void testNamespaceReplication() throws Exception { + String peerId = "2"; + Table htab1A = connection1.getTable(tabAName); Table htab2A = connection2.getTable(tabAName); Table htab1B = connection1.getTable(tabBName); Table htab2B = connection2.getTable(tabBName); - ReplicationPeerConfig rpc = admin.getPeerConfig("2"); - rpc.setReplicateAllUserTables(false); - admin.updatePeerConfig("2", rpc); + ReplicationPeerConfig rpc = admin1.getReplicationPeerConfig(peerId); + admin1.updateReplicationPeerConfig(peerId, + ReplicationPeerConfig.newBuilder(rpc).setReplicateAllUserTables(false).build()); // add ns1 to peer config which replicate to cluster2 - rpc = admin.getPeerConfig("2"); + rpc = admin1.getReplicationPeerConfig(peerId); Set namespaces = new HashSet<>(); namespaces.add(ns1); - rpc.setNamespaces(namespaces); - admin.updatePeerConfig("2", rpc); + admin1.updateReplicationPeerConfig(peerId, + ReplicationPeerConfig.newBuilder(rpc).setNamespaces(namespaces).build()); LOG.info("update peer config"); // Table A can be replicated to cluster2 @@ -163,15 +161,14 @@ public class TestNamespaceReplication extends TestReplicationBase { ensureRowNotExisted(htab2B, row, f1Name, f2Name); // add ns1:TA => 'f1' and ns2 to peer config which replicate to cluster2 - rpc = admin.getPeerConfig("2"); + rpc = admin1.getReplicationPeerConfig(peerId); namespaces = new HashSet<>(); namespaces.add(ns2); - rpc.setNamespaces(namespaces); Map> tableCfs = new HashMap<>(); tableCfs.put(tabAName, new ArrayList<>()); tableCfs.get(tabAName).add("f1"); - rpc.setTableCFsMap(tableCfs); - admin.updatePeerConfig("2", rpc); + admin1.updateReplicationPeerConfig(peerId, ReplicationPeerConfig.newBuilder(rpc) + .setNamespaces(namespaces).setTableCFsMap(tableCfs).build()); LOG.info("update peer config"); // Only family f1 of Table A can replicated to cluster2 @@ -186,7 +183,7 @@ public class TestNamespaceReplication extends TestReplicationBase { delete(htab1B, row, f1Name, f2Name); ensureRowNotExisted(htab2B, row, f1Name, f2Name); - admin.removePeer("2"); + admin1.removeReplicationPeer(peerId); } private void put(Table source, byte[] row, byte[]... families) diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationDisableInactivePeer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationDisableInactivePeer.java index 1675496..24bb4d0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationDisableInactivePeer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationDisableInactivePeer.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -47,9 +46,6 @@ public class TestReplicationDisableInactivePeer extends TestReplicationBase { */ @Test(timeout = 600000) public void testDisableInactivePeer() throws Exception { - - // enabling and shutdown the peer - admin.enablePeer("2"); utility2.shutdownMiniHBaseCluster(); byte[] rowkey = Bytes.toBytes("disable inactive peer"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationProcedureRetry.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationProcedureRetry.java new file mode 100644 index 0000000..ab35b46 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationProcedureRetry.java @@ -0,0 +1,200 @@ +/** + * 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; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyBoolean; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.spy; + +import java.io.IOException; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.master.HMaster; +import org.apache.hadoop.hbase.master.replication.ReplicationPeerManager; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.testclassification.ReplicationTests; +import org.apache.zookeeper.KeeperException; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.mockito.invocation.InvocationOnMock; + +/** + * All the modification method will fail once in the test and should finally succeed. + */ +@Category({ ReplicationTests.class, MediumTests.class }) +public class TestReplicationProcedureRetry { + + private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); + + @BeforeClass + public static void setUp() throws Exception { + UTIL.getConfiguration().setClass(HConstants.MASTER_IMPL, MockHMaster.class, HMaster.class); + UTIL.startMiniCluster(3); + } + + @AfterClass + public static void tearDown() throws Exception { + UTIL.shutdownMiniCluster(); + } + + @After + public void tearDownAfterTest() throws IOException { + for (ReplicationPeerDescription desc : UTIL.getAdmin().listReplicationPeers()) { + UTIL.getAdmin().removeReplicationPeer(desc.getPeerId()); + } + } + + private void doTest() throws IOException { + Admin admin = UTIL.getAdmin(); + String peerId = "1"; + ReplicationPeerConfig peerConfig = ReplicationPeerConfig.newBuilder() + .setClusterKey("localhost:" + UTIL.getZkCluster().getClientPort() + ":/hbase2").build(); + admin.addReplicationPeer(peerId, peerConfig, true); + + assertEquals(peerConfig.getClusterKey(), + admin.getReplicationPeerConfig(peerId).getClusterKey()); + ReplicationPeerConfig newPeerConfig = + ReplicationPeerConfig.newBuilder(peerConfig).setBandwidth(123456).build(); + admin.updateReplicationPeerConfig(peerId, newPeerConfig); + assertEquals(newPeerConfig.getBandwidth(), + admin.getReplicationPeerConfig(peerId).getBandwidth()); + + admin.disableReplicationPeer(peerId); + assertFalse(admin.listReplicationPeers().get(0).isEnabled()); + + admin.enableReplicationPeer(peerId); + assertTrue(admin.listReplicationPeers().get(0).isEnabled()); + + admin.removeReplicationPeer(peerId); + assertTrue(admin.listReplicationPeers().isEmpty()); + + // make sure that we have run into the mocked method + MockHMaster master = (MockHMaster) UTIL.getHBaseCluster().getMaster(); + assertTrue(master.addPeerCalled); + assertTrue(master.removePeerCalled); + assertTrue(master.updatePeerConfigCalled); + assertTrue(master.enablePeerCalled); + assertTrue(master.disablePeerCalled); + } + + @Test + public void testErrorBeforeUpdate() throws IOException, ReplicationException { + ((MockHMaster) UTIL.getHBaseCluster().getMaster()).reset(true); + doTest(); + } + + @Test + public void testErrorAfterUpdate() throws IOException, ReplicationException { + ((MockHMaster) UTIL.getHBaseCluster().getMaster()).reset(false); + doTest(); + } + + public static final class MockHMaster extends HMaster { + + volatile boolean addPeerCalled; + + volatile boolean removePeerCalled; + + volatile boolean updatePeerConfigCalled; + + volatile boolean enablePeerCalled; + + volatile boolean disablePeerCalled; + + private ReplicationPeerManager manager; + + public MockHMaster(Configuration conf) throws IOException, KeeperException { + super(conf); + } + + private Object invokeWithError(InvocationOnMock invocation, boolean errorBeforeUpdate) + throws Throwable { + if (errorBeforeUpdate) { + throw new ReplicationException("mock error before update"); + } + invocation.callRealMethod(); + throw new ReplicationException("mock error after update"); + } + + public void reset(boolean errorBeforeUpdate) throws ReplicationException { + addPeerCalled = false; + removePeerCalled = false; + updatePeerConfigCalled = false; + enablePeerCalled = false; + disablePeerCalled = false; + ReplicationPeerManager m = super.getReplicationPeerManager(); + manager = spy(m); + doAnswer(invocation -> { + if (!addPeerCalled) { + addPeerCalled = true; + return invokeWithError(invocation, errorBeforeUpdate); + } else { + return invocation.callRealMethod(); + } + }).when(manager).addPeer(anyString(), any(ReplicationPeerConfig.class), anyBoolean()); + doAnswer(invocation -> { + if (!removePeerCalled) { + removePeerCalled = true; + return invokeWithError(invocation, errorBeforeUpdate); + } else { + return invocation.callRealMethod(); + } + }).when(manager).removePeer(anyString()); + doAnswer(invocation -> { + if (!updatePeerConfigCalled) { + updatePeerConfigCalled = true; + return invokeWithError(invocation, errorBeforeUpdate); + } else { + return invocation.callRealMethod(); + } + }).when(manager).updatePeerConfig(anyString(), any(ReplicationPeerConfig.class)); + doAnswer(invocation -> { + if (!enablePeerCalled) { + enablePeerCalled = true; + return invokeWithError(invocation, errorBeforeUpdate); + } else { + return invocation.callRealMethod(); + } + }).when(manager).enablePeer(anyString()); + doAnswer(invocation -> { + if (!disablePeerCalled) { + disablePeerCalled = true; + return invokeWithError(invocation, errorBeforeUpdate); + } else { + return invocation.callRealMethod(); + } + }).when(manager).disablePeer(anyString()); + } + + @Override + public ReplicationPeerManager getReplicationPeerManager() { + return manager; + } + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java index 48d8924..2194000 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSource.java @@ -32,28 +32,28 @@ import java.util.concurrent.atomic.AtomicLong; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.MiniHBaseCluster; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.Waiter; -import org.apache.hadoop.hbase.Waiter.Predicate; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.MiniHBaseCluster; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.Waiter; +import org.apache.hadoop.hbase.Waiter.Predicate; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.regionserver.HRegionServer; -import org.apache.hadoop.hbase.replication.regionserver.Replication; -import org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceManager; -import org.apache.hadoop.hbase.wal.WAL; -import org.apache.hadoop.hbase.wal.WALKeyImpl; -import org.apache.hadoop.hbase.wal.WALProvider; -import org.apache.hadoop.hbase.wal.WALFactory; -import org.apache.hadoop.hbase.wal.WALEdit; import org.apache.hadoop.hbase.replication.regionserver.HBaseInterClusterReplicationEndpoint; +import org.apache.hadoop.hbase.replication.regionserver.Replication; import org.apache.hadoop.hbase.replication.regionserver.ReplicationSource; +import org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceManager; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.ReplicationTests; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.wal.WAL; +import org.apache.hadoop.hbase.wal.WALEdit; +import org.apache.hadoop.hbase.wal.WALFactory; +import org.apache.hadoop.hbase.wal.WALKeyImpl; +import org.apache.hadoop.hbase.wal.WALProvider; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; @@ -159,15 +159,14 @@ public class TestReplicationSource { } }; replicationEndpoint.start(); - ReplicationPeers mockPeers = Mockito.mock(ReplicationPeers.class); ReplicationPeer mockPeer = Mockito.mock(ReplicationPeer.class); Mockito.when(mockPeer.getPeerBandwidth()).thenReturn(0L); Configuration testConf = HBaseConfiguration.create(); testConf.setInt("replication.source.maxretriesmultiplier", 1); ReplicationSourceManager manager = Mockito.mock(ReplicationSourceManager.class); Mockito.when(manager.getTotalBufferUsed()).thenReturn(new AtomicLong()); - source.init(testConf, null, manager, null, mockPeers, null, "testPeer", null, - replicationEndpoint, p -> OptionalLong.empty(), null); + source.init(testConf, null, manager, null, mockPeer, null, "testPeer", null, + p -> OptionalLong.empty(), null); ExecutorService executor = Executors.newSingleThreadExecutor(); Future future = executor.submit(new Runnable() { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java deleted file mode 100644 index 29c0930..0000000 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java +++ /dev/null @@ -1,378 +0,0 @@ -/** - * 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; - -import static org.junit.Assert.*; - -import java.util.ArrayList; -import java.util.List; - -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.ServerName; -import org.apache.hadoop.hbase.util.Pair; -import org.apache.hadoop.hbase.zookeeper.ZKConfig; -import org.apache.zookeeper.KeeperException; -import org.junit.Before; -import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * White box testing for replication state interfaces. Implementations should extend this class, and - * initialize the interfaces properly. - */ -public abstract class TestReplicationStateBasic { - - protected ReplicationQueues rq1; - protected ReplicationQueues rq2; - protected ReplicationQueues rq3; - protected ReplicationQueuesClient rqc; - protected String server1 = ServerName.valueOf("hostname1.example.org", 1234, -1L).toString(); - protected String server2 = ServerName.valueOf("hostname2.example.org", 1234, -1L).toString(); - protected String server3 = ServerName.valueOf("hostname3.example.org", 1234, -1L).toString(); - protected ReplicationPeers rp; - protected static final String ID_ONE = "1"; - protected static final String ID_TWO = "2"; - protected static String KEY_ONE; - protected static String KEY_TWO; - - // For testing when we try to replicate to ourself - protected String OUR_ID = "3"; - protected String OUR_KEY; - - protected static int zkTimeoutCount; - protected static final int ZK_MAX_COUNT = 300; - protected static final int ZK_SLEEP_INTERVAL = 100; // millis - - private static final Logger LOG = LoggerFactory.getLogger(TestReplicationStateBasic.class); - - @Before - public void setUp() { - zkTimeoutCount = 0; - } - - @Test - public void testReplicationQueuesClient() throws ReplicationException, KeeperException { - rqc.init(); - // Test methods with empty state - assertEquals(0, rqc.getListOfReplicators().size()); - assertNull(rqc.getLogsInQueue(server1, "qId1")); - assertNull(rqc.getAllQueues(server1)); - - /* - * Set up data Two replicators: -- server1: three queues with 0, 1 and 2 log files each -- - * server2: zero queues - */ - rq1.init(server1); - rq2.init(server2); - rq1.addLog("qId1", "trash"); - rq1.removeLog("qId1", "trash"); - rq1.addLog("qId2", "filename1"); - rq1.addLog("qId3", "filename2"); - rq1.addLog("qId3", "filename3"); - rq2.addLog("trash", "trash"); - rq2.removeQueue("trash"); - - List reps = rqc.getListOfReplicators(); - assertEquals(2, reps.size()); - assertTrue(server1, reps.contains(server1)); - assertTrue(server2, reps.contains(server2)); - - assertNull(rqc.getLogsInQueue("bogus", "bogus")); - assertNull(rqc.getLogsInQueue(server1, "bogus")); - assertEquals(0, rqc.getLogsInQueue(server1, "qId1").size()); - assertEquals(1, rqc.getLogsInQueue(server1, "qId2").size()); - assertEquals("filename1", rqc.getLogsInQueue(server1, "qId2").get(0)); - - assertNull(rqc.getAllQueues("bogus")); - assertEquals(0, rqc.getAllQueues(server2).size()); - List list = rqc.getAllQueues(server1); - assertEquals(3, list.size()); - assertTrue(list.contains("qId2")); - assertTrue(list.contains("qId3")); - } - - @Test - public void testReplicationQueues() throws ReplicationException { - rq1.init(server1); - rq2.init(server2); - rq3.init(server3); - //Initialize ReplicationPeer so we can add peers (we don't transfer lone queues) - rp.init(); - - // 3 replicators should exist - assertEquals(3, rq1.getListOfReplicators().size()); - rq1.removeQueue("bogus"); - rq1.removeLog("bogus", "bogus"); - rq1.removeAllQueues(); - assertEquals(0, rq1.getAllQueues().size()); - assertEquals(0, rq1.getLogPosition("bogus", "bogus")); - assertNull(rq1.getLogsInQueue("bogus")); - assertNull(rq1.getUnClaimedQueueIds( - ServerName.valueOf("bogus", 1234, -1L).toString())); - - rq1.setLogPosition("bogus", "bogus", 5L); - - populateQueues(); - - assertEquals(3, rq1.getListOfReplicators().size()); - assertEquals(0, rq2.getLogsInQueue("qId1").size()); - assertEquals(5, rq3.getLogsInQueue("qId5").size()); - assertEquals(0, rq3.getLogPosition("qId1", "filename0")); - rq3.setLogPosition("qId5", "filename4", 354L); - assertEquals(354L, rq3.getLogPosition("qId5", "filename4")); - - assertEquals(5, rq3.getLogsInQueue("qId5").size()); - assertEquals(0, rq2.getLogsInQueue("qId1").size()); - assertEquals(0, rq1.getAllQueues().size()); - assertEquals(1, rq2.getAllQueues().size()); - assertEquals(5, rq3.getAllQueues().size()); - - assertEquals(0, rq3.getUnClaimedQueueIds(server1).size()); - rq3.removeReplicatorIfQueueIsEmpty(server1); - assertEquals(2, rq3.getListOfReplicators().size()); - - List queues = rq2.getUnClaimedQueueIds(server3); - assertEquals(5, queues.size()); - for(String queue: queues) { - rq2.claimQueue(server3, queue); - } - rq2.removeReplicatorIfQueueIsEmpty(server3); - assertEquals(1, rq2.getListOfReplicators().size()); - - // Try to claim our own queues - assertNull(rq2.getUnClaimedQueueIds(server2)); - rq2.removeReplicatorIfQueueIsEmpty(server2); - - assertEquals(6, rq2.getAllQueues().size()); - - rq2.removeAllQueues(); - - assertEquals(0, rq2.getListOfReplicators().size()); - } - - @Test - public void testInvalidClusterKeys() throws ReplicationException, KeeperException { - rp.init(); - - try { - rp.registerPeer(ID_ONE, - new ReplicationPeerConfig().setClusterKey("hostname1.example.org:1234:hbase")); - fail("Should throw an IllegalArgumentException because " - + "zookeeper.znode.parent is missing leading '/'."); - } catch (IllegalArgumentException e) { - // Expected. - } - - try { - rp.registerPeer(ID_ONE, - new ReplicationPeerConfig().setClusterKey("hostname1.example.org:1234:/")); - fail("Should throw an IllegalArgumentException because zookeeper.znode.parent is missing."); - } catch (IllegalArgumentException e) { - // Expected. - } - - try { - rp.registerPeer(ID_ONE, - new ReplicationPeerConfig().setClusterKey("hostname1.example.org::/hbase")); - fail("Should throw an IllegalArgumentException because " - + "hbase.zookeeper.property.clientPort is missing."); - } catch (IllegalArgumentException e) { - // Expected. - } - } - - @Test - public void testHfileRefsReplicationQueues() throws ReplicationException, KeeperException { - rp.init(); - rq1.init(server1); - rqc.init(); - - List> files1 = new ArrayList<>(3); - files1.add(new Pair<>(null, new Path("file_1"))); - files1.add(new Pair<>(null, new Path("file_2"))); - files1.add(new Pair<>(null, new Path("file_3"))); - assertNull(rqc.getReplicableHFiles(ID_ONE)); - assertEquals(0, rqc.getAllPeersFromHFileRefsQueue().size()); - rp.registerPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE)); - rq1.addPeerToHFileRefs(ID_ONE); - rq1.addHFileRefs(ID_ONE, files1); - assertEquals(1, rqc.getAllPeersFromHFileRefsQueue().size()); - assertEquals(3, rqc.getReplicableHFiles(ID_ONE).size()); - List hfiles2 = new ArrayList<>(files1.size()); - for (Pair p : files1) { - hfiles2.add(p.getSecond().getName()); - } - String removedString = hfiles2.remove(0); - rq1.removeHFileRefs(ID_ONE, hfiles2); - assertEquals(1, rqc.getReplicableHFiles(ID_ONE).size()); - hfiles2 = new ArrayList<>(1); - hfiles2.add(removedString); - rq1.removeHFileRefs(ID_ONE, hfiles2); - assertEquals(0, rqc.getReplicableHFiles(ID_ONE).size()); - rp.unregisterPeer(ID_ONE); - } - - @Test - public void testRemovePeerForHFileRefs() throws ReplicationException, KeeperException { - rq1.init(server1); - rqc.init(); - - rp.init(); - rp.registerPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE)); - rq1.addPeerToHFileRefs(ID_ONE); - rp.registerPeer(ID_TWO, new ReplicationPeerConfig().setClusterKey(KEY_TWO)); - rq1.addPeerToHFileRefs(ID_TWO); - - List> files1 = new ArrayList<>(3); - files1.add(new Pair<>(null, new Path("file_1"))); - files1.add(new Pair<>(null, new Path("file_2"))); - files1.add(new Pair<>(null, new Path("file_3"))); - rq1.addHFileRefs(ID_ONE, files1); - rq1.addHFileRefs(ID_TWO, files1); - assertEquals(2, rqc.getAllPeersFromHFileRefsQueue().size()); - assertEquals(3, rqc.getReplicableHFiles(ID_ONE).size()); - assertEquals(3, rqc.getReplicableHFiles(ID_TWO).size()); - - rp.unregisterPeer(ID_ONE); - rq1.removePeerFromHFileRefs(ID_ONE); - assertEquals(1, rqc.getAllPeersFromHFileRefsQueue().size()); - assertNull(rqc.getReplicableHFiles(ID_ONE)); - assertEquals(3, rqc.getReplicableHFiles(ID_TWO).size()); - - rp.unregisterPeer(ID_TWO); - rq1.removePeerFromHFileRefs(ID_TWO); - assertEquals(0, rqc.getAllPeersFromHFileRefsQueue().size()); - assertNull(rqc.getReplicableHFiles(ID_TWO)); - } - - @Test - public void testReplicationPeers() throws Exception { - rp.init(); - - // Test methods with non-existent peer ids - try { - rp.unregisterPeer("bogus"); - fail("Should have thrown an IllegalArgumentException when passed a bogus peerId"); - } catch (IllegalArgumentException e) { - } - try { - rp.enablePeer("bogus"); - fail("Should have thrown an IllegalArgumentException when passed a bogus peerId"); - } catch (IllegalArgumentException e) { - } - try { - rp.disablePeer("bogus"); - fail("Should have thrown an IllegalArgumentException when passed a bogus peerId"); - } catch (IllegalArgumentException e) { - } - try { - rp.getStatusOfPeer("bogus"); - fail("Should have thrown an IllegalArgumentException when passed a bogus peerId"); - } catch (IllegalArgumentException e) { - } - assertFalse(rp.peerConnected("bogus")); - rp.peerDisconnected("bogus"); - - assertNull(rp.getPeerConf("bogus")); - assertNumberOfPeers(0); - - // Add some peers - rp.registerPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE)); - assertNumberOfPeers(1); - rp.registerPeer(ID_TWO, new ReplicationPeerConfig().setClusterKey(KEY_TWO)); - assertNumberOfPeers(2); - - // Test methods with a peer that is added but not connected - try { - rp.getStatusOfPeer(ID_ONE); - fail("There are no connected peers, should have thrown an IllegalArgumentException"); - } catch (IllegalArgumentException e) { - } - assertEquals(KEY_ONE, ZKConfig.getZooKeeperClusterKey(rp.getPeerConf(ID_ONE).getSecond())); - rp.unregisterPeer(ID_ONE); - rp.peerDisconnected(ID_ONE); - assertNumberOfPeers(1); - - // Add one peer - rp.registerPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE)); - rp.peerConnected(ID_ONE); - assertNumberOfPeers(2); - assertTrue(rp.getStatusOfPeer(ID_ONE)); - rp.disablePeer(ID_ONE); - assertConnectedPeerStatus(false, ID_ONE); - rp.enablePeer(ID_ONE); - assertConnectedPeerStatus(true, ID_ONE); - - // Disconnect peer - rp.peerDisconnected(ID_ONE); - assertNumberOfPeers(2); - try { - rp.getStatusOfPeer(ID_ONE); - fail("There are no connected peers, should have thrown an IllegalArgumentException"); - } catch (IllegalArgumentException e) { - } - } - - protected void assertConnectedPeerStatus(boolean status, String peerId) throws Exception { - // we can first check if the value was changed in the store, if it wasn't then fail right away - if (status != rp.getStatusOfPeerFromBackingStore(peerId)) { - fail("ConnectedPeerStatus was " + !status + " but expected " + status + " in ZK"); - } - while (true) { - if (status == rp.getStatusOfPeer(peerId)) { - return; - } - if (zkTimeoutCount < ZK_MAX_COUNT) { - LOG.debug("ConnectedPeerStatus was " + !status + " but expected " + status - + ", sleeping and trying again."); - Thread.sleep(ZK_SLEEP_INTERVAL); - } else { - fail("Timed out waiting for ConnectedPeerStatus to be " + status); - } - } - } - - protected void assertNumberOfPeers(int total) { - assertEquals(total, rp.getAllPeerConfigs().size()); - assertEquals(total, rp.getAllPeerIds().size()); - assertEquals(total, rp.getAllPeerIds().size()); - } - - /* - * three replicators: rq1 has 0 queues, rq2 has 1 queue with no logs, rq3 has 5 queues with 1, 2, - * 3, 4, 5 log files respectively - */ - protected void populateQueues() throws ReplicationException { - rq1.addLog("trash", "trash"); - rq1.removeQueue("trash"); - - rq2.addLog("qId1", "trash"); - rq2.removeLog("qId1", "trash"); - - for (int i = 1; i < 6; i++) { - for (int j = 0; j < i; j++) { - rq3.addLog("qId" + i, "filename" + j); - } - //Add peers for the corresponding queues so they are not orphans - rp.registerPeer("qId" + i, new ReplicationPeerConfig().setClusterKey("localhost:2818:/bogus" + i)); - } - } -} - diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java deleted file mode 100644 index 231d655..0000000 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java +++ /dev/null @@ -1,227 +0,0 @@ -/** - * 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; - -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - -import java.io.IOException; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.hbase.ChoreService; -import org.apache.hadoop.hbase.ClusterId; -import org.apache.hadoop.hbase.CoordinatedStateManager; -import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.Server; -import org.apache.hadoop.hbase.ServerName; -import org.apache.hadoop.hbase.client.ClusterConnection; -import org.apache.hadoop.hbase.client.Connection; -import org.apache.hadoop.hbase.testclassification.MediumTests; -import org.apache.hadoop.hbase.testclassification.ReplicationTests; -import org.apache.hadoop.hbase.zookeeper.MetaTableLocator; -import org.apache.hadoop.hbase.zookeeper.ZKClusterId; -import org.apache.hadoop.hbase.zookeeper.ZKConfig; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; -import org.apache.hadoop.hbase.zookeeper.ZKWatcher; -import org.apache.hadoop.hbase.zookeeper.ZNodePaths; -import org.apache.zookeeper.KeeperException; -import org.junit.After; -import org.junit.AfterClass; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -@Category({ReplicationTests.class, MediumTests.class}) -public class TestReplicationStateZKImpl extends TestReplicationStateBasic { - - private static final Logger LOG = LoggerFactory.getLogger(TestReplicationStateZKImpl.class); - - private static Configuration conf; - private static HBaseTestingUtility utility; - private static ZKWatcher zkw; - private static String replicationZNode; - private ReplicationQueuesZKImpl rqZK; - - @BeforeClass - public static void setUpBeforeClass() throws Exception { - utility = new HBaseTestingUtility(); - utility.startMiniZKCluster(); - conf = utility.getConfiguration(); - conf.setBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY, true); - zkw = HBaseTestingUtility.getZooKeeperWatcher(utility); - String replicationZNodeName = conf.get("zookeeper.znode.replication", "replication"); - replicationZNode = ZNodePaths.joinZNode(zkw.znodePaths.baseZNode, replicationZNodeName); - KEY_ONE = initPeerClusterState("/hbase1"); - KEY_TWO = initPeerClusterState("/hbase2"); - } - - private static String initPeerClusterState(String baseZKNode) - throws IOException, KeeperException { - // Add a dummy region server and set up the cluster id - Configuration testConf = new Configuration(conf); - testConf.set(HConstants.ZOOKEEPER_ZNODE_PARENT, baseZKNode); - ZKWatcher zkw1 = new ZKWatcher(testConf, "test1", null); - String fakeRs = ZNodePaths.joinZNode(zkw1.znodePaths.rsZNode, "hostname1.example.org:1234"); - ZKUtil.createWithParents(zkw1, fakeRs); - ZKClusterId.setClusterId(zkw1, new ClusterId()); - return ZKConfig.getZooKeeperClusterKey(testConf); - } - - @Before - @Override - public void setUp() { - super.setUp(); - DummyServer ds1 = new DummyServer(server1); - DummyServer ds2 = new DummyServer(server2); - DummyServer ds3 = new DummyServer(server3); - try { - rq1 = ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(conf, ds1, zkw)); - rq2 = ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(conf, ds2, zkw)); - rq3 = ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(conf, ds3, zkw)); - rqc = ReplicationFactory.getReplicationQueuesClient( - new ReplicationQueuesClientArguments(conf, ds1, zkw)); - } catch (Exception e) { - // This should not occur, because getReplicationQueues() only throws for - // TableBasedReplicationQueuesImpl - fail("ReplicationFactory.getReplicationQueues() threw an IO Exception"); - } - rp = ReplicationFactory.getReplicationPeers(zkw, conf, zkw); - OUR_KEY = ZKConfig.getZooKeeperClusterKey(conf); - rqZK = new ReplicationQueuesZKImpl(zkw, conf, ds1); - } - - @After - public void tearDown() throws KeeperException, IOException { - ZKUtil.deleteNodeRecursively(zkw, replicationZNode); - } - - @AfterClass - public static void tearDownAfterClass() throws Exception { - utility.shutdownMiniZKCluster(); - } - - @Test - public void testIsPeerPath_PathToParentOfPeerNode() { - assertFalse(rqZK.isPeerPath(rqZK.peersZNode)); - } - - @Test - public void testIsPeerPath_PathToChildOfPeerNode() { - String peerChild = ZNodePaths.joinZNode(ZNodePaths.joinZNode(rqZK.peersZNode, "1"), "child"); - assertFalse(rqZK.isPeerPath(peerChild)); - } - - @Test - public void testIsPeerPath_ActualPeerPath() { - String peerPath = ZNodePaths.joinZNode(rqZK.peersZNode, "1"); - assertTrue(rqZK.isPeerPath(peerPath)); - } - - static class DummyServer implements Server { - private String serverName; - private boolean isAborted = false; - private boolean isStopped = false; - - public DummyServer(String serverName) { - this.serverName = serverName; - } - - @Override - public Configuration getConfiguration() { - return conf; - } - - @Override - public ZKWatcher getZooKeeper() { - return zkw; - } - - @Override - public CoordinatedStateManager getCoordinatedStateManager() { - return null; - } - - @Override - public ClusterConnection getConnection() { - return null; - } - - @Override - public MetaTableLocator getMetaTableLocator() { - return null; - } - - @Override - public ServerName getServerName() { - return ServerName.valueOf(this.serverName); - } - - @Override - public void abort(String why, Throwable e) { - LOG.info("Aborting " + serverName); - this.isAborted = true; - } - - @Override - public boolean isAborted() { - return this.isAborted; - } - - @Override - public void stop(String why) { - this.isStopped = true; - } - - @Override - public boolean isStopped() { - return this.isStopped; - } - - @Override - public ChoreService getChoreService() { - return null; - } - - @Override - public ClusterConnection getClusterConnection() { - // TODO Auto-generated method stub - return null; - } - - @Override - public FileSystem getFileSystem() { - return null; - } - - @Override - public boolean isStopping() { - return false; - } - - @Override - public Connection createConnection(Configuration conf) throws IOException { - return null; - } - } -} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpTool.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpTool.java index 0a602ad..0313b3b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpTool.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpTool.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -64,7 +64,6 @@ public class TestReplicationSyncUpTool extends TestReplicationBase { @Before public void setUp() throws Exception { - HColumnDescriptor fam; t1_syncupSource = new HTableDescriptor(t1_su); @@ -100,7 +99,7 @@ public class TestReplicationSyncUpTool extends TestReplicationBase { * check's gone Also check the puts and deletes are not replicated back to * the originating cluster. */ - @Test(timeout = 300000) + @Test public void testSyncUpTool() throws Exception { /** @@ -176,7 +175,6 @@ public class TestReplicationSyncUpTool extends TestReplicationBase { * verify correctly replicated to Slave */ mimicSyncUpAfterPut(); - } protected void setupReplication() throws Exception { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java index a04d524..757d9a9 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java @@ -19,12 +19,9 @@ package org.apache.hadoop.hbase.replication; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import java.io.IOException; -import java.util.ArrayList; -import java.util.List; import java.util.concurrent.atomic.AtomicInteger; import org.apache.hadoop.conf.Configuration; @@ -45,6 +42,7 @@ import org.apache.hadoop.hbase.zookeeper.ZKClusterId; import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hbase.zookeeper.ZKWatcher; import org.apache.hadoop.hbase.zookeeper.ZNodePaths; +import org.apache.zookeeper.KeeperException; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; @@ -74,10 +72,6 @@ public class TestReplicationTrackerZKImpl { private ReplicationTracker rt; private AtomicInteger rsRemovedCount; private String rsRemovedData; - private AtomicInteger plChangedCount; - private List plChangedData; - private AtomicInteger peerRemovedCount; - private String peerRemovedData; @BeforeClass public static void setUpBeforeClass() throws Exception { @@ -94,18 +88,15 @@ public class TestReplicationTrackerZKImpl { String fakeRs1 = ZNodePaths.joinZNode(zkw.znodePaths.rsZNode, "hostname1.example.org:1234"); try { ZKClusterId.setClusterId(zkw, new ClusterId()); - rp = ReplicationFactory.getReplicationPeers(zkw, conf, zkw); + rp = ReplicationFactory.getReplicationPeers(zkw, conf); rp.init(); - rt = ReplicationFactory.getReplicationTracker(zkw, rp, conf, zkw, new DummyServer(fakeRs1)); + rt = ReplicationFactory.getReplicationTracker(zkw, new DummyServer(fakeRs1), + new DummyServer(fakeRs1)); } catch (Exception e) { fail("Exception during test setup: " + e); } rsRemovedCount = new AtomicInteger(0); rsRemovedData = ""; - plChangedCount = new AtomicInteger(0); - plChangedData = new ArrayList<>(); - peerRemovedCount = new AtomicInteger(0); - peerRemovedData = ""; } @AfterClass @@ -156,62 +147,24 @@ public class TestReplicationTrackerZKImpl { } @Test(timeout = 30000) - public void testPeerRemovedEvent() throws Exception { - rp.registerPeer("5", new ReplicationPeerConfig().setClusterKey(utility.getClusterKey())); - rt.registerListener(new DummyReplicationListener()); - rp.unregisterPeer("5"); - // wait for event - while (peerRemovedCount.get() < 1) { - Thread.sleep(5); - } - assertEquals("5", peerRemovedData); - } - - @Test(timeout = 30000) - public void testPeerListChangedEvent() throws Exception { - // add a peer - rp.registerPeer("5", new ReplicationPeerConfig().setClusterKey(utility.getClusterKey())); - zkw.getRecoverableZooKeeper().getZooKeeper().getChildren("/hbase/replication/peers/5", true); - rt.registerListener(new DummyReplicationListener()); - rp.disablePeer("5"); - int tmp = plChangedCount.get(); - LOG.info("Peer count=" + tmp); - ZKUtil.deleteNode(zkw, "/hbase/replication/peers/5/peer-state"); - // wait for event - while (plChangedCount.get() <= tmp) { - Thread.sleep(100); - LOG.info("Peer count=" + tmp); - } - assertEquals(1, plChangedData.size()); - assertTrue(plChangedData.contains("5")); - - // clean up - //ZKUtil.deleteNode(zkw, "/hbase/replication/peers/5"); - rp.unregisterPeer("5"); - } - - @Test(timeout = 30000) public void testPeerNameControl() throws Exception { int exists = 0; - int hyphen = 0; - rp.registerPeer("6", new ReplicationPeerConfig().setClusterKey(utility.getClusterKey())); + rp.getPeerStorage().addPeer("6", + ReplicationPeerConfig.newBuilder().setClusterKey(utility.getClusterKey()).build(), true); - try{ - rp.registerPeer("6", new ReplicationPeerConfig().setClusterKey(utility.getClusterKey())); - }catch(IllegalArgumentException e){ - exists++; + try { + rp.getPeerStorage().addPeer("6", + ReplicationPeerConfig.newBuilder().setClusterKey(utility.getClusterKey()).build(), true); + } catch (ReplicationException e) { + if (e.getCause() instanceof KeeperException.NodeExistsException) { + exists++; + } } - try{ - rp.registerPeer("6-ec2", new ReplicationPeerConfig().setClusterKey(utility.getClusterKey())); - }catch(IllegalArgumentException e){ - hyphen++; - } assertEquals(1, exists); - assertEquals(1, hyphen); // clean up - rp.unregisterPeer("6"); + rp.getPeerStorage().removePeer("6"); } private class DummyReplicationListener implements ReplicationListener { @@ -222,21 +175,6 @@ public class TestReplicationTrackerZKImpl { rsRemovedCount.getAndIncrement(); LOG.debug("Received regionServerRemoved event: " + regionServer); } - - @Override - public void peerRemoved(String peerId) { - peerRemovedData = peerId; - peerRemovedCount.getAndIncrement(); - LOG.debug("Received peerDisconnected event: " + peerId); - } - - @Override - public void peerListChanged(List peerIds) { - plChangedData.clear(); - plChangedData.addAll(peerIds); - int count = plChangedCount.getAndIncrement(); - LOG.debug("Received peerListChanged event " + count); - } } private class DummyServer implements Server { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/master/TestTableCFsUpdater.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/master/TestTableCFsUpdater.java index 2993043..19acc75 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/master/TestTableCFsUpdater.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/master/TestTableCFsUpdater.java @@ -25,14 +25,13 @@ import org.apache.hadoop.hbase.Abortable; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil; -import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; +import org.apache.hadoop.hbase.replication.ZKReplicationPeerStorage; import org.apache.hadoop.hbase.testclassification.ReplicationTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hbase.zookeeper.ZKWatcher; -import org.apache.zookeeper.KeeperException; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Rule; @@ -57,12 +56,19 @@ public class TestTableCFsUpdater extends ReplicationPeerConfigUpgrader { private static ZKWatcher zkw = null; private static Abortable abortable = null; + private static ZKStorageUtil zkStorageUtil = null; + + private static class ZKStorageUtil extends ZKReplicationPeerStorage { + public ZKStorageUtil(ZKWatcher zookeeper, Configuration conf) { + super(zookeeper, conf); + } + } @Rule public TestName name = new TestName(); public TestTableCFsUpdater() { - super(zkw, TEST_UTIL.getConfiguration(), abortable); + super(zkw, TEST_UTIL.getConfiguration()); } @BeforeClass @@ -81,6 +87,7 @@ public class TestTableCFsUpdater extends ReplicationPeerConfigUpgrader { } }; zkw = new ZKWatcher(conf, "TableCFs", abortable, true); + zkStorageUtil = new ZKStorageUtil(zkw, conf); } @AfterClass @@ -89,8 +96,7 @@ public class TestTableCFsUpdater extends ReplicationPeerConfigUpgrader { } @Test - public void testUpgrade() throws KeeperException, InterruptedException, - DeserializationException { + public void testUpgrade() throws Exception { String peerId = "1"; final TableName tableName1 = TableName.valueOf(name.getMethodName() + "1"); final TableName tableName2 = TableName.valueOf(name.getMethodName() + "2"); @@ -98,13 +104,13 @@ public class TestTableCFsUpdater extends ReplicationPeerConfigUpgrader { ReplicationPeerConfig rpc = new ReplicationPeerConfig(); rpc.setClusterKey(zkw.getQuorum()); - String peerNode = getPeerNode(peerId); + String peerNode = zkStorageUtil.getPeerNode(peerId); ZKUtil.createWithParents(zkw, peerNode, ReplicationPeerConfigUtil.toByteArray(rpc)); String tableCFs = tableName1 + ":cf1,cf2;" + tableName2 + ":cf3;" + tableName3; String tableCFsNode = getTableCFsNode(peerId); LOG.info("create tableCFs :" + tableCFsNode + " for peerId=" + peerId); - ZKUtil.createWithParents(zkw, tableCFsNode , Bytes.toBytes(tableCFs)); + ZKUtil.createWithParents(zkw, tableCFsNode, Bytes.toBytes(tableCFs)); ReplicationPeerConfig actualRpc = ReplicationPeerConfigUtil.parsePeerFrom(ZKUtil.getData(zkw, peerNode)); @@ -117,13 +123,13 @@ public class TestTableCFsUpdater extends ReplicationPeerConfigUpgrader { peerId = "2"; rpc = new ReplicationPeerConfig(); rpc.setClusterKey(zkw.getQuorum()); - peerNode = getPeerNode(peerId); + peerNode = zkStorageUtil.getPeerNode(peerId); ZKUtil.createWithParents(zkw, peerNode, ReplicationPeerConfigUtil.toByteArray(rpc)); tableCFs = tableName1 + ":cf1,cf3;" + tableName2 + ":cf2"; tableCFsNode = getTableCFsNode(peerId); LOG.info("create tableCFs :" + tableCFsNode + " for peerId=" + peerId); - ZKUtil.createWithParents(zkw, tableCFsNode , Bytes.toBytes(tableCFs)); + ZKUtil.createWithParents(zkw, tableCFsNode, Bytes.toBytes(tableCFs)); actualRpc = ReplicationPeerConfigUtil.parsePeerFrom(ZKUtil.getData(zkw, peerNode)); actualTableCfs = Bytes.toString(ZKUtil.getData(zkw, tableCFsNode)); @@ -135,13 +141,13 @@ public class TestTableCFsUpdater extends ReplicationPeerConfigUpgrader { peerId = "3"; rpc = new ReplicationPeerConfig(); rpc.setClusterKey(zkw.getQuorum()); - peerNode = getPeerNode(peerId); + peerNode = zkStorageUtil.getPeerNode(peerId); ZKUtil.createWithParents(zkw, peerNode, ReplicationPeerConfigUtil.toByteArray(rpc)); tableCFs = ""; tableCFsNode = getTableCFsNode(peerId); LOG.info("create tableCFs :" + tableCFsNode + " for peerId=" + peerId); - ZKUtil.createWithParents(zkw, tableCFsNode , Bytes.toBytes(tableCFs)); + ZKUtil.createWithParents(zkw, tableCFsNode, Bytes.toBytes(tableCFs)); actualRpc = ReplicationPeerConfigUtil.parsePeerFrom(ZKUtil.getData(zkw, peerNode)); actualTableCfs = Bytes.toString(ZKUtil.getData(zkw, tableCFsNode)); @@ -153,7 +159,7 @@ public class TestTableCFsUpdater extends ReplicationPeerConfigUpgrader { peerId = "4"; rpc = new ReplicationPeerConfig(); rpc.setClusterKey(zkw.getQuorum()); - peerNode = getPeerNode(peerId); + peerNode = zkStorageUtil.getPeerNode(peerId); ZKUtil.createWithParents(zkw, peerNode, ReplicationPeerConfigUtil.toByteArray(rpc)); tableCFsNode = getTableCFsNode(peerId); @@ -167,7 +173,7 @@ public class TestTableCFsUpdater extends ReplicationPeerConfigUpgrader { copyTableCFs(); peerId = "1"; - peerNode = getPeerNode(peerId); + peerNode = zkStorageUtil.getPeerNode(peerId); actualRpc = ReplicationPeerConfigUtil.parsePeerFrom(ZKUtil.getData(zkw, peerNode)); assertEquals(rpc.getClusterKey(), actualRpc.getClusterKey()); Map> tableNameListMap = actualRpc.getTableCFsMap(); @@ -182,9 +188,8 @@ public class TestTableCFsUpdater extends ReplicationPeerConfigUpgrader { assertEquals("cf3", tableNameListMap.get(tableName2).get(0)); assertNull(tableNameListMap.get(tableName3)); - peerId = "2"; - peerNode = getPeerNode(peerId); + peerNode = zkStorageUtil.getPeerNode(peerId); actualRpc = ReplicationPeerConfigUtil.parsePeerFrom(ZKUtil.getData(zkw, peerNode)); assertEquals(rpc.getClusterKey(), actualRpc.getClusterKey()); tableNameListMap = actualRpc.getTableCFsMap(); @@ -198,19 +203,17 @@ public class TestTableCFsUpdater extends ReplicationPeerConfigUpgrader { assertEquals("cf2", tableNameListMap.get(tableName2).get(0)); peerId = "3"; - peerNode = getPeerNode(peerId); + peerNode = zkStorageUtil.getPeerNode(peerId); actualRpc = ReplicationPeerConfigUtil.parsePeerFrom(ZKUtil.getData(zkw, peerNode)); assertEquals(rpc.getClusterKey(), actualRpc.getClusterKey()); tableNameListMap = actualRpc.getTableCFsMap(); assertNull(tableNameListMap); peerId = "4"; - peerNode = getPeerNode(peerId); + peerNode = zkStorageUtil.getPeerNode(peerId); actualRpc = ReplicationPeerConfigUtil.parsePeerFrom(ZKUtil.getData(zkw, peerNode)); assertEquals(rpc.getClusterKey(), actualRpc.getClusterKey()); tableNameListMap = actualRpc.getTableCFsMap(); assertNull(tableNameListMap); } - - } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java index 27e4cf1..3a49385 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java @@ -1,5 +1,4 @@ -/* - * +/** * 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 @@ -40,7 +39,6 @@ import java.util.TreeSet; import java.util.UUID; import java.util.concurrent.CountDownLatch; import java.util.stream.Collectors; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -63,14 +61,14 @@ import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl; import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener; -import org.apache.hadoop.hbase.replication.ReplicationEndpoint; import org.apache.hadoop.hbase.replication.ReplicationFactory; +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.ReplicationQueues; -import org.apache.hadoop.hbase.replication.ReplicationQueuesArguments; +import org.apache.hadoop.hbase.replication.ReplicationQueueStorage; import org.apache.hadoop.hbase.replication.ReplicationSourceDummy; -import org.apache.hadoop.hbase.replication.ReplicationStateZKBase; +import org.apache.hadoop.hbase.replication.ReplicationStorageFactory; +import org.apache.hadoop.hbase.replication.ZKReplicationPeerStorage; import org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceManager.NodeFailoverWorker; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.ReplicationTests; @@ -95,8 +93,10 @@ import org.junit.experimental.categories.Category; import org.junit.rules.TestName; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.hbase.thirdparty.com.google.common.collect.Sets; import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations; + import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.BulkLoadDescriptor; @@ -162,9 +162,9 @@ public abstract class TestReplicationSourceManager { + conf.get(HConstants.ZOOKEEPER_CLIENT_PORT) + ":/1")); ZKUtil.createWithParents(zkw, "/hbase/replication/peers/1/peer-state"); ZKUtil.setData(zkw, "/hbase/replication/peers/1/peer-state", - ReplicationStateZKBase.ENABLED_ZNODE_BYTES); + ZKReplicationPeerStorage.ENABLED_ZNODE_BYTES); ZKUtil.createWithParents(zkw, "/hbase/replication/state"); - ZKUtil.setData(zkw, "/hbase/replication/state", ReplicationStateZKBase.ENABLED_ZNODE_BYTES); + ZKUtil.setData(zkw, "/hbase/replication/state", ZKReplicationPeerStorage.ENABLED_ZNODE_BYTES); ZKClusterId.setClusterId(zkw, new ClusterId()); FSUtils.setRootDir(utility.getConfiguration(), utility.getDataTestDir()); @@ -176,6 +176,12 @@ public abstract class TestReplicationSourceManager { replication = new Replication(new DummyServer(), fs, logDir, oldLogDir); managerOfCluster = getManagerFromCluster(); + if (managerOfCluster != null) { + // After replication procedure, we need to add peer by hand (other than by receiving + // notification from zk) + managerOfCluster.addPeer(slaveId); + } + manager = replication.getReplicationManager(); manager.addSource(slaveId); if (managerOfCluster != null) { @@ -306,7 +312,7 @@ public abstract class TestReplicationSourceManager { wal.rollWriter(); manager.logPositionAndCleanOldLogs(manager.getSources().get(0).getCurrentPath(), - "1", 0, false, false); + "1", 0, false); wal.append(hri, new WALKeyImpl(hri.getEncodedNameAsBytes(), test, System.currentTimeMillis(), mvcc, scopes), @@ -322,18 +328,14 @@ public abstract class TestReplicationSourceManager { @Test public void testClaimQueues() throws Exception { - final Server server = new DummyServer("hostname0.example.org"); - - - ReplicationQueues rq = - ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(server.getConfiguration(), server, - server.getZooKeeper())); - rq.init(server.getServerName().toString()); + Server server = new DummyServer("hostname0.example.org"); + ReplicationQueueStorage rq = ReplicationStorageFactory + .getReplicationQueueStorage(server.getZooKeeper(), server.getConfiguration()); // populate some znodes in the peer znode files.add("log1"); files.add("log2"); for (String file : files) { - rq.addLog("1", file); + rq.addWAL(server.getServerName(), "1", file); } // create 3 DummyServers Server s1 = new DummyServer("dummyserver1.example.org"); @@ -341,12 +343,9 @@ public abstract class TestReplicationSourceManager { Server s3 = new DummyServer("dummyserver3.example.org"); // create 3 DummyNodeFailoverWorkers - DummyNodeFailoverWorker w1 = new DummyNodeFailoverWorker( - server.getServerName().getServerName(), s1); - DummyNodeFailoverWorker w2 = new DummyNodeFailoverWorker( - server.getServerName().getServerName(), s2); - DummyNodeFailoverWorker w3 = new DummyNodeFailoverWorker( - server.getServerName().getServerName(), s3); + DummyNodeFailoverWorker w1 = new DummyNodeFailoverWorker(server.getServerName(), s1); + DummyNodeFailoverWorker w2 = new DummyNodeFailoverWorker(server.getServerName(), s2); + DummyNodeFailoverWorker w3 = new DummyNodeFailoverWorker(server.getServerName(), s3); latch = new CountDownLatch(3); // start the threads @@ -365,11 +364,9 @@ public abstract class TestReplicationSourceManager { @Test public void testCleanupFailoverQueues() throws Exception { - final Server server = new DummyServer("hostname1.example.org"); - ReplicationQueues rq = - ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(server.getConfiguration(), server, - server.getZooKeeper())); - rq.init(server.getServerName().toString()); + Server server = new DummyServer("hostname1.example.org"); + ReplicationQueueStorage rq = ReplicationStorageFactory + .getReplicationQueueStorage(server.getZooKeeper(), server.getConfiguration()); // populate some znodes in the peer znode SortedSet files = new TreeSet<>(); String group = "testgroup"; @@ -378,19 +375,14 @@ public abstract class TestReplicationSourceManager { files.add(file1); files.add(file2); for (String file : files) { - rq.addLog("1", file); + rq.addWAL(server.getServerName(), "1", file); } Server s1 = new DummyServer("dummyserver1.example.org"); - ReplicationQueues rq1 = - ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(s1.getConfiguration(), s1, - s1.getZooKeeper())); - rq1.init(s1.getServerName().toString()); ReplicationPeers rp1 = - ReplicationFactory.getReplicationPeers(s1.getZooKeeper(), s1.getConfiguration(), s1); + ReplicationFactory.getReplicationPeers(s1.getZooKeeper(), s1.getConfiguration()); rp1.init(); NodeFailoverWorker w1 = - manager.new NodeFailoverWorker(server.getServerName().getServerName(), rq1, rp1, new UUID( - new Long(1), new Long(2))); + manager.new NodeFailoverWorker(server.getServerName()); w1.run(); assertEquals(1, manager.getWalsByIdRecoveredQueues().size()); String id = "1-" + server.getServerName().getServerName(); @@ -402,17 +394,16 @@ public abstract class TestReplicationSourceManager { @Test public void testCleanupUnknownPeerZNode() throws Exception { - final Server server = new DummyServer("hostname2.example.org"); - ReplicationQueues rq = ReplicationFactory.getReplicationQueues( - new ReplicationQueuesArguments(server.getConfiguration(), server, server.getZooKeeper())); - rq.init(server.getServerName().toString()); + Server server = new DummyServer("hostname2.example.org"); + ReplicationQueueStorage rq = ReplicationStorageFactory + .getReplicationQueueStorage(server.getZooKeeper(), server.getConfiguration()); // populate some znodes in the peer znode // add log to an unknown peer String group = "testgroup"; - rq.addLog("2", group + ".log1"); - rq.addLog("2", group + ".log2"); + rq.addWAL(server.getServerName(), "2", group + ".log1"); + rq.addWAL(server.getServerName(), "2", group + ".log2"); - NodeFailoverWorker w1 = manager.new NodeFailoverWorker(server.getServerName().getServerName()); + NodeFailoverWorker w1 = manager.new NodeFailoverWorker(server.getServerName()); w1.run(); // The log of the unknown peer should be removed from zk @@ -475,10 +466,8 @@ public abstract class TestReplicationSourceManager { .setClusterKey("localhost:" + utility.getZkCluster().getClientPort() + ":/hbase"); try { DummyServer server = new DummyServer(); - final ReplicationQueues rq = - ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments( - server.getConfiguration(), server, server.getZooKeeper())); - rq.init(server.getServerName().toString()); + ReplicationQueueStorage rq = ReplicationStorageFactory + .getReplicationQueueStorage(server.getZooKeeper(), server.getConfiguration()); // Purposely fail ReplicationSourceManager.addSource() by causing ReplicationSourceInterface // initialization to throw an exception. conf.set("replication.replicationsource.implementation", @@ -492,11 +481,11 @@ public abstract class TestReplicationSourceManager { assertNull(manager.getSource(peerId)); // Create a replication queue for the fake peer - rq.addLog(peerId, "FakeFile"); + rq.addWAL(server.getServerName(), peerId, "FakeFile"); // Unregister peer, this should remove the peer and clear all queues associated with it // Need to wait for the ReplicationTracker to pick up the changes and notify listeners. removePeerAndWait(peerId); - assertFalse(rq.getAllQueues().contains(peerId)); + assertFalse(rq.getAllQueues(server.getServerName()).contains(peerId)); } finally { conf.set("replication.replicationsource.implementation", replicationSourceImplName); removePeerAndWait(peerId); @@ -535,18 +524,16 @@ public abstract class TestReplicationSourceManager { final int globalLogQueueSizeInitial = globalSource.getSizeOfLogQueue(); final long sizeOfLatestPath = getSizeOfLatestPath(); addPeerAndWait(peerId, peerConfig, true); - assertEquals(sizeOfLatestPath + globalLogQueueSizeInitial, - globalSource.getSizeOfLogQueue()); + assertEquals(sizeOfLatestPath + globalLogQueueSizeInitial, globalSource.getSizeOfLogQueue()); ReplicationSourceInterface source = manager.getSource(peerId); // Sanity check assertNotNull(source); final int sizeOfSingleLogQueue = source.getSourceMetrics().getSizeOfLogQueue(); // Enqueue log and check if metrics updated source.enqueueLog(new Path("abc")); - assertEquals(1 + sizeOfSingleLogQueue, - source.getSourceMetrics().getSizeOfLogQueue()); - assertEquals(source.getSourceMetrics().getSizeOfLogQueue() - + globalLogQueueSizeInitial, globalSource.getSizeOfLogQueue()); + assertEquals(1 + sizeOfSingleLogQueue, source.getSourceMetrics().getSizeOfLogQueue()); + assertEquals(source.getSourceMetrics().getSizeOfLogQueue() + globalLogQueueSizeInitial, + globalSource.getSizeOfLogQueue()); // Removing the peer should reset the global metrics removePeerAndWait(peerId); @@ -556,9 +543,8 @@ public abstract class TestReplicationSourceManager { addPeerAndWait(peerId, peerConfig, true); source = manager.getSource(peerId); assertNotNull(source); - assertEquals(sizeOfLatestPath, source.getSourceMetrics().getSizeOfLogQueue()); - assertEquals(source.getSourceMetrics().getSizeOfLogQueue() - + globalLogQueueSizeInitial, globalSource.getSizeOfLogQueue()); + assertEquals(source.getSourceMetrics().getSizeOfLogQueue() + globalLogQueueSizeInitial, + globalSource.getSizeOfLogQueue()); } finally { removePeerAndWait(peerId); } @@ -574,9 +560,15 @@ public abstract class TestReplicationSourceManager { private void addPeerAndWait(final String peerId, final ReplicationPeerConfig peerConfig, final boolean waitForSource) throws Exception { final ReplicationPeers rp = manager.getReplicationPeers(); - rp.registerPeer(peerId, peerConfig); + rp.getPeerStorage().addPeer(peerId, peerConfig, true); + try { + manager.addPeer(peerId); + } catch (Exception e) { + // ignore the failed exception, because we'll test both success & failed case. + } waitPeer(peerId, manager, waitForSource); if (managerOfCluster != null) { + managerOfCluster.addPeer(peerId); waitPeer(peerId, managerOfCluster, waitForSource); } } @@ -595,7 +587,7 @@ public abstract class TestReplicationSourceManager { } return true; } else { - return (rp.getConnectedPeer(peerId) != null); + return (rp.getPeer(peerId) != null); } }); } @@ -607,15 +599,20 @@ public abstract class TestReplicationSourceManager { */ private void removePeerAndWait(final String peerId) throws Exception { final ReplicationPeers rp = manager.getReplicationPeers(); - if (rp.getAllPeerIds().contains(peerId)) { - rp.unregisterPeer(peerId); + if (rp.getPeerStorage().listPeerIds().contains(peerId)) { + rp.getPeerStorage().removePeer(peerId); + try { + manager.removePeer(peerId); + } catch (Exception e) { + // ignore the failed exception and continue. + } } Waiter.waitFor(conf, 20000, new Waiter.Predicate() { - @Override public boolean evaluate() throws Exception { - List peers = rp.getAllPeerIds(); - return (!manager.getAllQueues().contains(peerId)) && (rp.getConnectedPeer(peerId) == null) - && (!peers.contains(peerId)) - && manager.getSource(peerId) == null; + @Override + public boolean evaluate() throws Exception { + Collection peers = rp.getPeerStorage().listPeerIds(); + return (!manager.getAllQueues().contains(peerId)) && (rp.getPeer(peerId) == null) + && (!peers.contains(peerId)) && manager.getSource(peerId) == null; } }); } @@ -658,25 +655,24 @@ public abstract class TestReplicationSourceManager { static class DummyNodeFailoverWorker extends Thread { private Map> logZnodesMap; Server server; - private String deadRsZnode; - ReplicationQueues rq; + private ServerName deadRS; + ReplicationQueueStorage rq; - public DummyNodeFailoverWorker(String znode, Server s) throws Exception { - this.deadRsZnode = znode; + public DummyNodeFailoverWorker(ServerName deadRS, Server s) throws Exception { + this.deadRS = deadRS; this.server = s; - this.rq = - ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(server.getConfiguration(), server, - server.getZooKeeper())); - this.rq.init(this.server.getServerName().toString()); + this.rq = ReplicationStorageFactory.getReplicationQueueStorage(server.getZooKeeper(), + server.getConfiguration()); } @Override public void run() { try { logZnodesMap = new HashMap<>(); - List queues = rq.getUnClaimedQueueIds(deadRsZnode); - for(String queue:queues){ - Pair> pair = rq.claimQueue(deadRsZnode, queue); + List queues = rq.getAllQueues(deadRS); + for (String queue : queues) { + Pair> pair = + rq.claimQueue(deadRS, queue, server.getServerName()); if (pair != null) { logZnodesMap.put(pair.getFirst(), pair.getSecond()); } @@ -715,9 +711,9 @@ public abstract class TestReplicationSourceManager { @Override public void init(Configuration conf, FileSystem fs, ReplicationSourceManager manager, - ReplicationQueues rq, ReplicationPeers rp, Server server, String peerClusterId, - UUID clusterId, ReplicationEndpoint replicationEndpoint, - WALFileLengthProvider walFileLengthProvider, MetricsSource metrics) throws IOException { + ReplicationQueueStorage rq, ReplicationPeer rp, Server server, String peerClusterId, + UUID clusterId, WALFileLengthProvider walFileLengthProvider, MetricsSource metrics) + throws IOException { throw new IOException("Failing deliberately"); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManagerZkImpl.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManagerZkImpl.java index b47a8d3..490c4b5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManagerZkImpl.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManagerZkImpl.java @@ -1,53 +1,47 @@ -/* -* -* 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. -*/ +/** + * 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 static org.junit.Assert.assertTrue; + +import java.util.List; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.ServerName; -import org.apache.hadoop.hbase.replication.ReplicationFactory; import org.apache.hadoop.hbase.replication.ReplicationQueueInfo; -import org.apache.hadoop.hbase.replication.ReplicationQueues; -import org.apache.hadoop.hbase.replication.ReplicationQueuesArguments; -import org.apache.hadoop.hbase.replication.ReplicationQueuesClientArguments; -import org.apache.hadoop.hbase.replication.ReplicationQueuesClientZKImpl; +import org.apache.hadoop.hbase.replication.ReplicationQueueStorage; import org.apache.hadoop.hbase.replication.ReplicationSourceDummy; +import org.apache.hadoop.hbase.replication.ReplicationStorageFactory; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.ReplicationTests; import org.junit.BeforeClass; import org.junit.Test; import org.junit.experimental.categories.Category; -import java.util.List; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - /** * Tests the ReplicationSourceManager with ReplicationQueueZkImpl's and * ReplicationQueuesClientZkImpl. Also includes extra tests outside of those in * TestReplicationSourceManager that test ReplicationQueueZkImpl-specific behaviors. */ -@Category({ReplicationTests.class, MediumTests.class}) +@Category({ ReplicationTests.class, MediumTests.class }) public class TestReplicationSourceManagerZkImpl extends TestReplicationSourceManager { + @BeforeClass public static void setUpBeforeClass() throws Exception { conf = HBaseConfiguration.create(); @@ -63,16 +57,14 @@ public class TestReplicationSourceManagerZkImpl extends TestReplicationSourceMan // Tests the naming convention of adopted queues for ReplicationQueuesZkImpl @Test public void testNodeFailoverDeadServerParsing() throws Exception { - final Server server = new DummyServer("ec2-54-234-230-108.compute-1.amazonaws.com"); - ReplicationQueues repQueues = - ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(conf, server, - server.getZooKeeper())); - repQueues.init(server.getServerName().toString()); + Server server = new DummyServer("ec2-54-234-230-108.compute-1.amazonaws.com"); + ReplicationQueueStorage queueStorage = + ReplicationStorageFactory.getReplicationQueueStorage(server.getZooKeeper(), conf); // populate some znodes in the peer znode files.add("log1"); files.add("log2"); for (String file : files) { - repQueues.addLog("1", file); + queueStorage.addWAL(server.getServerName(), "1", file); } // create 3 DummyServers @@ -81,30 +73,22 @@ public class TestReplicationSourceManagerZkImpl extends TestReplicationSourceMan Server s3 = new DummyServer("ec2-23-20-187-167.compute-1.amazonaws.com"); // simulate three servers fail sequentially - ReplicationQueues rq1 = - ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(s1.getConfiguration(), s1, - s1.getZooKeeper())); - rq1.init(s1.getServerName().toString()); - String serverName = server.getServerName().getServerName(); - List unclaimed = rq1.getUnClaimedQueueIds(serverName); - rq1.claimQueue(serverName, unclaimed.get(0)).getSecond(); - rq1.removeReplicatorIfQueueIsEmpty(unclaimed.get(0)); - ReplicationQueues rq2 = - ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(s2.getConfiguration(), s2, - s2.getZooKeeper())); - rq2.init(s2.getServerName().toString()); - serverName = s1.getServerName().getServerName(); - unclaimed = rq2.getUnClaimedQueueIds(serverName); - rq2.claimQueue(serverName, unclaimed.get(0)).getSecond(); - rq2.removeReplicatorIfQueueIsEmpty(unclaimed.get(0)); - ReplicationQueues rq3 = - ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(s3.getConfiguration(), s3, - s3.getZooKeeper())); - rq3.init(s3.getServerName().toString()); - serverName = s2.getServerName().getServerName(); - unclaimed = rq3.getUnClaimedQueueIds(serverName); - String queue3 = rq3.claimQueue(serverName, unclaimed.get(0)).getFirst(); - rq3.removeReplicatorIfQueueIsEmpty(unclaimed.get(0)); + ServerName serverName = server.getServerName(); + List unclaimed = queueStorage.getAllQueues(serverName); + queueStorage.claimQueue(serverName, unclaimed.get(0), s1.getServerName()); + queueStorage.removeReplicatorIfQueueIsEmpty(serverName); + + serverName = s1.getServerName(); + unclaimed = queueStorage.getAllQueues(serverName); + queueStorage.claimQueue(serverName, unclaimed.get(0), s2.getServerName()); + queueStorage.removeReplicatorIfQueueIsEmpty(serverName); + + serverName = s2.getServerName(); + unclaimed = queueStorage.getAllQueues(serverName); + String queue3 = + queueStorage.claimQueue(serverName, unclaimed.get(0), s3.getServerName()).getFirst(); + queueStorage.removeReplicatorIfQueueIsEmpty(serverName); + ReplicationQueueInfo replicationQueueInfo = new ReplicationQueueInfo(queue3); List result = replicationQueueInfo.getDeadRegionServers(); // verify @@ -114,41 +98,4 @@ public class TestReplicationSourceManagerZkImpl extends TestReplicationSourceMan server.stop(""); } - - @Test - public void testFailoverDeadServerCversionChange() throws Exception { - final Server s0 = new DummyServer("cversion-change0.example.org"); - ReplicationQueues repQueues = - ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(conf, s0, - s0.getZooKeeper())); - repQueues.init(s0.getServerName().toString()); - // populate some znodes in the peer znode - files.add("log1"); - files.add("log2"); - for (String file : files) { - repQueues.addLog("1", file); - } - // simulate queue transfer - Server s1 = new DummyServer("cversion-change1.example.org"); - ReplicationQueues rq1 = - ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(s1.getConfiguration(), s1, - s1.getZooKeeper())); - rq1.init(s1.getServerName().toString()); - - ReplicationQueuesClientZKImpl client = - (ReplicationQueuesClientZKImpl)ReplicationFactory.getReplicationQueuesClient( - new ReplicationQueuesClientArguments(s1.getConfiguration(), s1, s1.getZooKeeper())); - - int v0 = client.getQueuesZNodeCversion(); - List queues = rq1.getUnClaimedQueueIds(s0.getServerName().getServerName()); - for(String queue : queues) { - rq1.claimQueue(s0.getServerName().getServerName(), queue); - } - rq1.removeReplicatorIfQueueIsEmpty(s0.getServerName().getServerName()); - int v1 = client.getQueuesZNodeCversion(); - // cversion should increase by 1 since a child node is deleted - assertEquals(v0 + 1, v1); - - s0.stop(""); - } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java index 2435532..83886b0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -37,7 +37,6 @@ import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.List; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; @@ -3138,4 +3137,18 @@ public class TestAccessController extends SecureTestUtil { verifyAllowed( action, SUPERUSER, USER_ADMIN, USER_CREATE, USER_RW, USER_RO, USER_NONE, USER_OWNER); } + + @Test + public void testExecuteProcedures() throws Exception { + AccessTestAction action = new AccessTestAction() { + @Override + public Object run() throws Exception { + ACCESS_CONTROLLER.preExecuteProcedures(ObserverContextImpl.createAndPrepare(RSCP_ENV)); + return null; + } + }; + + verifyAllowed(action, SUPERUSER); + verifyDenied(action, USER_CREATE, USER_RW, USER_RO, USER_NONE, USER_OWNER, USER_ADMIN); + } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckReplication.java new file mode 100644 index 0000000..e64255c --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckReplication.java @@ -0,0 +1,101 @@ +/** + * 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.util; + +import static org.junit.Assert.assertEquals; + +import java.util.List; +import java.util.stream.Stream; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; +import org.apache.hadoop.hbase.replication.ReplicationPeerStorage; +import org.apache.hadoop.hbase.replication.ReplicationQueueStorage; +import org.apache.hadoop.hbase.replication.ReplicationStorageFactory; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.testclassification.ReplicationTests; +import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter.ERROR_CODE; +import org.apache.hadoop.hbase.util.hbck.HbckTestingUtil; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category({ ReplicationTests.class, MediumTests.class }) +public class TestHBaseFsckReplication { + + private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); + + @BeforeClass + public static void setUp() throws Exception { + UTIL.startMiniCluster(1); + } + + @AfterClass + public static void tearDown() throws Exception { + UTIL.shutdownMiniCluster(); + } + + @Test + public void test() throws Exception { + ReplicationPeerStorage peerStorage = ReplicationStorageFactory + .getReplicationPeerStorage(UTIL.getZooKeeperWatcher(), UTIL.getConfiguration()); + ReplicationQueueStorage queueStorage = ReplicationStorageFactory + .getReplicationQueueStorage(UTIL.getZooKeeperWatcher(), UTIL.getConfiguration()); + + String peerId1 = "1"; + String peerId2 = "2"; + peerStorage.addPeer(peerId1, ReplicationPeerConfig.newBuilder().setClusterKey("key").build(), + true); + peerStorage.addPeer(peerId2, ReplicationPeerConfig.newBuilder().setClusterKey("key").build(), + true); + for (int i = 0; i < 10; i++) { + queueStorage.addWAL(ServerName.valueOf("localhost", 10000 + i, 100000 + i), peerId1, + "file-" + i); + } + queueStorage.addWAL(ServerName.valueOf("localhost", 10000, 100000), peerId2, "file"); + HBaseFsck fsck = HbckTestingUtil.doFsck(UTIL.getConfiguration(), true); + HbckTestingUtil.assertNoErrors(fsck); + + // should not remove anything since the replication peer is still alive + assertEquals(10, queueStorage.getListOfReplicators().size()); + peerStorage.removePeer(peerId1); + // there should be orphan queues + assertEquals(10, queueStorage.getListOfReplicators().size()); + fsck = HbckTestingUtil.doFsck(UTIL.getConfiguration(), false); + HbckTestingUtil.assertErrors(fsck, Stream.generate(() -> { + return ERROR_CODE.UNDELETED_REPLICATION_QUEUE; + }).limit(10).toArray(ERROR_CODE[]::new)); + + // should not delete anything when fix is false + assertEquals(10, queueStorage.getListOfReplicators().size()); + + fsck = HbckTestingUtil.doFsck(UTIL.getConfiguration(), true); + HbckTestingUtil.assertErrors(fsck, Stream.generate(() -> { + return ERROR_CODE.UNDELETED_REPLICATION_QUEUE; + }).limit(10).toArray(ERROR_CODE[]::new)); + + List replicators = queueStorage.getListOfReplicators(); + // should not remove the server with queue for peerId2 + assertEquals(1, replicators.size()); + assertEquals(ServerName.valueOf("localhost", 10000, 100000), replicators.get(0)); + for (String queueId : queueStorage.getAllQueues(replicators.get(0))) { + assertEquals(peerId2, queueId); + } + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/HbckTestingUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/HbckTestingUtil.java index 60d7324..99e4f08 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/HbckTestingUtil.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/HbckTestingUtil.java @@ -46,7 +46,7 @@ public class HbckTestingUtil { public static HBaseFsck doFsck(Configuration conf, boolean fixAssignments, boolean fixMeta, boolean fixHdfsHoles, boolean fixHdfsOverlaps, boolean fixHdfsOrphans, boolean fixTableOrphans, boolean fixVersionFile, boolean fixReferenceFiles, boolean fixHFileLinks, - boolean fixEmptyMetaRegionInfo, boolean fixTableLocks, Boolean fixReplication, + boolean fixEmptyMetaRegionInfo, boolean fixTableLocks, boolean fixReplication, TableName table) throws Exception { HBaseFsck fsck = new HBaseFsck(conf, exec); try { @@ -78,10 +78,8 @@ public class HbckTestingUtil { /** * Runs hbck with the -sidelineCorruptHFiles option - * @param conf * @param table table constraint - * @return - * @throws Exception + * @return hbckInstance */ public static HBaseFsck doHFileQuarantine(Configuration conf, TableName table) throws Exception { String[] args = {"-sidelineCorruptHFiles", "-ignorePreCheckPermission", table.getNameAsString()}; diff --git a/hbase-shell/src/test/ruby/shell/list_locks_test.rb b/hbase-shell/src/test/ruby/shell/list_locks_test.rb index f465a6b..ef1c0ce 100644 --- a/hbase-shell/src/test/ruby/shell/list_locks_test.rb +++ b/hbase-shell/src/test/ruby/shell/list_locks_test.rb @@ -67,6 +67,25 @@ module Hbase proc_id) end + define_test 'list peer locks' do + lock = create_exclusive_lock(0) + peer_id = '1' + + @scheduler.waitPeerExclusiveLock(lock, peer_id) + output = capture_stdout { @list_locks.command } + @scheduler.wakePeerExclusiveLock(lock, peer_id) + + assert_equal( + "PEER(1)\n" \ + "Lock type: EXCLUSIVE, procedure: {" \ + "\"className\"=>\"org.apache.hadoop.hbase.master.locking.LockProcedure\", " \ + "\"procId\"=>\"0\", \"submittedTime\"=>\"0\", \"state\"=>\"RUNNABLE\", " \ + "\"lastUpdate\"=>\"0\", " \ + "\"stateMessage\"=>[{\"lockType\"=>\"EXCLUSIVE\", \"description\"=>\"description\"}]" \ + "}\n\n", + output) + end + define_test 'list server locks' do lock = create_exclusive_lock(0) diff --git a/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/HBaseZKTestingUtility.java b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/HBaseZKTestingUtility.java index fc31c37..b755c32 100644 --- a/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/HBaseZKTestingUtility.java +++ b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/HBaseZKTestingUtility.java @@ -182,8 +182,7 @@ public class HBaseZKTestingUtility extends HBaseCommonTestingUtility { /** * Gets a ZKWatcher. */ - public static ZKWatcher getZooKeeperWatcher(HBaseZKTestingUtility testUtil) - throws ZooKeeperConnectionException, IOException { + public static ZKWatcher getZooKeeperWatcher(HBaseZKTestingUtility testUtil) throws IOException { ZKWatcher zkw = new ZKWatcher(testUtil.getConfiguration(), "unittest", new Abortable() { boolean aborted = false; -- 2.7.4