diff --git a/dev-support/design-docs/HBASE-27109 Move replication queue storage from zookeeper to a separated HBase table.pdf b/dev-support/design-docs/HBASE-27109 Move replication queue storage from zookeeper to a separated HBase table.pdf
new file mode 100644
index 000000000000..095fe0b1c9f9
Binary files /dev/null and b/dev-support/design-docs/HBASE-27109 Move replication queue storage from zookeeper to a separated HBase table.pdf differ
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/util/JsonMapper.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/util/JsonMapper.java
index 0ff131f23bf2..f2c4585a6a85 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/util/JsonMapper.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/util/JsonMapper.java
@@ -40,4 +40,8 @@ public static String writeMapAsString(Map map) throws IOExceptio
public static String writeObjectAsString(Object object) throws IOException {
return GSON.toJson(object);
}
+
+ public static T fromJson(String json, Class clazz) {
+ return GSON.fromJson(json, clazz);
+ }
}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZNodePaths.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZNodePaths.java
index d19d21004667..3f66c7cdc0c2 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZNodePaths.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZNodePaths.java
@@ -220,7 +220,11 @@ public String getRsPath(ServerName sn) {
* @param suffix ending of znode name
* @return result of properly joining prefix with suffix
*/
- public static String joinZNode(String prefix, String suffix) {
- return prefix + ZNodePaths.ZNODE_PATH_SEPARATOR + suffix;
+ public static String joinZNode(String prefix, String... suffix) {
+ StringBuilder sb = new StringBuilder(prefix);
+ for (String s : suffix) {
+ sb.append(ZNodePaths.ZNODE_PATH_SEPARATOR).append(s);
+ }
+ return sb.toString();
}
}
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 12f899d7565b..401410170097 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
@@ -26,5 +26,6 @@ public enum LockedResourceType {
TABLE,
REGION,
PEER,
- META
+ META,
+ GLOBAL
}
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/Procedure.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/Procedure.java
index 34c74d92c161..43adba2bc21a 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/Procedure.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/Procedure.java
@@ -21,6 +21,7 @@
import java.util.Arrays;
import java.util.List;
import java.util.Map;
+import java.util.concurrent.ThreadLocalRandom;
import org.apache.hadoop.hbase.exceptions.TimeoutIOException;
import org.apache.hadoop.hbase.metrics.Counter;
import org.apache.hadoop.hbase.metrics.Histogram;
@@ -33,6 +34,7 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureState;
/**
@@ -1011,6 +1013,19 @@ final void doReleaseLock(TEnvironment env, ProcedureStore store) {
releaseLock(env);
}
+ protected final ProcedureSuspendedException suspend(int timeoutMillis, boolean jitter)
+ throws ProcedureSuspendedException {
+ if (jitter) {
+ // 10% possible jitter
+ double add = (double) timeoutMillis * ThreadLocalRandom.current().nextDouble(0.1);
+ timeoutMillis += add;
+ }
+ setTimeout(timeoutMillis);
+ setState(ProcedureProtos.ProcedureState.WAITING_TIMEOUT);
+ skipPersistence();
+ throw new ProcedureSuspendedException();
+ }
+
@Override
public int compareTo(final Procedure other) {
return Long.compare(getProcId(), other.getProcId());
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/TimeoutExecutorThread.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/TimeoutExecutorThread.java
index 3b99781a5585..c0287a99435c 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/TimeoutExecutorThread.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/TimeoutExecutorThread.java
@@ -78,9 +78,13 @@ public void add(InlineChore chore) {
}
public void add(Procedure procedure) {
- LOG.info("ADDED {}; timeout={}, timestamp={}", procedure, procedure.getTimeout(),
- procedure.getTimeoutTimestamp());
- queue.add(new DelayedProcedure<>(procedure));
+ if (procedure.getTimeout() > 0) {
+ LOG.info("ADDED {}; timeout={}, timestamp={}", procedure, procedure.getTimeout(),
+ procedure.getTimeoutTimestamp());
+ queue.add(new DelayedProcedure<>(procedure));
+ } else {
+ LOG.info("Got negative timeout {} for {}, skip adding", procedure.getTimeout(), procedure);
+ }
}
public boolean remove(Procedure procedure) {
diff --git a/hbase-protocol-shaded/src/main/protobuf/server/master/MasterProcedure.proto b/hbase-protocol-shaded/src/main/protobuf/server/master/MasterProcedure.proto
index 59bb031589af..901abf6bd0c5 100644
--- a/hbase-protocol-shaded/src/main/protobuf/server/master/MasterProcedure.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/server/master/MasterProcedure.proto
@@ -515,6 +515,7 @@ message UpdatePeerConfigStateData {
message RemovePeerStateData {
optional ReplicationPeer peer_config = 1;
+ repeated int64 ongoing_assign_replication_queues_proc_ids = 2;
}
message EnablePeerStateData {
@@ -679,16 +680,13 @@ message ClaimReplicationQueueRemoteStateData {
required ServerName crashed_server = 1;
required string queue = 2;
required ServerName target_server = 3;
+ optional ServerName source_server = 4;
}
message ClaimReplicationQueueRemoteParameter {
required ServerName crashed_server = 1;
required string queue = 2;
-}
-
-enum ClaimReplicationQueuesState {
- CLAIM_REPLICATION_QUEUES_DISPATCH = 1;
- CLAIM_REPLICATION_QUEUES_FINISH = 2;
+ optional ServerName source_server = 3;
}
enum ModifyTableDescriptorState {
@@ -715,3 +713,27 @@ message ModifyStoreFileTrackerStateData {
message ModifyColumnFamilyStoreFileTrackerStateData {
required bytes family = 1;
}
+
+enum AssignReplicationQueuesState {
+ ASSIGN_REPLICATION_QUEUES_ADD_MISSING_QUEUES = 1;
+ ASSIGN_REPLICATION_QUEUES_CLAIM = 2;
+ ASSIGN_REPLICATION_QUEUES_REMOVE_QUEUES = 3;
+}
+
+message AssignReplicationQueuesStateData {
+ required ServerName crashed_server = 1;
+}
+
+enum MigrateReplicationQueueFromZkToTableState {
+ MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_DISABLE_CLEANER = 1;
+ MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_PREPARE = 2;
+ MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_DISABLE_PEER = 3;
+ MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_MIGRATE = 4;
+ MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_WAIT_UPGRADING = 5;
+ MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_ENABLE_PEER = 6;
+ MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_ENABLE_CLEANER = 7;
+}
+
+message MigrateReplicationQueueFromZkToTableStateData {
+ repeated string disabled_peer_id = 1;
+}
diff --git a/hbase-replication/pom.xml b/hbase-replication/pom.xml
index 9acab39599fa..b4f1cfa224da 100644
--- a/hbase-replication/pom.xml
+++ b/hbase-replication/pom.xml
@@ -104,6 +104,16 @@
junit
test
+
+ org.hamcrest
+ hamcrest-core
+ test
+
+
+ org.hamcrest
+ hamcrest-library
+ test
+
org.mockito
mockito-core
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationGroupOffset.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationGroupOffset.java
new file mode 100644
index 000000000000..bd13594b99a0
--- /dev/null
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationGroupOffset.java
@@ -0,0 +1,57 @@
+/*
+ * 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.yetus.audience.InterfaceAudience;
+
+@InterfaceAudience.Private
+public class ReplicationGroupOffset {
+
+ public static final ReplicationGroupOffset BEGIN = new ReplicationGroupOffset("", 0L);
+
+ private final String wal;
+
+ private final long offset;
+
+ public ReplicationGroupOffset(String wal, long offset) {
+ this.wal = wal;
+ this.offset = offset;
+ }
+
+ public String getWal() {
+ return wal;
+ }
+
+ /**
+ * A negative value means this file has already been fully replicated out
+ */
+ public long getOffset() {
+ return offset;
+ }
+
+ @Override
+ public String toString() {
+ return wal + ":" + offset;
+ }
+
+ public static ReplicationGroupOffset parse(String str) {
+ int index = str.lastIndexOf(':');
+ return new ReplicationGroupOffset(str.substring(0, index),
+ Long.parseLong(str.substring(index + 1)));
+ }
+}
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueData.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueData.java
new file mode 100644
index 000000000000..794ae9d3a558
--- /dev/null
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueData.java
@@ -0,0 +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.
+ */
+package org.apache.hadoop.hbase.replication;
+
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableMap;
+
+/**
+ * Representing all the information for a replication queue.
+ */
+@InterfaceAudience.Private
+public class ReplicationQueueData {
+
+ private final ReplicationQueueId id;
+
+ private final ImmutableMap offsets;
+
+ public ReplicationQueueData(ReplicationQueueId id,
+ ImmutableMap offsets) {
+ this.id = id;
+ this.offsets = offsets;
+ }
+
+ public ReplicationQueueId getId() {
+ return id;
+ }
+
+ public ImmutableMap getOffsets() {
+ return offsets;
+ }
+}
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueId.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueId.java
new file mode 100644
index 000000000000..73633dda9a4a
--- /dev/null
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueId.java
@@ -0,0 +1,141 @@
+/*
+ * 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.Objects;
+import java.util.Optional;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+
+@InterfaceAudience.Private
+public class ReplicationQueueId {
+
+ private final ServerName serverName;
+
+ private final String peerId;
+
+ private final Optional sourceServerName;
+
+ // we do not allow '-' in peer names so it is safe to use it as the separator for peer id and
+ // server name
+ private static final char PEER_ID_SEPARATOR = '-';
+
+ // The '/' character is not valid for a hostname or a nodename(FQDN, so it is safe to use it as
+ // the separator for server names)
+ private static final char SERVER_NAME_SEPARATOR = '/';
+
+ public ReplicationQueueId(ServerName serverName, String peerId) {
+ this.serverName = Objects.requireNonNull(serverName);
+ this.peerId = Objects.requireNonNull(peerId);
+ this.sourceServerName = Optional.empty();
+ }
+
+ public ReplicationQueueId(ServerName serverName, String peerId, ServerName sourceServerName) {
+ this.serverName = Objects.requireNonNull(serverName);
+ this.peerId = Objects.requireNonNull(peerId);
+ this.sourceServerName = Optional.of(sourceServerName);
+ }
+
+ public ServerName getServerName() {
+ return serverName;
+ }
+
+ public String getPeerId() {
+ return peerId;
+ }
+
+ public Optional getSourceServerName() {
+ return sourceServerName;
+ }
+
+ public ServerName getServerWALsBelongTo() {
+ return sourceServerName.orElse(serverName);
+ }
+
+ public boolean isRecovered() {
+ return sourceServerName.isPresent();
+ }
+
+ public ReplicationQueueId claim(ServerName targetServerName) {
+ ServerName newSourceServerName = sourceServerName.orElse(serverName);
+ return new ReplicationQueueId(targetServerName, peerId, newSourceServerName);
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(peerId, serverName, sourceServerName);
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (this == obj) {
+ return true;
+ }
+ if (!(obj instanceof ReplicationQueueId)) {
+ return false;
+ }
+ ReplicationQueueId other = (ReplicationQueueId) obj;
+ return Objects.equals(peerId, other.peerId) && Objects.equals(serverName, other.serverName)
+ && Objects.equals(sourceServerName, other.sourceServerName);
+ }
+
+ @Override
+ public String toString() {
+ StringBuilder sb =
+ new StringBuilder().append(peerId).append(PEER_ID_SEPARATOR).append(serverName);
+ sourceServerName.ifPresent(s -> sb.append(SERVER_NAME_SEPARATOR).append(s.toString()));
+ return sb.toString();
+ }
+
+ public static ReplicationQueueId parse(String str) {
+ int dashIndex = str.indexOf(PEER_ID_SEPARATOR);
+ String peerId = str.substring(0, dashIndex);
+ int slashIndex = str.indexOf(SERVER_NAME_SEPARATOR, dashIndex + 1);
+ if (slashIndex < 0) {
+ String serverName = str.substring(dashIndex + 1);
+ return new ReplicationQueueId(ServerName.valueOf(serverName), peerId);
+ } else {
+ String serverName = str.substring(dashIndex + 1, slashIndex);
+ String sourceServerName = str.substring(slashIndex + 1);
+ return new ReplicationQueueId(ServerName.valueOf(serverName), peerId,
+ ServerName.valueOf(sourceServerName));
+ }
+ }
+
+ public static String getPeerId(String str) {
+ int dashIndex = str.indexOf(PEER_ID_SEPARATOR);
+ return str.substring(0, dashIndex);
+ }
+
+ public static byte[] getScanPrefix(ServerName serverName, String peerId) {
+ return Bytes.toBytes(peerId + PEER_ID_SEPARATOR + serverName.toString());
+ }
+
+ public static byte[] getScanPrefix(String peerId) {
+ return Bytes.toBytes(peerId + PEER_ID_SEPARATOR);
+ }
+
+ private static char getNextChar(char c) {
+ return (char) ((int) c + 1);
+ }
+
+ public static byte[] getScanStartRowForNextPeerId(String peerId) {
+ return Bytes.toBytes(peerId + getNextChar(PEER_ID_SEPARATOR));
+ }
+}
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
index 0f95c04b2542..b5bc64eb55aa 100644
--- 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
@@ -20,9 +20,9 @@
import java.util.List;
import java.util.Map;
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.replication.ZKReplicationQueueStorageForMigration.ZkLastPushedSeqId;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.yetus.audience.InterfaceAudience;
@@ -33,40 +33,79 @@
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.
+ * Set the current offset for a specific WAL group in a given queue.
+ * @param queueId the id of the queue
+ * @param walGroup the group of the WAL, can be empty if multi wal is not enabled
+ * @param offset the current offset of replication progress
+ * @param lastSeqIds map with {encodedRegionName, sequenceId} pairs for serial replication.
*/
- void removeQueue(ServerName serverName, String queueId) throws ReplicationException;
+ void setOffset(ReplicationQueueId queueId, String walGroup, ReplicationGroupOffset offset,
+ Map lastSeqIds) 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
+ * Get the current offset of all the WAL groups for a queue
+ * @param queueId the id of the queue
+ * @return a map of all offsets of the WAL groups. The key the is WAL group and the value is the
+ * position.
*/
- void addWAL(ServerName serverName, String queueId, String fileName) throws ReplicationException;
+ Map getOffsets(ReplicationQueueId queueId)
+ 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
+ * Get a list of all queues for the specific peer.
+ * @param peerId the id of the peer
+ * @return a list of queueIds
*/
- void removeWAL(ServerName serverName, String queueId, String fileName)
+ List listAllQueueIds(String peerId) throws ReplicationException;
+
+ /**
+ * Get a list of all queues for the specific region server.
+ * @param serverName the server name of the region server that owns the set of queues
+ * @return a list of queueIds
+ */
+ List listAllQueueIds(ServerName serverName) throws ReplicationException;
+
+ /**
+ * Get a list of all queues for the specific region server and the specific peer
+ * @param peerId the id of the peer
+ * @param serverName the server name of the region server that owns the set of queues
+ * @return a list of queueIds
+ */
+ List listAllQueueIds(String peerId, ServerName serverName)
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. Will ignore if less than or equal to 0.
- * @param lastSeqIds map with {encodedRegionName, sequenceId} pairs for serial replication.
+ * Get a list of all queues and the offsets.
*/
- void setWALPosition(ServerName serverName, String queueId, String fileName, long position,
- Map lastSeqIds) throws ReplicationException;
+ List listAllQueues() 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 listAllReplicators() throws ReplicationException;
+
+ /**
+ * Change ownership for the queue identified by queueId and belongs to a dead region server.
+ * @param queueId the id of the queue
+ * @param targetServerName the name of the target region server
+ * @return the new PeerId and A SortedSet of WALs in its queue
+ */
+ Map claimQueue(ReplicationQueueId queueId,
+ ServerName targetServerName) throws ReplicationException;
+
+ /**
+ * Remove a replication queue
+ * @param queueId the id of the queue to remove
+ */
+ void removeQueue(ReplicationQueueId queueId) throws ReplicationException;
+
+ /**
+ * Remove all the replication queues for the given peer. Usually used when removing a peer.
+ * @param peerId the id of the peer
+ */
+ void removeAllQueues(String peerId) throws ReplicationException;
/**
* Read the max sequence id of the specific region for a given peer. For serial replication, we
@@ -99,67 +138,6 @@ void setWALPosition(ServerName serverName, String queueId, String fileName, long
void removeLastSequenceIds(String peerId, List encodedRegionNames)
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
@@ -203,9 +181,47 @@ Pair> claimQueue(ServerName sourceServerName, String q
Set getAllHFileRefs() throws ReplicationException;
/**
- * Get full znode name for given region server
- * @param serverName the name of the region server
- * @return full znode name
+ * Whether the replication queue table exists.
+ * @return Whether the replication queue table exists
+ */
+ boolean hasData() throws ReplicationException;
+
+ // the below 3 methods are used for migrating
+ /**
+ * Update the replication queue datas for a given region server.
+ */
+ void batchUpdateQueues(ServerName serverName, List datas)
+ throws ReplicationException;
+
+ /**
+ * Update last pushed sequence id for the given regions and peers.
+ */
+ void batchUpdateLastSequenceIds(List lastPushedSeqIds)
+ throws ReplicationException;
+
+ /**
+ * Add the given hfile refs to the given peer.
+ */
+ void batchUpdateHFileRefs(String peerId, List hfileRefs) throws ReplicationException;
+
+ // the below method is for clean up stale data after running ReplicatoinSyncUp
+ /**
+ * Remove all the last sequence ids and hfile references data which are written before the given
+ * timestamp.
+ *
+ * The data of these two types are not used by replication directly.
+ *
+ * For last sequence ids, we will check it in serial replication, to make sure that we will
+ * replicate all edits in order, so if there are stale data, the worst case is that we will stop
+ * replicating as we think we still need to finish previous ranges first, although actually we
+ * have already replicated them out.
+ *
+ * For hfile references, it is just used by hfile cleaner to not remove these hfiles before we
+ * replicate them out, so if there are stale data, the worst case is that we can not remove these
+ * hfiles, although actually they have already been replicated out.
+ *
+ * So it is OK for us to just bring up the cluster first, and then use this method to delete the
+ * stale data, i.e, the data which are written before a specific timestamp.
*/
- String getRsNode(ServerName serverName);
+ void removeLastSequenceIdsAndHFileRefsBefore(long ts) throws ReplicationException;
}
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
index 0124dbdd113d..ada127ee7831 100644
--- 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
@@ -17,12 +17,23 @@
*/
package org.apache.hadoop.hbase.replication;
+import java.io.IOException;
import java.lang.reflect.Constructor;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hbase.Coprocessor;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.CoprocessorDescriptorBuilder;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.util.ReflectionUtils;
import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
/**
* Used to create replication storage(peer, queue) classes.
@@ -30,6 +41,8 @@
@InterfaceAudience.Private
public final class ReplicationStorageFactory {
+ private static final Logger LOG = LoggerFactory.getLogger(ReplicationStorageFactory.class);
+
public static final String REPLICATION_PEER_STORAGE_IMPL = "hbase.replication.peer.storage.impl";
// must use zookeeper here, otherwise when user upgrading from an old version without changing the
@@ -37,6 +50,29 @@ public final class ReplicationStorageFactory {
public static final ReplicationPeerStorageType DEFAULT_REPLICATION_PEER_STORAGE_IMPL =
ReplicationPeerStorageType.ZOOKEEPER;
+ public static final String REPLICATION_QUEUE_TABLE_NAME = "hbase.replication.queue.table.name";
+
+ public static final TableName REPLICATION_QUEUE_TABLE_NAME_DEFAULT =
+ TableName.valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "replication");
+
+ public static final String REPLICATION_QUEUE_IMPL = "hbase.replication.queue.storage.impl";
+
+ public static TableDescriptor createReplicationQueueTableDescriptor(TableName tableName)
+ throws IOException {
+ return TableDescriptorBuilder.newBuilder(tableName)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.of(TableReplicationQueueStorage.QUEUE_FAMILY))
+ .setColumnFamily(
+ ColumnFamilyDescriptorBuilder.of(TableReplicationQueueStorage.LAST_SEQUENCE_ID_FAMILY))
+ .setColumnFamily(
+ ColumnFamilyDescriptorBuilder.of(TableReplicationQueueStorage.HFILE_REF_FAMILY))
+ .setValue("hbase.regionserver.region.split_restriction.type", "DelimitedKeyPrefix")
+ .setValue("hbase.regionserver.region.split_restriction.delimiter", "-")
+ .setCoprocessor(CoprocessorDescriptorBuilder
+ .newBuilder("org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint")
+ .setPriority(Coprocessor.PRIORITY_SYSTEM).build())
+ .build();
+ }
+
private ReplicationStorageFactory() {
}
@@ -76,8 +112,34 @@ public static ReplicationPeerStorage getReplicationPeerStorage(FileSystem fs, ZK
/**
* Create a new {@link ReplicationQueueStorage}.
*/
- public static ReplicationQueueStorage getReplicationQueueStorage(ZKWatcher zk,
+ public static ReplicationQueueStorage getReplicationQueueStorage(Connection conn,
Configuration conf) {
- return new ZKReplicationQueueStorage(zk, conf);
+ return getReplicationQueueStorage(conn, conf, TableName.valueOf(conf
+ .get(REPLICATION_QUEUE_TABLE_NAME, REPLICATION_QUEUE_TABLE_NAME_DEFAULT.getNameAsString())));
+ }
+
+ /**
+ * Create a new {@link ReplicationQueueStorage}.
+ */
+ public static ReplicationQueueStorage getReplicationQueueStorage(Connection conn,
+ Configuration conf, TableName tableName) {
+ Class extends ReplicationQueueStorage> clazz = conf.getClass(REPLICATION_QUEUE_IMPL,
+ TableReplicationQueueStorage.class, ReplicationQueueStorage.class);
+ try {
+ Constructor extends ReplicationQueueStorage> c =
+ clazz.getConstructor(Connection.class, TableName.class);
+ return c.newInstance(conn, tableName);
+ } catch (Exception e) {
+ LOG.debug(
+ "failed to create ReplicationQueueStorage with Connection, try creating with Configuration",
+ e);
+ return ReflectionUtils.newInstance(clazz, conf, tableName);
+ }
+ }
+
+ public static boolean isReplicationQueueTable(Configuration conf, TableName tableName) {
+ TableName replicationQueueTableName = TableName.valueOf(conf.get(REPLICATION_QUEUE_TABLE_NAME,
+ REPLICATION_QUEUE_TABLE_NAME_DEFAULT.getNameAsString()));
+ return replicationQueueTableName.equals(tableName);
}
}
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
index d1bca8b4b042..ae78781a3133 100644
--- 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
@@ -28,7 +28,6 @@
import org.apache.hadoop.hbase.CompoundConfiguration;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
@@ -81,20 +80,6 @@ public static Configuration getPeerClusterConfiguration(ReplicationPeerConfig pe
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;
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/TableReplicationQueueStorage.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/TableReplicationQueueStorage.java
new file mode 100644
index 000000000000..e59edd52f793
--- /dev/null
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/TableReplicationQueueStorage.java
@@ -0,0 +1,617 @@
+/*
+ * 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.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Set;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellScanner;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.CompareOperator;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.AsyncTable;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Delete;
+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.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Scan.ReadType;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.filter.KeyOnlyFilter;
+import org.apache.hadoop.hbase.replication.ZKReplicationQueueStorageForMigration.ZkLastPushedSeqId;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.FutureUtils;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableMap;
+
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.MutationType;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MultiRowMutationProtos;
+
+/**
+ * HBase table based replication queue storage.
+ */
+@InterfaceAudience.Private
+public class TableReplicationQueueStorage implements ReplicationQueueStorage {
+
+ public static final byte[] QUEUE_FAMILY = Bytes.toBytes("queue");
+
+ public static final byte[] LAST_SEQUENCE_ID_FAMILY = Bytes.toBytes("sid");
+
+ public static final byte[] HFILE_REF_FAMILY = Bytes.toBytes("hfileref");
+
+ private final Connection conn;
+
+ private final TableName tableName;
+
+ public TableReplicationQueueStorage(Connection conn, TableName tableName) {
+ this.conn = conn;
+ this.tableName = tableName;
+ }
+
+ private void addLastSeqIdsPut(MultiRowMutationProtos.MutateRowsRequest.Builder builder,
+ String peerId, Map lastSeqIds, AsyncTable> table) throws IOException {
+ // get the previous sequence ids first
+ byte[] row = Bytes.toBytes(peerId);
+ Get get = new Get(row);
+ lastSeqIds.keySet().forEach(encodedRegionName -> get.addColumn(LAST_SEQUENCE_ID_FAMILY,
+ Bytes.toBytes(encodedRegionName)));
+ Result result = FutureUtils.get(table.get(get));
+ Put put = new Put(row);
+ for (Map.Entry entry : lastSeqIds.entrySet()) {
+ String encodedRegionName = entry.getKey();
+ long lastSeqId = entry.getValue();
+ byte[] encodedRegionNameAsBytes = Bytes.toBytes(encodedRegionName);
+ byte[] previousLastSeqIdAsBytes =
+ result.getValue(LAST_SEQUENCE_ID_FAMILY, encodedRegionNameAsBytes);
+ if (previousLastSeqIdAsBytes != null) {
+ long previousLastSeqId = Bytes.toLong(previousLastSeqIdAsBytes);
+ if (lastSeqId > previousLastSeqId) {
+ // update last seq id when it is greater, and use CAS to make sure we do not overwrite
+ // other's value.
+ put.addColumn(LAST_SEQUENCE_ID_FAMILY, encodedRegionNameAsBytes,
+ Bytes.toBytes(lastSeqId));
+ builder.addCondition(ProtobufUtil.toCondition(row, LAST_SEQUENCE_ID_FAMILY,
+ encodedRegionNameAsBytes, CompareOperator.EQUAL, previousLastSeqIdAsBytes, null));
+ }
+ } else {
+ // also update last seq id when there is no value yet, and use CAS to make sure we do not
+ // overwrite
+ // other's value.
+ put.addColumn(LAST_SEQUENCE_ID_FAMILY, encodedRegionNameAsBytes, Bytes.toBytes(lastSeqId));
+ builder.addCondition(ProtobufUtil.toCondition(row, LAST_SEQUENCE_ID_FAMILY,
+ encodedRegionNameAsBytes, CompareOperator.EQUAL, null, null));
+ }
+ }
+ if (!put.isEmpty()) {
+ builder.addMutationRequest(ProtobufUtil.toMutation(MutationType.PUT, put));
+ }
+ }
+
+ @Override
+ public void setOffset(ReplicationQueueId queueId, String walGroup, ReplicationGroupOffset offset,
+ Map lastSeqIds) throws ReplicationException {
+ Put put = new Put(Bytes.toBytes(queueId.toString())).addColumn(QUEUE_FAMILY,
+ Bytes.toBytes(walGroup), Bytes.toBytes(offset.toString()));
+ AsyncTable> asyncTable = conn.toAsyncConnection().getTable(tableName);
+ try {
+ if (lastSeqIds.isEmpty()) {
+ FutureUtils.get(asyncTable.put(put));
+ } else {
+ for (;;) {
+ MultiRowMutationProtos.MutateRowsRequest.Builder builder =
+ MultiRowMutationProtos.MutateRowsRequest.newBuilder();
+ addLastSeqIdsPut(builder, queueId.getPeerId(), lastSeqIds, asyncTable);
+ if (builder.getMutationRequestCount() > 0) {
+ // use MultiRowMutationService to atomically update offset and last sequence ids
+ MultiRowMutationProtos.MutateRowsRequest request =
+ builder.addMutationRequest(ProtobufUtil.toMutation(MutationType.PUT, put)).build();
+ MultiRowMutationProtos.MutateRowsResponse responose =
+ FutureUtils.get(asyncTable. coprocessorService(
+ MultiRowMutationProtos.MultiRowMutationService::newStub,
+ (stub, controller, done) -> stub.mutateRows(controller, request, done),
+ put.getRow()));
+ if (responose.getProcessed()) {
+ break;
+ }
+ } else {
+ // we do not need to update last seq id, fallback to single put
+ FutureUtils.get(asyncTable.put(put));
+ break;
+ }
+ }
+ }
+ } catch (IOException e) {
+ throw new ReplicationException("failed to setOffset, queueId=" + queueId + ", walGroup="
+ + walGroup + ", offset=" + offset + ", lastSeqIds=" + lastSeqIds, e);
+ }
+ }
+
+ private ImmutableMap parseOffsets(Result result) {
+ ImmutableMap.Builder builder =
+ ImmutableMap.builderWithExpectedSize(result.size());
+ NavigableMap map = result.getFamilyMap(QUEUE_FAMILY);
+ if (map != null) {
+ map.forEach((k, v) -> {
+ String walGroup = Bytes.toString(k);
+ ReplicationGroupOffset offset = ReplicationGroupOffset.parse(Bytes.toString(v));
+ builder.put(walGroup, offset);
+ });
+ }
+ return builder.build();
+ }
+
+ private Map getOffsets0(Table table, ReplicationQueueId queueId)
+ throws IOException {
+ Result result = table.get(new Get(Bytes.toBytes(queueId.toString())).addFamily(QUEUE_FAMILY));
+ return parseOffsets(result);
+ }
+
+ @Override
+ public Map getOffsets(ReplicationQueueId queueId)
+ throws ReplicationException {
+ try (Table table = conn.getTable(tableName)) {
+ return getOffsets0(table, queueId);
+ } catch (IOException e) {
+ throw new ReplicationException("failed to getOffsets, queueId=" + queueId, e);
+ }
+ }
+
+ private void listAllQueueIds(Table table, Scan scan, List queueIds)
+ throws IOException {
+ try (ResultScanner scanner = table.getScanner(scan)) {
+ for (;;) {
+ Result result = scanner.next();
+ if (result == null) {
+ break;
+ }
+ ReplicationQueueId queueId = ReplicationQueueId.parse(Bytes.toString(result.getRow()));
+ queueIds.add(queueId);
+ }
+ }
+ }
+
+ private void listAllQueueIds(Table table, String peerId, ServerName serverName,
+ List queueIds) throws IOException {
+ listAllQueueIds(table,
+ new Scan().setStartStopRowForPrefixScan(ReplicationQueueId.getScanPrefix(serverName, peerId))
+ .addFamily(QUEUE_FAMILY).setFilter(new KeyOnlyFilter()),
+ queueIds);
+ }
+
+ @Override
+ public List listAllQueueIds(String peerId) throws ReplicationException {
+ Scan scan = new Scan().setStartStopRowForPrefixScan(ReplicationQueueId.getScanPrefix(peerId))
+ .addFamily(QUEUE_FAMILY).setFilter(new KeyOnlyFilter());
+ List queueIds = new ArrayList<>();
+ try (Table table = conn.getTable(tableName)) {
+ listAllQueueIds(table, scan, queueIds);
+ } catch (IOException e) {
+ throw new ReplicationException("failed to listAllQueueIds, peerId=" + peerId, e);
+ }
+ return queueIds;
+ }
+
+ @Override
+ public List listAllQueueIds(ServerName serverName)
+ throws ReplicationException {
+ List queueIds = new ArrayList<>();
+ try (Table table = conn.getTable(tableName)) {
+ KeyOnlyFilter keyOnlyFilter = new KeyOnlyFilter();
+ String previousPeerId = null;
+ for (;;) {
+ // first, get the next peerId
+ Scan peerScan =
+ new Scan().addFamily(QUEUE_FAMILY).setOneRowLimit().setFilter(keyOnlyFilter);
+ if (previousPeerId != null) {
+ peerScan.withStartRow(ReplicationQueueId.getScanStartRowForNextPeerId(previousPeerId));
+ }
+ String peerId;
+ try (ResultScanner scanner = table.getScanner(peerScan)) {
+ Result result = scanner.next();
+ if (result == null) {
+ // no more peers, break
+ break;
+ }
+ peerId = ReplicationQueueId.getPeerId(Bytes.toString(result.getRow()));
+ }
+ listAllQueueIds(table, peerId, serverName, queueIds);
+ previousPeerId = peerId;
+ }
+ } catch (IOException e) {
+ throw new ReplicationException("failed to listAllQueueIds, serverName=" + serverName, e);
+ }
+ return queueIds;
+ }
+
+ @Override
+ public List listAllQueueIds(String peerId, ServerName serverName)
+ throws ReplicationException {
+ List queueIds = new ArrayList<>();
+ try (Table table = conn.getTable(tableName)) {
+ listAllQueueIds(table, peerId, serverName, queueIds);
+ } catch (IOException e) {
+ throw new ReplicationException(
+ "failed to listAllQueueIds, peerId=" + peerId + ", serverName=" + serverName, e);
+ }
+ return queueIds;
+ }
+
+ @Override
+ public List listAllQueues() throws ReplicationException {
+ List queues = new ArrayList<>();
+ Scan scan = new Scan().addFamily(QUEUE_FAMILY).setReadType(ReadType.STREAM);
+ try (Table table = conn.getTable(tableName); ResultScanner scanner = table.getScanner(scan)) {
+ for (;;) {
+ Result result = scanner.next();
+ if (result == null) {
+ break;
+ }
+ ReplicationQueueId queueId = ReplicationQueueId.parse(Bytes.toString(result.getRow()));
+ ReplicationQueueData queueData = new ReplicationQueueData(queueId, parseOffsets(result));
+ queues.add(queueData);
+ }
+ } catch (IOException e) {
+ throw new ReplicationException("failed to listAllQueues", e);
+ }
+ return queues;
+ }
+
+ @Override
+ public List listAllReplicators() throws ReplicationException {
+ Set replicators = new HashSet<>();
+ Scan scan = new Scan().addFamily(QUEUE_FAMILY).setFilter(new KeyOnlyFilter())
+ .setReadType(ReadType.STREAM);
+ try (Table table = conn.getTable(tableName); ResultScanner scanner = table.getScanner(scan)) {
+ for (;;) {
+ Result result = scanner.next();
+ if (result == null) {
+ break;
+ }
+ ReplicationQueueId queueId = ReplicationQueueId.parse(Bytes.toString(result.getRow()));
+ replicators.add(queueId.getServerName());
+ }
+ } catch (IOException e) {
+ throw new ReplicationException("failed to listAllReplicators", e);
+ }
+ return new ArrayList<>(replicators);
+ }
+
+ @Override
+ public Map claimQueue(ReplicationQueueId queueId,
+ ServerName targetServerName) throws ReplicationException {
+ ReplicationQueueId newQueueId = queueId.claim(targetServerName);
+ byte[] coprocessorRow = ReplicationQueueId.getScanPrefix(queueId.getPeerId());
+ AsyncTable> asyncTable = conn.toAsyncConnection().getTable(tableName);
+ try (Table table = conn.getTable(tableName)) {
+ for (;;) {
+ Map offsets = getOffsets0(table, queueId);
+ if (offsets.isEmpty()) {
+ return Collections.emptyMap();
+ }
+ Map.Entry entry = offsets.entrySet().iterator().next();
+ ClientProtos.Condition condition = ProtobufUtil.toCondition(
+ Bytes.toBytes(queueId.toString()), QUEUE_FAMILY, Bytes.toBytes(entry.getKey()),
+ CompareOperator.EQUAL, Bytes.toBytes(entry.getValue().toString()), null);
+ Delete delete = new Delete(Bytes.toBytes(queueId.toString())).addFamily(QUEUE_FAMILY);
+ Put put = new Put(Bytes.toBytes(newQueueId.toString()));
+ offsets.forEach((walGroup, offset) -> put.addColumn(QUEUE_FAMILY, Bytes.toBytes(walGroup),
+ Bytes.toBytes(offset.toString())));
+ MultiRowMutationProtos.MutateRowsRequest request =
+ MultiRowMutationProtos.MutateRowsRequest.newBuilder().addCondition(condition)
+ .addMutationRequest(ProtobufUtil.toMutation(MutationType.DELETE, delete))
+ .addMutationRequest(ProtobufUtil.toMutation(MutationType.PUT, put)).build();
+ MultiRowMutationProtos.MutateRowsResponse resp =
+ FutureUtils.get(asyncTable. coprocessorService(
+ MultiRowMutationProtos.MultiRowMutationService::newStub,
+ (stub, controller, done) -> stub.mutateRows(controller, request, done),
+ coprocessorRow));
+ if (resp.getProcessed()) {
+ return offsets;
+ }
+ // if the multi is not processed, which usually the queue has already been claimed by
+ // others, for safety, let's try claiming again, usually the next get operation above will
+ // return an empty map and we will quit the loop.
+ }
+ } catch (IOException e) {
+ throw new ReplicationException(
+ "failed to claimQueue, queueId=" + queueId + ", targetServerName=" + targetServerName, e);
+ }
+ }
+
+ @Override
+ public void removeQueue(ReplicationQueueId queueId) throws ReplicationException {
+ try (Table table = conn.getTable(tableName)) {
+ table.delete(new Delete(Bytes.toBytes(queueId.toString())).addFamily(QUEUE_FAMILY));
+ } catch (IOException e) {
+ throw new ReplicationException("failed to removeQueue, queueId=" + queueId, e);
+ }
+ }
+
+ @Override
+ public void removeAllQueues(String peerId) throws ReplicationException {
+ Scan scan = new Scan().setStartStopRowForPrefixScan(ReplicationQueueId.getScanPrefix(peerId))
+ .addFamily(QUEUE_FAMILY).setFilter(new KeyOnlyFilter());
+ try (Table table = conn.getTable(tableName); ResultScanner scanner = table.getScanner(scan)) {
+ for (;;) {
+ Result result = scanner.next();
+ if (result == null) {
+ break;
+ }
+ table.delete(new Delete(result.getRow()));
+ }
+ } catch (IOException e) {
+ throw new ReplicationException("failed to listAllQueueIds, peerId=" + peerId, e);
+ }
+ }
+
+ @Override
+ public long getLastSequenceId(String encodedRegionName, String peerId)
+ throws ReplicationException {
+ byte[] qual = Bytes.toBytes(encodedRegionName);
+ try (Table table = conn.getTable(tableName)) {
+ Result result =
+ table.get(new Get(Bytes.toBytes(peerId)).addColumn(LAST_SEQUENCE_ID_FAMILY, qual));
+ byte[] lastSeqId = result.getValue(LAST_SEQUENCE_ID_FAMILY, qual);
+ return lastSeqId != null ? Bytes.toLong(lastSeqId) : HConstants.NO_SEQNUM;
+ } catch (IOException e) {
+ throw new ReplicationException("failed to getLastSequenceId, encodedRegionName="
+ + encodedRegionName + ", peerId=" + peerId, e);
+ }
+ }
+
+ @Override
+ public void setLastSequenceIds(String peerId, Map lastSeqIds)
+ throws ReplicationException {
+ // No need CAS and retry here, because it'll call setLastSequenceIds() for disabled peers
+ // only, so no conflict happen.
+ Put put = new Put(Bytes.toBytes(peerId));
+ lastSeqIds.forEach((encodedRegionName, lastSeqId) -> put.addColumn(LAST_SEQUENCE_ID_FAMILY,
+ Bytes.toBytes(encodedRegionName), Bytes.toBytes(lastSeqId)));
+ try (Table table = conn.getTable(tableName)) {
+ table.put(put);
+ } catch (IOException e) {
+ throw new ReplicationException(
+ "failed to setLastSequenceIds, peerId=" + peerId + ", lastSeqIds=" + lastSeqIds, e);
+ }
+ }
+
+ @Override
+ public void removeLastSequenceIds(String peerId) throws ReplicationException {
+ Delete delete = new Delete(Bytes.toBytes(peerId)).addFamily(LAST_SEQUENCE_ID_FAMILY);
+ try (Table table = conn.getTable(tableName)) {
+ table.delete(delete);
+ } catch (IOException e) {
+ throw new ReplicationException("failed to removeLastSequenceIds, peerId=" + peerId, e);
+ }
+ }
+
+ @Override
+ public void removeLastSequenceIds(String peerId, List encodedRegionNames)
+ throws ReplicationException {
+ Delete delete = new Delete(Bytes.toBytes(peerId));
+ encodedRegionNames.forEach(n -> delete.addColumns(LAST_SEQUENCE_ID_FAMILY, Bytes.toBytes(n)));
+ try (Table table = conn.getTable(tableName)) {
+ table.delete(delete);
+ } catch (IOException e) {
+ throw new ReplicationException("failed to removeLastSequenceIds, peerId=" + peerId
+ + ", encodedRegionNames=" + encodedRegionNames, e);
+ }
+ }
+
+ @Override
+ public void removePeerFromHFileRefs(String peerId) throws ReplicationException {
+ try (Table table = conn.getTable(tableName)) {
+ table.delete(new Delete(Bytes.toBytes(peerId)).addFamily(HFILE_REF_FAMILY));
+ } catch (IOException e) {
+ throw new ReplicationException("failed to removePeerFromHFileRefs, peerId=" + peerId, e);
+ }
+ }
+
+ @Override
+ public void addHFileRefs(String peerId, List> pairs)
+ throws ReplicationException {
+ Put put = new Put(Bytes.toBytes(peerId));
+ pairs.forEach(p -> put.addColumn(HFILE_REF_FAMILY, Bytes.toBytes(p.getSecond().getName()),
+ HConstants.EMPTY_BYTE_ARRAY));
+ try (Table table = conn.getTable(tableName)) {
+ table.put(put);
+ } catch (IOException e) {
+ throw new ReplicationException(
+ "failed to addHFileRefs, peerId=" + peerId + ", pairs=" + pairs, e);
+ }
+ }
+
+ @Override
+ public void removeHFileRefs(String peerId, List files) throws ReplicationException {
+ Delete delete = new Delete(Bytes.toBytes(peerId));
+ files.forEach(f -> delete.addColumns(HFILE_REF_FAMILY, Bytes.toBytes(f)));
+ try (Table table = conn.getTable(tableName)) {
+ table.delete(delete);
+ } catch (IOException e) {
+ throw new ReplicationException(
+ "failed to removeHFileRefs, peerId=" + peerId + ", files=" + files, e);
+ }
+ }
+
+ @Override
+ public List getAllPeersFromHFileRefsQueue() throws ReplicationException {
+ List peerIds = new ArrayList<>();
+ Scan scan = new Scan().addFamily(HFILE_REF_FAMILY).setReadType(ReadType.STREAM)
+ .setFilter(new KeyOnlyFilter());
+ try (Table table = conn.getTable(tableName); ResultScanner scanner = table.getScanner(scan)) {
+ for (;;) {
+ Result result = scanner.next();
+ if (result == null) {
+ break;
+ }
+ peerIds.add(Bytes.toString(result.getRow()));
+ }
+ } catch (IOException e) {
+ throw new ReplicationException("failed to getAllPeersFromHFileRefsQueue", e);
+ }
+ return peerIds;
+ }
+
+ private > T scanHFiles(Scan scan, Supplier creator)
+ throws IOException {
+ T files = creator.get();
+ try (Table table = conn.getTable(tableName); ResultScanner scanner = table.getScanner(scan)) {
+ for (;;) {
+ Result result = scanner.next();
+ if (result == null) {
+ break;
+ }
+ CellScanner cellScanner = result.cellScanner();
+ while (cellScanner.advance()) {
+ Cell cell = cellScanner.current();
+ files.add(Bytes.toString(CellUtil.cloneQualifier(cell)));
+ }
+ }
+ }
+ return files;
+ }
+
+ @Override
+ public List getReplicableHFiles(String peerId) throws ReplicationException {
+ // use scan to avoid getting a too large row one time, which may cause a very huge memory usage.
+ Scan scan = new Scan().addFamily(HFILE_REF_FAMILY)
+ .setStartStopRowForPrefixScan(Bytes.toBytes(peerId)).setAllowPartialResults(true);
+ try {
+ return scanHFiles(scan, ArrayList::new);
+ } catch (IOException e) {
+ throw new ReplicationException("failed to getReplicableHFiles, peerId=" + peerId, e);
+ }
+ }
+
+ @Override
+ public Set getAllHFileRefs() throws ReplicationException {
+ Scan scan = new Scan().addFamily(HFILE_REF_FAMILY).setReadType(ReadType.STREAM)
+ .setAllowPartialResults(true);
+ try {
+ return scanHFiles(scan, HashSet::new);
+ } catch (IOException e) {
+ throw new ReplicationException("failed to getAllHFileRefs", e);
+ }
+ }
+
+ @Override
+ public boolean hasData() throws ReplicationException {
+ try {
+ return conn.getAdmin().getDescriptor(tableName) != null;
+ } catch (IOException e) {
+ throw new ReplicationException("failed to get replication queue table", e);
+ }
+ }
+
+ @Override
+ public void batchUpdateQueues(ServerName serverName, List datas)
+ throws ReplicationException {
+ List puts = new ArrayList<>();
+ for (ReplicationQueueData data : datas) {
+ if (data.getOffsets().isEmpty()) {
+ continue;
+ }
+ Put put = new Put(Bytes.toBytes(data.getId().toString()));
+ data.getOffsets().forEach((walGroup, offset) -> {
+ put.addColumn(QUEUE_FAMILY, Bytes.toBytes(walGroup), Bytes.toBytes(offset.toString()));
+ });
+ puts.add(put);
+ }
+ try (Table table = conn.getTable(tableName)) {
+ table.put(puts);
+ } catch (IOException e) {
+ throw new ReplicationException("failed to batch update queues", e);
+ }
+ }
+
+ @Override
+ public void batchUpdateLastSequenceIds(List lastPushedSeqIds)
+ throws ReplicationException {
+ Map peerId2Put = new HashMap<>();
+ for (ZkLastPushedSeqId lastPushedSeqId : lastPushedSeqIds) {
+ peerId2Put
+ .computeIfAbsent(lastPushedSeqId.getPeerId(), peerId -> new Put(Bytes.toBytes(peerId)))
+ .addColumn(LAST_SEQUENCE_ID_FAMILY, Bytes.toBytes(lastPushedSeqId.getEncodedRegionName()),
+ Bytes.toBytes(lastPushedSeqId.getLastPushedSeqId()));
+ }
+ try (Table table = conn.getTable(tableName)) {
+ table
+ .put(peerId2Put.values().stream().filter(p -> !p.isEmpty()).collect(Collectors.toList()));
+ } catch (IOException e) {
+ throw new ReplicationException("failed to batch update last pushed sequence ids", e);
+ }
+ }
+
+ @Override
+ public void batchUpdateHFileRefs(String peerId, List hfileRefs)
+ throws ReplicationException {
+ if (hfileRefs.isEmpty()) {
+ return;
+ }
+ Put put = new Put(Bytes.toBytes(peerId));
+ for (String ref : hfileRefs) {
+ put.addColumn(HFILE_REF_FAMILY, Bytes.toBytes(ref), HConstants.EMPTY_BYTE_ARRAY);
+ }
+ try (Table table = conn.getTable(tableName)) {
+ table.put(put);
+ } catch (IOException e) {
+ throw new ReplicationException("failed to batch update hfile references", e);
+ }
+ }
+
+ @Override
+ public void removeLastSequenceIdsAndHFileRefsBefore(long ts) throws ReplicationException {
+ try (Table table = conn.getTable(tableName);
+ ResultScanner scanner = table.getScanner(new Scan().addFamily(LAST_SEQUENCE_ID_FAMILY)
+ .addFamily(HFILE_REF_FAMILY).setFilter(new KeyOnlyFilter()))) {
+ for (;;) {
+ Result r = scanner.next();
+ if (r == null) {
+ break;
+ }
+ Delete delete = new Delete(r.getRow()).addFamily(LAST_SEQUENCE_ID_FAMILY, ts)
+ .addFamily(HFILE_REF_FAMILY, ts);
+ table.delete(delete);
+ }
+ } catch (IOException e) {
+ throw new ReplicationException(
+ "failed to remove last sequence ids and hfile references before timestamp " + ts, 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
deleted file mode 100644
index f3506ad3555a..000000000000
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java
+++ /dev/null
@@ -1,689 +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 java.util.stream.Collectors.toList;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.SortedSet;
-import java.util.TreeSet;
-import java.util.stream.Collectors;
-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.client.RegionInfo;
-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.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.org.apache.commons.collections4.CollectionUtils;
-
-/**
- * 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";
-
- public static final String ZOOKEEPER_ZNODE_REPLICATION_REGIONS_KEY =
- "zookeeper.znode.replication.regions";
- public static final String ZOOKEEPER_ZNODE_REPLICATION_REGIONS_DEFAULT = "regions";
-
- /**
- * 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;
-
- final String regionsZNode;
-
- 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);
- this.regionsZNode = ZNodePaths.joinZNode(replicationZNode, conf
- .get(ZOOKEEPER_ZNODE_REPLICATION_REGIONS_KEY, ZOOKEEPER_ZNODE_REPLICATION_REGIONS_DEFAULT));
- }
-
- @Override
- public 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);
- }
-
- /**
- *
- * Put all regions under /hbase/replication/regions znode will lead to too many children because
- * of the huge number of regions in real production environment. So here we will distribute the
- * znodes to multiple directories.
- *
- *
- * So the final znode path will be format like this:
- *
- *
- * /hbase/replication/regions/dd/04/e76a6966d4ffa908ed0586764767-100
- *
- *
- * Here the full encoded region name is dd04e76a6966d4ffa908ed0586764767, and we use the first two
- * characters 'dd' as the first level directory name, and use the next two characters '04' as the
- * second level directory name, and the rest part as the prefix of the znode, and the suffix '100'
- * is the peer id.
- *
- * @param encodedRegionName the encoded region name.
- * @param peerId peer id for replication.
- * @return ZNode path to persist the max sequence id that we've pushed for the given region and
- * peer.
- */
- String getSerialReplicationRegionPeerNode(String encodedRegionName, String peerId) {
- if (encodedRegionName == null || encodedRegionName.length() != RegionInfo.MD5_HEX_LENGTH) {
- throw new IllegalArgumentException(
- "Invalid encoded region name: " + encodedRegionName + ", length should be 32.");
- }
- return new StringBuilder(regionsZNode).append(ZNodePaths.ZNODE_PATH_SEPARATOR)
- .append(encodedRegionName, 0, 2).append(ZNodePaths.ZNODE_PATH_SEPARATOR)
- .append(encodedRegionName, 2, 4).append(ZNodePaths.ZNODE_PATH_SEPARATOR)
- .append(encodedRegionName, 4, encodedRegionName.length()).append("-").append(peerId)
- .toString();
- }
-
- @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("{} already deleted when removing log", fileNode);
- } catch (KeeperException e) {
- throw new ReplicationException("Failed to remove wal from queue (serverName=" + serverName
- + ", queueId=" + queueId + ", fileName=" + fileName + ")", e);
- }
- }
-
- private void addLastSeqIdsToOps(String queueId, Map lastSeqIds,
- List listOfOps) throws KeeperException, ReplicationException {
- String peerId = new ReplicationQueueInfo(queueId).getPeerId();
- for (Entry lastSeqEntry : lastSeqIds.entrySet()) {
- String path = getSerialReplicationRegionPeerNode(lastSeqEntry.getKey(), peerId);
- Pair p = getLastSequenceIdWithVersion(lastSeqEntry.getKey(), peerId);
- byte[] data = ZKUtil.positionToByteArray(lastSeqEntry.getValue());
- if (p.getSecond() < 0) { // ZNode does not exist.
- ZKUtil.createWithParents(zookeeper,
- path.substring(0, path.lastIndexOf(ZNodePaths.ZNODE_PATH_SEPARATOR)));
- listOfOps.add(ZKUtilOp.createAndFailSilent(path, data));
- continue;
- }
- // Perform CAS in a specific version v0 (HBASE-20138)
- int v0 = p.getSecond();
- long lastPushedSeqId = p.getFirst();
- if (lastSeqEntry.getValue() <= lastPushedSeqId) {
- continue;
- }
- listOfOps.add(ZKUtilOp.setData(path, data, v0));
- }
- }
-
- @Override
- public void setWALPosition(ServerName serverName, String queueId, String fileName, long position,
- Map lastSeqIds) throws ReplicationException {
- try {
- for (int retry = 0;; retry++) {
- List listOfOps = new ArrayList<>();
- if (position > 0) {
- listOfOps.add(ZKUtilOp.setData(getFileNode(serverName, queueId, fileName),
- ZKUtil.positionToByteArray(position)));
- }
- // Persist the max sequence id(s) of regions for serial replication atomically.
- addLastSeqIdsToOps(queueId, lastSeqIds, listOfOps);
- if (listOfOps.isEmpty()) {
- return;
- }
- try {
- ZKUtil.multiOrSequential(zookeeper, listOfOps, false);
- return;
- } catch (KeeperException.BadVersionException | KeeperException.NodeExistsException e) {
- LOG.warn(
- "Bad version(or node exist) when persist the last pushed sequence id to zookeeper "
- + "storage, Retry = " + retry + ", serverName=" + serverName + ", queueId=" + queueId
- + ", fileName=" + fileName);
- }
- }
- } catch (KeeperException e) {
- throw new ReplicationException("Failed to set log position (serverName=" + serverName
- + ", queueId=" + queueId + ", fileName=" + fileName + ", position=" + position + ")", e);
- }
- }
-
- /**
- * Return the {lastPushedSequenceId, ZNodeDataVersion} pair. if ZNodeDataVersion is -1, it means
- * that the ZNode does not exist.
- */
- protected Pair getLastSequenceIdWithVersion(String encodedRegionName,
- String peerId) throws KeeperException {
- Stat stat = new Stat();
- String path = getSerialReplicationRegionPeerNode(encodedRegionName, peerId);
- byte[] data = ZKUtil.getDataNoWatch(zookeeper, path, stat);
- if (data == null) {
- // ZNode does not exist, so just return version -1 to indicate that no node exist.
- return Pair.newPair(HConstants.NO_SEQNUM, -1);
- }
- try {
- return Pair.newPair(ZKUtil.parseWALPositionFrom(data), stat.getVersion());
- } catch (DeserializationException de) {
- LOG.warn("Failed to parse log position (region=" + encodedRegionName + ", peerId=" + peerId
- + "), data=" + Bytes.toStringBinary(data));
- }
- return Pair.newPair(HConstants.NO_SEQNUM, stat.getVersion());
- }
-
- @Override
- public long getLastSequenceId(String encodedRegionName, String peerId)
- throws ReplicationException {
- try {
- return getLastSequenceIdWithVersion(encodedRegionName, peerId).getFirst();
- } catch (KeeperException e) {
- throw new ReplicationException("Failed to get last pushed sequence id (encodedRegionName="
- + encodedRegionName + ", peerId=" + peerId + ")", e);
- }
- }
-
- @Override
- public void setLastSequenceIds(String peerId, Map lastSeqIds)
- throws ReplicationException {
- try {
- // No need CAS and retry here, because it'll call setLastSequenceIds() for disabled peers
- // only, so no conflict happen.
- List listOfOps = new ArrayList<>();
- for (Entry lastSeqEntry : lastSeqIds.entrySet()) {
- String path = getSerialReplicationRegionPeerNode(lastSeqEntry.getKey(), peerId);
- ZKUtil.createWithParents(zookeeper, path);
- listOfOps.add(ZKUtilOp.setData(path, ZKUtil.positionToByteArray(lastSeqEntry.getValue())));
- }
- if (!listOfOps.isEmpty()) {
- ZKUtil.multiOrSequential(zookeeper, listOfOps, true);
- }
- } catch (KeeperException e) {
- throw new ReplicationException("Failed to set last sequence ids, peerId=" + peerId
- + ", size of lastSeqIds=" + lastSeqIds.size(), e);
- }
- }
-
- @Override
- public void removeLastSequenceIds(String peerId) throws ReplicationException {
- String suffix = "-" + peerId;
- try {
- StringBuilder sb = new StringBuilder(regionsZNode);
- int regionsZNodeLength = regionsZNode.length();
- int levelOneLength = regionsZNodeLength + 3;
- int levelTwoLength = levelOneLength + 3;
- List levelOneDirs = ZKUtil.listChildrenNoWatch(zookeeper, regionsZNode);
- // it is possible that levelOneDirs is null if we haven't write any last pushed sequence ids
- // yet, so we need an extra check here.
- if (CollectionUtils.isEmpty(levelOneDirs)) {
- return;
- }
- for (String levelOne : levelOneDirs) {
- sb.append(ZNodePaths.ZNODE_PATH_SEPARATOR).append(levelOne);
- for (String levelTwo : ZKUtil.listChildrenNoWatch(zookeeper, sb.toString())) {
- sb.append(ZNodePaths.ZNODE_PATH_SEPARATOR).append(levelTwo);
- for (String znode : ZKUtil.listChildrenNoWatch(zookeeper, sb.toString())) {
- if (znode.endsWith(suffix)) {
- sb.append(ZNodePaths.ZNODE_PATH_SEPARATOR).append(znode);
- ZKUtil.deleteNode(zookeeper, sb.toString());
- sb.setLength(levelTwoLength);
- }
- }
- sb.setLength(levelOneLength);
- }
- sb.setLength(regionsZNodeLength);
- }
- } catch (KeeperException e) {
- throw new ReplicationException("Failed to remove all last sequence ids, peerId=" + peerId, e);
- }
- }
-
- @Override
- public void removeLastSequenceIds(String peerId, List encodedRegionNames)
- throws ReplicationException {
- try {
- List listOfOps =
- encodedRegionNames.stream().map(n -> getSerialReplicationRegionPeerNode(n, peerId))
- .map(ZKUtilOp::deleteNodeFailSilent).collect(Collectors.toList());
- ZKUtil.multiOrSequential(zookeeper, listOfOps, true);
- } catch (KeeperException e) {
- throw new ReplicationException("Failed to remove last sequence ids, peerId=" + peerId
- + ", encodedRegionNames.size=" + encodedRegionNames.size(), 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 parse log position (serverName={}, queueId={}, fileName={})", serverName,
- queueId, fileName);
- }
- // if we can not parse the position, start at the beginning of the wal file again
- return 0;
- }
-
- /**
- * This implement must update the cversion of root {@link #queuesZNode}. The optimistic lock of
- * the {@link #getAllWALs()} method is based on the cversion of root {@link #queuesZNode}.
- * @see #getAllWALs() to show the usage of the cversion of root {@link #queuesZNode} .
- */
- @Override
- public Pair> claimQueue(ServerName sourceServerName, String queueId,
- ServerName destServerName) throws ReplicationException {
- LOG.info("Atomically moving {}/{}'s WALs to {}", sourceServerName, queueId, 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);
- }
- String newQueueId = queueId + "-" + sourceServerName;
- try {
- String oldQueueNode = getQueueNode(sourceServerName, queueId);
- List wals = ZKUtil.listChildrenNoWatch(zookeeper, oldQueueNode);
- if (CollectionUtils.isEmpty(wals)) {
- ZKUtil.deleteNodeFailSilent(zookeeper, oldQueueNode);
- LOG.info("Removed empty {}/{}", sourceServerName, queueId);
- 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);
- LOG.debug("Creating {} with data {}", wal, 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));
- // Append new queue id for prevent lock competition in zookeeper server.
- String claimLockZNode = ZNodePaths.joinZNode(queuesZNode, "cversion_" + newQueueId);
- // A trick for update the cversion of root queuesZNode .
- // The optimistic lock of the getAllWALs() method is based on the cversion of root queuesZNode
- listOfOps.add(ZKUtilOp.createAndFailSilent(claimLockZNode, HConstants.EMPTY_BYTE_ARRAY));
- listOfOps.add(ZKUtilOp.deleteNodeFailSilent(claimLockZNode));
-
- LOG.trace("The multi list size is {}", listOfOps.size());
- ZKUtil.multiOrSequential(zookeeper, listOfOps, false);
-
- LOG.info("Atomically moved {}/{}'s WALs to {}", sourceServerName, queueId, 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 return an empty
- // queue to tell the upper layer that claim nothing. For other types of exception should be
- // thrown out to notify the upper layer.
- LOG.info("Claim queue queueId={} from {} to {} failed with {}, someone else took the log?",
- queueId, sourceServerName, destServerName, e.toString());
- return new Pair<>(newQueueId, Collections.emptySortedSet());
- } 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 {
- List children = ZKUtil.listChildrenNoWatch(zookeeper, queuesZNode);
- if (children == null) {
- children = Collections.emptyList();
- }
- return children.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 {
- List children =
- ZKUtil.listChildrenNoWatch(zookeeper, getQueueNode(serverName, queueId));
- return children != null ? children : Collections.emptyList();
- }
-
- @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 {
- List children = ZKUtil.listChildrenNoWatch(zookeeper, getRsNode(serverName));
- return children != null ? children : Collections.emptyList();
- }
-
- @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
- protected int getQueuesZNodeCversion() throws KeeperException {
- Stat stat = new Stat();
- ZKUtil.getDataNoWatch(this.zookeeper, this.queuesZNode, stat);
- return stat.getCversion();
- }
-
- /**
- * The optimistic lock of this implement is based on the cversion of root {@link #queuesZNode}.
- * Therefore, we must update the cversion of root {@link #queuesZNode} when migrate wal nodes to
- * other queues.
- * @see #claimQueue(ServerName, String, ServerName) as an example of updating root
- * {@link #queuesZNode} cversion.
- */
- @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 a RegionServer that replicates, won't prevent 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("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 {} to hfile reference queue.", peerId);
- 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) {
- LOG.debug("Peer {} not found in hfile reference queue.", peerNode);
- } else {
- LOG.info("Removing peer {} from hfile reference queue.", peerNode);
- 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);
- LOG.debug("Adding hfile references {} in queue {}", pairs, 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());
- LOG.debug("The multi list size for adding hfile references in zk for node {} is {}", peerNode,
- 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);
- LOG.debug("Removing hfile references {} from queue {}", files, peerNode);
-
- List listOfOps = files.stream().map(n -> getHFileNode(peerNode, n))
- .map(ZKUtilOp::deleteNodeFailSilent).collect(toList());
- LOG.debug("The multi list size for removing hfile references in zk for node {} is {}", peerNode,
- 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 {
- List children = ZKUtil.listChildrenNoWatch(zookeeper, hfileRefsZNode);
- return children != null ? children : Collections.emptyList();
- }
-
- @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 {
- List children =
- ZKUtil.listChildrenNoWatch(this.zookeeper, getHFileRefsPeerNode(peerId));
- return children != null ? children : Collections.emptyList();
- }
-
- @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
- 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 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("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/ZKReplicationQueueStorageForMigration.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorageForMigration.java
new file mode 100644
index 000000000000..22cc13145225
--- /dev/null
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorageForMigration.java
@@ -0,0 +1,351 @@
+/*
+ * 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 com.google.errorprone.annotations.RestrictedApi;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.util.Pair;
+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.base.Splitter;
+
+/**
+ * Just retain a small set of the methods for the old zookeeper based replication queue storage, for
+ * migrating.
+ */
+@InterfaceAudience.Private
+public class ZKReplicationQueueStorageForMigration extends ZKReplicationStorageBase {
+
+ 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 static final String ZOOKEEPER_ZNODE_REPLICATION_REGIONS_KEY =
+ "zookeeper.znode.replication.regions";
+ public static final String ZOOKEEPER_ZNODE_REPLICATION_REGIONS_DEFAULT = "regions";
+
+ /**
+ * 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;
+
+ private final String regionsZNode;
+
+ public ZKReplicationQueueStorageForMigration(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);
+ this.regionsZNode = ZNodePaths.joinZNode(replicationZNode, conf
+ .get(ZOOKEEPER_ZNODE_REPLICATION_REGIONS_KEY, ZOOKEEPER_ZNODE_REPLICATION_REGIONS_DEFAULT));
+ }
+
+ public interface MigrationIterator {
+
+ T next() throws Exception;
+ }
+
+ @SuppressWarnings("rawtypes")
+ private static final MigrationIterator EMPTY_ITER = new MigrationIterator() {
+
+ @Override
+ public Object next() {
+ return null;
+ }
+ };
+
+ public static final class ZkReplicationQueueData {
+
+ private final ReplicationQueueId queueId;
+
+ private final Map walOffsets;
+
+ public ZkReplicationQueueData(ReplicationQueueId queueId, Map walOffsets) {
+ this.queueId = queueId;
+ this.walOffsets = walOffsets;
+ }
+
+ public ReplicationQueueId getQueueId() {
+ return queueId;
+ }
+
+ public Map getWalOffsets() {
+ return walOffsets;
+ }
+ }
+
+ 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);
+ }
+
+ @SuppressWarnings("unchecked")
+ public MigrationIterator>> listAllQueues()
+ throws KeeperException {
+ List replicators = ZKUtil.listChildrenNoWatch(zookeeper, queuesZNode);
+ if (replicators == null || replicators.isEmpty()) {
+ ZKUtil.deleteNodeRecursively(zookeeper, queuesZNode);
+ return EMPTY_ITER;
+ }
+ Iterator iter = replicators.iterator();
+ return new MigrationIterator>>() {
+
+ private ServerName previousServerName;
+
+ @Override
+ public Pair> next() throws Exception {
+ if (previousServerName != null) {
+ ZKUtil.deleteNodeRecursively(zookeeper, getRsNode(previousServerName));
+ }
+ if (!iter.hasNext()) {
+ ZKUtil.deleteNodeRecursively(zookeeper, queuesZNode);
+ return null;
+ }
+ String replicator = iter.next();
+ ServerName serverName = ServerName.parseServerName(replicator);
+ previousServerName = serverName;
+ List queueIdList = ZKUtil.listChildrenNoWatch(zookeeper, getRsNode(serverName));
+ if (queueIdList == null || queueIdList.isEmpty()) {
+ return Pair.newPair(serverName, Collections.emptyList());
+ }
+ List queueDataList = new ArrayList<>(queueIdList.size());
+ for (String queueIdStr : queueIdList) {
+ ReplicationQueueInfo queueInfo = new ReplicationQueueInfo(queueIdStr);
+ ReplicationQueueId queueId;
+ if (queueInfo.getDeadRegionServers().isEmpty()) {
+ queueId = new ReplicationQueueId(serverName, queueInfo.getPeerId());
+ } else {
+ queueId = new ReplicationQueueId(serverName, queueInfo.getPeerId(),
+ queueInfo.getDeadRegionServers().get(0));
+ }
+ List wals =
+ ZKUtil.listChildrenNoWatch(zookeeper, getQueueNode(serverName, queueIdStr));
+ ZkReplicationQueueData queueData;
+ if (wals == null || wals.isEmpty()) {
+ queueData = new ZkReplicationQueueData(queueId, Collections.emptyMap());
+ } else {
+ Map walOffsets = new HashMap<>();
+ for (String wal : wals) {
+ byte[] data = ZKUtil.getData(zookeeper, getFileNode(serverName, queueIdStr, wal));
+ if (data == null || data.length == 0) {
+ walOffsets.put(wal, 0L);
+ } else {
+ walOffsets.put(wal, ZKUtil.parseWALPositionFrom(data));
+ }
+ }
+ queueData = new ZkReplicationQueueData(queueId, walOffsets);
+ }
+ queueDataList.add(queueData);
+ }
+ return Pair.newPair(serverName, queueDataList);
+ }
+ };
+ }
+
+ public static final class ZkLastPushedSeqId {
+
+ private final String encodedRegionName;
+
+ private final String peerId;
+
+ private final long lastPushedSeqId;
+
+ ZkLastPushedSeqId(String encodedRegionName, String peerId, long lastPushedSeqId) {
+ this.encodedRegionName = encodedRegionName;
+ this.peerId = peerId;
+ this.lastPushedSeqId = lastPushedSeqId;
+ }
+
+ public String getEncodedRegionName() {
+ return encodedRegionName;
+ }
+
+ public String getPeerId() {
+ return peerId;
+ }
+
+ public long getLastPushedSeqId() {
+ return lastPushedSeqId;
+ }
+
+ }
+
+ @SuppressWarnings("unchecked")
+ public MigrationIterator> listAllLastPushedSeqIds()
+ throws KeeperException {
+ List level1Prefixs = ZKUtil.listChildrenNoWatch(zookeeper, regionsZNode);
+ if (level1Prefixs == null || level1Prefixs.isEmpty()) {
+ ZKUtil.deleteNodeRecursively(zookeeper, regionsZNode);
+ return EMPTY_ITER;
+ }
+ Iterator level1Iter = level1Prefixs.iterator();
+ return new MigrationIterator>() {
+
+ private String level1Prefix;
+
+ private Iterator level2Iter;
+
+ private String level2Prefix;
+
+ @Override
+ public List next() throws Exception {
+ for (;;) {
+ if (level2Iter == null || !level2Iter.hasNext()) {
+ if (!level1Iter.hasNext()) {
+ ZKUtil.deleteNodeRecursively(zookeeper, regionsZNode);
+ return null;
+ }
+ if (level1Prefix != null) {
+ // this will also delete the previous level2Prefix which is under this level1Prefix
+ ZKUtil.deleteNodeRecursively(zookeeper,
+ ZNodePaths.joinZNode(regionsZNode, level1Prefix));
+ }
+ level1Prefix = level1Iter.next();
+ List level2Prefixes = ZKUtil.listChildrenNoWatch(zookeeper,
+ ZNodePaths.joinZNode(regionsZNode, level1Prefix));
+ if (level2Prefixes != null) {
+ level2Iter = level2Prefixes.iterator();
+ // reset level2Prefix as we have switched level1Prefix, otherwise the below delete
+ // level2Prefix section will delete the znode with this level2Prefix under the new
+ // level1Prefix
+ level2Prefix = null;
+ }
+ } else {
+ if (level2Prefix != null) {
+ ZKUtil.deleteNodeRecursively(zookeeper,
+ ZNodePaths.joinZNode(regionsZNode, level1Prefix, level2Prefix));
+ }
+ level2Prefix = level2Iter.next();
+ List encodedRegionNameAndPeerIds = ZKUtil.listChildrenNoWatch(zookeeper,
+ ZNodePaths.joinZNode(regionsZNode, level1Prefix, level2Prefix));
+ if (encodedRegionNameAndPeerIds == null || encodedRegionNameAndPeerIds.isEmpty()) {
+ return Collections.emptyList();
+ }
+ List lastPushedSeqIds = new ArrayList<>();
+ for (String encodedRegionNameAndPeerId : encodedRegionNameAndPeerIds) {
+ byte[] data = ZKUtil.getData(zookeeper, ZNodePaths.joinZNode(regionsZNode,
+ level1Prefix, level2Prefix, encodedRegionNameAndPeerId));
+ long lastPushedSeqId = ZKUtil.parseWALPositionFrom(data);
+ Iterator iter = Splitter.on('-').split(encodedRegionNameAndPeerId).iterator();
+ String encodedRegionName = level1Prefix + level2Prefix + iter.next();
+ String peerId = iter.next();
+ lastPushedSeqIds
+ .add(new ZkLastPushedSeqId(encodedRegionName, peerId, lastPushedSeqId));
+ }
+ return Collections.unmodifiableList(lastPushedSeqIds);
+ }
+ }
+ }
+ };
+ }
+
+ private String getHFileRefsPeerNode(String peerId) {
+ return ZNodePaths.joinZNode(hfileRefsZNode, peerId);
+ }
+
+ /**
+ * Pair<PeerId, List<HFileRefs>>
+ */
+ @SuppressWarnings("unchecked")
+ public MigrationIterator>> listAllHFileRefs() throws KeeperException {
+ List peerIds = ZKUtil.listChildrenNoWatch(zookeeper, hfileRefsZNode);
+ if (peerIds == null || peerIds.isEmpty()) {
+ ZKUtil.deleteNodeRecursively(zookeeper, hfileRefsZNode);
+ return EMPTY_ITER;
+ }
+ Iterator iter = peerIds.iterator();
+ return new MigrationIterator>>() {
+
+ private String previousPeerId;
+
+ @Override
+ public Pair> next() throws KeeperException {
+ if (previousPeerId != null) {
+ ZKUtil.deleteNodeRecursively(zookeeper, getHFileRefsPeerNode(previousPeerId));
+ }
+ if (!iter.hasNext()) {
+ ZKUtil.deleteNodeRecursively(zookeeper, hfileRefsZNode);
+ return null;
+ }
+ String peerId = iter.next();
+ List refs = ZKUtil.listChildrenNoWatch(zookeeper, getHFileRefsPeerNode(peerId));
+ previousPeerId = peerId;
+ return Pair.newPair(peerId, refs != null ? refs : Collections.emptyList());
+ }
+ };
+ }
+
+ public boolean hasData() throws KeeperException {
+ return ZKUtil.checkExists(zookeeper, queuesZNode) != -1
+ || ZKUtil.checkExists(zookeeper, regionsZNode) != -1
+ || ZKUtil.checkExists(zookeeper, hfileRefsZNode) != -1;
+ }
+
+ public void deleteAllData() throws KeeperException {
+ ZKUtil.deleteNodeRecursively(zookeeper, queuesZNode);
+ ZKUtil.deleteNodeRecursively(zookeeper, regionsZNode);
+ ZKUtil.deleteNodeRecursively(zookeeper, hfileRefsZNode);
+ }
+
+ @RestrictedApi(explanation = "Should only be called in tests", link = "",
+ allowedOnPath = ".*/src/test/.*")
+ String getQueuesZNode() {
+ return queuesZNode;
+ }
+
+ @RestrictedApi(explanation = "Should only be called in tests", link = "",
+ allowedOnPath = ".*/src/test/.*")
+ String getHfileRefsZNode() {
+ return hfileRefsZNode;
+ }
+
+ @RestrictedApi(explanation = "Should only be called in tests", link = "",
+ allowedOnPath = ".*/src/test/.*")
+ String getRegionsZNode() {
+ return regionsZNode;
+ }
+}
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
index 15cf5b1f1f64..dc46e4f1c7c8 100644
--- 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
@@ -17,30 +17,18 @@
*/
package org.apache.hadoop.hbase.replication;
-import static org.hamcrest.CoreMatchers.hasItems;
-import static org.hamcrest.MatcherAssert.assertThat;
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.Collections;
-import java.util.List;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HConstants;
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.MiniZooKeeperCluster;
import org.apache.hadoop.hbase.zookeeper.ZKConfig;
-import org.apache.zookeeper.KeeperException;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableMap;
-
/**
* White box testing for replication state interfaces. Implementations should extend this class, and
* initialize the interfaces properly.
@@ -49,7 +37,6 @@ 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);
@@ -66,161 +53,6 @@ public abstract class TestReplicationStateBasic {
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, Collections.emptyMap());
- 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,
- SyncReplicationState.NONE);
- 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,
- SyncReplicationState.NONE);
- rqs.addPeerToHFileRefs(ID_ONE);
- rp.getPeerStorage().addPeer(ID_TWO,
- ReplicationPeerConfig.newBuilder().setClusterKey(KEY_TWO).build(), true,
- SyncReplicationState.NONE);
- 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();
@@ -286,55 +118,7 @@ public void testReplicationPeers() throws Exception {
assertNumberOfPeers(2);
}
- private String getFileName(String base, int i) {
- return String.format(base + "-%04d", i);
- }
-
- @Test
- public void testPersistLogPositionAndSeqIdAtomically() throws Exception {
- ServerName serverName1 = ServerName.valueOf("127.0.0.1", 8000, 10000);
- assertTrue(rqs.getAllQueues(serverName1).isEmpty());
- String queue1 = "1";
- String region0 = "6b2c8f8555335cc9af74455b94516cbe",
- region1 = "6ecd2e9e010499f8ddef97ee8f70834f";
- for (int i = 0; i < 10; i++) {
- rqs.addWAL(serverName1, queue1, getFileName("file1", i));
- }
- List queueIds = rqs.getAllQueues(serverName1);
- assertEquals(1, queueIds.size());
- assertThat(queueIds, hasItems("1"));
-
- List wals1 = rqs.getWALsInQueue(serverName1, queue1);
- assertEquals(10, wals1.size());
- for (int i = 0; i < 10; i++) {
- assertThat(wals1, hasItems(getFileName("file1", i)));
- }
-
- for (int i = 0; i < 10; i++) {
- assertEquals(0, rqs.getWALPosition(serverName1, queue1, getFileName("file1", i)));
- }
- assertEquals(HConstants.NO_SEQNUM, rqs.getLastSequenceId(region0, queue1));
- assertEquals(HConstants.NO_SEQNUM, rqs.getLastSequenceId(region1, queue1));
-
- for (int i = 0; i < 10; i++) {
- rqs.setWALPosition(serverName1, queue1, getFileName("file1", i), (i + 1) * 100,
- ImmutableMap.of(region0, i * 100L, region1, (i + 1) * 100L));
- }
-
- for (int i = 0; i < 10; i++) {
- assertEquals((i + 1) * 100, rqs.getWALPosition(serverName1, queue1, getFileName("file1", i)));
- }
- assertEquals(900L, rqs.getLastSequenceId(region0, queue1));
- assertEquals(1000L, rqs.getLastSequenceId(region1, queue1));
-
- // Try to decrease the last pushed id by setWALPosition method.
- rqs.setWALPosition(serverName1, queue1, getFileName("file1", 0), 11 * 100,
- ImmutableMap.of(region0, 899L, region1, 1001L));
- assertEquals(900L, rqs.getLastSequenceId(region0, queue1));
- assertEquals(1001L, rqs.getLastSequenceId(region1, queue1));
- }
-
- protected void assertConnectedPeerStatus(boolean status, String peerId) throws Exception {
+ private 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");
@@ -353,30 +137,7 @@ protected void assertConnectedPeerStatus(boolean status, String peerId) throws E
}
}
- protected void assertNumberOfPeers(int total) throws ReplicationException {
+ private 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(MiniZooKeeperCluster.HOST + ":2818:/bogus" + i).build(),
- true, SyncReplicationState.NONE);
- }
- }
}
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
index d2540987906a..c5c8769282a6 100644
--- 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
@@ -80,7 +80,6 @@ private static String initPeerClusterState(String baseZKNode)
@Before
public void setUp() throws IOException {
zkTimeoutCount = 0;
- rqs = ReplicationStorageFactory.getReplicationQueueStorage(zkw, conf);
rp =
ReplicationFactory.getReplicationPeers(FileSystem.get(utility.getConfiguration()), zkw, conf);
OUR_KEY = ZKConfig.getZooKeeperClusterKey(conf);
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
index ccd3c17f3bca..e38b7b134e99 100644
--- 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
@@ -17,37 +17,50 @@
*/
package org.apache.hadoop.hbase.replication;
-import static org.hamcrest.CoreMatchers.hasItems;
import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.empty;
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 java.io.IOException;
-import java.util.Arrays;
-import java.util.Collections;
+import java.util.HashMap;
import java.util.List;
+import java.util.Map;
import java.util.Set;
-import java.util.SortedSet;
-import org.apache.hadoop.fs.Path;
+import java.util.concurrent.ThreadLocalRandom;
+import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HBaseZKTestingUtil;
-import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.replication.ZKReplicationQueueStorageForMigration.MigrationIterator;
+import org.apache.hadoop.hbase.replication.ZKReplicationQueueStorageForMigration.ZkLastPushedSeqId;
+import org.apache.hadoop.hbase.replication.ZKReplicationQueueStorageForMigration.ZkReplicationQueueData;
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.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.MD5Hash;
import org.apache.hadoop.hbase.util.Pair;
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.ClassRule;
+import org.junit.Rule;
import org.junit.Test;
import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
-import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableMap;
+import org.apache.hbase.thirdparty.com.google.common.base.Splitter;
+import org.apache.hbase.thirdparty.com.google.common.collect.Iterables;
+import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
+import org.apache.hbase.thirdparty.com.google.common.io.Closeables;
@Category({ ReplicationTests.class, MediumTests.class })
public class TestZKReplicationQueueStorage {
@@ -58,284 +71,247 @@ public class TestZKReplicationQueueStorage {
private static final HBaseZKTestingUtil UTIL = new HBaseZKTestingUtil();
- private static ZKReplicationQueueStorage STORAGE;
+ private ZKWatcher zk;
+
+ private ZKReplicationQueueStorageForMigration storage;
+
+ @Rule
+ public final TestName name = new TestName();
@BeforeClass
- public static void setUp() throws Exception {
+ public static void setUpBeforeClass() throws Exception {
UTIL.startMiniZKCluster();
- STORAGE = new ZKReplicationQueueStorage(UTIL.getZooKeeperWatcher(), UTIL.getConfiguration());
}
@AfterClass
- public static void tearDown() throws IOException {
+ public static void tearDownAfterClass() throws IOException {
UTIL.shutdownMiniZKCluster();
}
- @After
- public void tearDownAfterTest() throws ReplicationException, KeeperException, IOException {
- 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);
- }
+ @Before
+ public void setUp() throws IOException {
+ Configuration conf = UTIL.getConfiguration();
+ conf.set(ZKReplicationStorageBase.REPLICATION_ZNODE, name.getMethodName());
+ zk = new ZKWatcher(conf, name.getMethodName(), null);
+ storage = new ZKReplicationQueueStorageForMigration(zk, conf);
}
- private ServerName getServerName(int i) {
- return ServerName.valueOf("127.0.0.1", 8000 + i, 10000 + i);
+ @After
+ public void tearDown() throws Exception {
+ ZKUtil.deleteNodeRecursively(zk, storage.replicationZNode);
+ Closeables.close(zk, true);
}
- @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)));
+ public static void mockQueuesData(ZKReplicationQueueStorageForMigration storage, int nServers,
+ String peerId, ServerName deadServer) throws KeeperException {
+ ZKWatcher zk = storage.zookeeper;
+ for (int i = 0; i < nServers; i++) {
+ ServerName sn =
+ ServerName.valueOf("test-hbase-" + i, 12345, EnvironmentEdgeManager.currentTime());
+ String rsZNode = ZNodePaths.joinZNode(storage.getQueuesZNode(), sn.toString());
+ String peerZNode = ZNodePaths.joinZNode(rsZNode, peerId);
+ ZKUtil.createWithParents(zk, peerZNode);
+ for (int j = 0; j < i; j++) {
+ String wal = ZNodePaths.joinZNode(peerZNode, sn.toString() + "." + j);
+ ZKUtil.createSetData(zk, wal, ZKUtil.positionToByteArray(j));
+ }
+ String deadServerPeerZNode = ZNodePaths.joinZNode(rsZNode, peerId + "-" + deadServer);
+ ZKUtil.createWithParents(zk, deadServerPeerZNode);
+ for (int j = 0; j < i; j++) {
+ String wal = ZNodePaths.joinZNode(deadServerPeerZNode, deadServer.toString() + "." + j);
+ if (j > 0) {
+ ZKUtil.createSetData(zk, wal, ZKUtil.positionToByteArray(j));
+ } else {
+ ZKUtil.createWithParents(zk, wal);
+ }
+ }
}
+ ZKUtil.createWithParents(zk,
+ ZNodePaths.joinZNode(storage.getQueuesZNode(), deadServer.toString()));
}
- private String getFileName(String base, int i) {
- return String.format(base + "-%04d", i);
+ private static String getLastPushedSeqIdZNode(String regionsZNode, String encodedName,
+ String peerId) {
+ return ZNodePaths.joinZNode(regionsZNode, encodedName.substring(0, 2),
+ encodedName.substring(2, 4), encodedName.substring(4) + "-" + peerId);
}
- @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, wals2.size());
- for (int i = 0; i < 10; i++) {
- assertThat(wals1, hasItems(getFileName("file1", i)));
- assertThat(wals2, hasItems(getFileName("file2", i)));
+ public static Map> mockLastPushedSeqIds(
+ ZKReplicationQueueStorageForMigration storage, String peerId1, String peerId2, int nRegions,
+ int emptyLevel1Count, int emptyLevel2Count) throws KeeperException {
+ ZKWatcher zk = storage.zookeeper;
+ Map> name2PeerIds = new HashMap<>();
+ byte[] bytes = new byte[32];
+ for (int i = 0; i < nRegions; i++) {
+ ThreadLocalRandom.current().nextBytes(bytes);
+ String encodeName = MD5Hash.getMD5AsHex(bytes);
+ String znode1 = getLastPushedSeqIdZNode(storage.getRegionsZNode(), encodeName, peerId1);
+ ZKUtil.createSetData(zk, znode1, ZKUtil.positionToByteArray(1));
+ String znode2 = getLastPushedSeqIdZNode(storage.getRegionsZNode(), encodeName, peerId2);
+ ZKUtil.createSetData(zk, znode2, ZKUtil.positionToByteArray(2));
+ name2PeerIds.put(encodeName, Sets.newHashSet(peerId1, peerId2));
}
-
- 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,
- Collections.emptyMap());
- STORAGE.setWALPosition(serverName1, queue2, getFileName("file2", i), (i + 1) * 100 + 10,
- Collections.emptyMap());
- }
-
- 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));
+ int addedEmptyZNodes = 0;
+ for (int i = 0; i < 256; i++) {
+ String level1ZNode =
+ ZNodePaths.joinZNode(storage.getRegionsZNode(), String.format("%02x", i));
+ if (ZKUtil.checkExists(zk, level1ZNode) == -1) {
+ ZKUtil.createWithParents(zk, level1ZNode);
+ addedEmptyZNodes++;
+ if (addedEmptyZNodes <= emptyLevel2Count) {
+ ZKUtil.createWithParents(zk, ZNodePaths.joinZNode(level1ZNode, "ab"));
+ }
+ if (addedEmptyZNodes >= emptyLevel1Count + emptyLevel2Count) {
+ break;
+ }
}
}
-
- 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)));
- }
+ return name2PeerIds;
}
- // For HBASE-12865, HBASE-26482
- @Test
- public void testClaimQueueChangeCversion() throws ReplicationException, KeeperException {
- ServerName serverName1 = ServerName.valueOf("127.0.0.1", 8000, 10000);
- STORAGE.addWAL(serverName1, "1", "file");
- STORAGE.addWAL(serverName1, "2", "file");
-
- ServerName serverName2 = ServerName.valueOf("127.0.0.1", 8001, 10001);
- // Avoid claimQueue update cversion for prepare server2 rsNode.
- STORAGE.addWAL(serverName2, "1", "file");
- STORAGE.addWAL(serverName2, "2", "file");
-
- int v0 = STORAGE.getQueuesZNodeCversion();
-
- STORAGE.claimQueue(serverName1, "1", serverName2);
- int v1 = STORAGE.getQueuesZNodeCversion();
- // cversion should be increased by claimQueue method.
- assertTrue(v1 > v0);
-
- STORAGE.claimQueue(serverName1, "2", serverName2);
- int v2 = STORAGE.getQueuesZNodeCversion();
- // cversion should be increased by claimQueue method.
- assertTrue(v2 > v1);
- }
-
- private ZKReplicationQueueStorage createWithUnstableVersion() throws IOException {
- return new ZKReplicationQueueStorage(UTIL.getZooKeeperWatcher(), UTIL.getConfiguration()) {
-
- private int called = 0;
- private int getLastSeqIdOpIndex = 0;
-
- @Override
- protected int getQueuesZNodeCversion() throws KeeperException {
- if (called < 4) {
- called++;
- }
- return called;
+ public static void mockHFileRefs(ZKReplicationQueueStorageForMigration storage, int nPeers)
+ throws KeeperException {
+ ZKWatcher zk = storage.zookeeper;
+ for (int i = 0; i < nPeers; i++) {
+ String peerId = "peer_" + i;
+ ZKUtil.createWithParents(zk, ZNodePaths.joinZNode(storage.getHfileRefsZNode(), peerId));
+ for (int j = 0; j < i; j++) {
+ ZKUtil.createWithParents(zk,
+ ZNodePaths.joinZNode(storage.getHfileRefsZNode(), peerId, "hfile-" + j));
}
-
- @Override
- protected Pair getLastSequenceIdWithVersion(String encodedRegionName,
- String peerId) throws KeeperException {
- Pair oldPair = super.getLastSequenceIdWithVersion(encodedRegionName, peerId);
- if (getLastSeqIdOpIndex < 100) {
- // Let the ZNode version increase.
- String path = getSerialReplicationRegionPeerNode(encodedRegionName, peerId);
- ZKUtil.createWithParents(zookeeper, path);
- ZKUtil.setData(zookeeper, path, ZKUtil.positionToByteArray(100L));
- }
- getLastSeqIdOpIndex++;
- return oldPair;
- }
- };
+ }
}
@Test
- public void testGetAllWALsCversionChange() throws IOException, ReplicationException {
- ZKReplicationQueueStorage storage = createWithUnstableVersion();
- 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"));
+ public void testDeleteAllData() throws Exception {
+ assertFalse(storage.hasData());
+ ZKUtil.createWithParents(zk, storage.getQueuesZNode());
+ assertTrue(storage.hasData());
+ storage.deleteAllData();
+ assertFalse(storage.hasData());
}
- // For HBASE-14621
@Test
- public void testGetAllHFileRefsCversionChange() throws IOException, ReplicationException {
- ZKReplicationQueueStorage storage = createWithUnstableVersion();
- 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"));
+ public void testEmptyIter() throws Exception {
+ ZKUtil.createWithParents(zk, storage.getQueuesZNode());
+ ZKUtil.createWithParents(zk, storage.getRegionsZNode());
+ ZKUtil.createWithParents(zk, storage.getHfileRefsZNode());
+ assertNull(storage.listAllQueues().next());
+ assertEquals(-1, ZKUtil.checkExists(zk, storage.getQueuesZNode()));
+ assertNull(storage.listAllLastPushedSeqIds().next());
+ assertEquals(-1, ZKUtil.checkExists(zk, storage.getRegionsZNode()));
+ assertNull(storage.listAllHFileRefs().next());
+ assertEquals(-1, ZKUtil.checkExists(zk, storage.getHfileRefsZNode()));
}
- // For HBASE-20138
@Test
- public void testSetWALPositionBadVersion() throws IOException, ReplicationException {
- ZKReplicationQueueStorage storage = createWithUnstableVersion();
- ServerName serverName1 = ServerName.valueOf("128.0.0.1", 8000, 10000);
- assertTrue(storage.getAllQueues(serverName1).isEmpty());
- String queue1 = "1";
- String fileName = getFileName("file1", 0);
- String encodedRegionName = "31d9792f4435b99d9fb1016f6fbc8dc6";
- storage.addWAL(serverName1, queue1, fileName);
-
- List wals1 = storage.getWALsInQueue(serverName1, queue1);
- assertEquals(1, wals1.size());
-
- assertEquals(0, storage.getWALPosition(serverName1, queue1, fileName));
- // This should return eventually when data version stabilizes
- storage.setWALPosition(serverName1, queue1, fileName, 100,
- ImmutableMap.of(encodedRegionName, 120L));
-
- assertEquals(100, storage.getWALPosition(serverName1, queue1, fileName));
- assertEquals(120L, storage.getLastSequenceId(encodedRegionName, queue1));
+ public void testListAllQueues() throws Exception {
+ String peerId = "1";
+ ServerName deadServer =
+ ServerName.valueOf("test-hbase-dead", 12345, EnvironmentEdgeManager.currentTime());
+ int nServers = 10;
+ mockQueuesData(storage, nServers, peerId, deadServer);
+ MigrationIterator>> iter =
+ storage.listAllQueues();
+ ServerName previousServerName = null;
+ for (int i = 0; i < nServers + 1; i++) {
+ Pair> pair = iter.next();
+ assertNotNull(pair);
+ if (previousServerName != null) {
+ assertEquals(-1, ZKUtil.checkExists(zk,
+ ZNodePaths.joinZNode(storage.getQueuesZNode(), previousServerName.toString())));
+ }
+ ServerName sn = pair.getFirst();
+ previousServerName = sn;
+ if (sn.equals(deadServer)) {
+ assertThat(pair.getSecond(), empty());
+ } else {
+ assertEquals(2, pair.getSecond().size());
+ int n = Integer.parseInt(Iterables.getLast(Splitter.on('-').split(sn.getHostname())));
+ ZkReplicationQueueData data0 = pair.getSecond().get(0);
+ assertEquals(peerId, data0.getQueueId().getPeerId());
+ assertEquals(sn, data0.getQueueId().getServerName());
+ assertEquals(n, data0.getWalOffsets().size());
+ for (int j = 0; j < n; j++) {
+ assertEquals(j,
+ data0.getWalOffsets().get(
+ (data0.getQueueId().isRecovered() ? deadServer.toString() : sn.toString()) + "." + j)
+ .intValue());
+ }
+ ZkReplicationQueueData data1 = pair.getSecond().get(1);
+ assertEquals(peerId, data1.getQueueId().getPeerId());
+ assertEquals(sn, data1.getQueueId().getServerName());
+ assertEquals(n, data1.getWalOffsets().size());
+ for (int j = 0; j < n; j++) {
+ assertEquals(j,
+ data1.getWalOffsets().get(
+ (data1.getQueueId().isRecovered() ? deadServer.toString() : sn.toString()) + "." + j)
+ .intValue());
+ }
+ // the order of the returned result is undetermined
+ if (data0.getQueueId().getSourceServerName().isPresent()) {
+ assertEquals(deadServer, data0.getQueueId().getSourceServerName().get());
+ assertFalse(data1.getQueueId().getSourceServerName().isPresent());
+ } else {
+ assertEquals(deadServer, data1.getQueueId().getSourceServerName().get());
+ }
+ }
+ }
+ assertNull(iter.next());
+ assertEquals(-1, ZKUtil.checkExists(zk, storage.getQueuesZNode()));
}
@Test
- public void testRegionsZNodeLayout() throws Exception {
- String peerId = "1";
- String encodedRegionName = "31d9792f4435b99d9fb1016f6fbc8dc7";
- String expectedPath = "/hbase/replication/regions/31/d9/792f4435b99d9fb1016f6fbc8dc7-" + peerId;
- String path = STORAGE.getSerialReplicationRegionPeerNode(encodedRegionName, peerId);
- assertEquals(expectedPath, path);
+ public void testListAllLastPushedSeqIds() throws Exception {
+ String peerId1 = "1";
+ String peerId2 = "2";
+ Map> name2PeerIds =
+ mockLastPushedSeqIds(storage, peerId1, peerId2, 100, 10, 10);
+ MigrationIterator> iter = storage.listAllLastPushedSeqIds();
+ int emptyListCount = 0;
+ for (;;) {
+ List list = iter.next();
+ if (list == null) {
+ break;
+ }
+ if (list.isEmpty()) {
+ emptyListCount++;
+ continue;
+ }
+ for (ZkLastPushedSeqId seqId : list) {
+ name2PeerIds.get(seqId.getEncodedRegionName()).remove(seqId.getPeerId());
+ if (seqId.getPeerId().equals(peerId1)) {
+ assertEquals(1, seqId.getLastPushedSeqId());
+ } else {
+ assertEquals(2, seqId.getLastPushedSeqId());
+ }
+ }
+ }
+ assertEquals(10, emptyListCount);
+ name2PeerIds.forEach((encodedRegionName, peerIds) -> {
+ assertThat(encodedRegionName + " still has unmigrated peers", peerIds, empty());
+ });
+ assertEquals(-1, ZKUtil.checkExists(zk, storage.getRegionsZNode()));
}
@Test
- public void testRemoveAllLastPushedSeqIdsForPeer() throws Exception {
- String peerId = "1";
- String peerIdToDelete = "2";
- for (int i = 0; i < 100; i++) {
- String encodedRegionName = MD5Hash.getMD5AsHex(Bytes.toBytes(i));
- STORAGE.setLastSequenceIds(peerId, ImmutableMap.of(encodedRegionName, (long) i));
- STORAGE.setLastSequenceIds(peerIdToDelete, ImmutableMap.of(encodedRegionName, (long) i));
- }
- for (int i = 0; i < 100; i++) {
- String encodedRegionName = MD5Hash.getMD5AsHex(Bytes.toBytes(i));
- assertEquals(i, STORAGE.getLastSequenceId(encodedRegionName, peerId));
- assertEquals(i, STORAGE.getLastSequenceId(encodedRegionName, peerIdToDelete));
- }
- STORAGE.removeLastSequenceIds(peerIdToDelete);
- for (int i = 0; i < 100; i++) {
- String encodedRegionName = MD5Hash.getMD5AsHex(Bytes.toBytes(i));
- assertEquals(i, STORAGE.getLastSequenceId(encodedRegionName, peerId));
- assertEquals(HConstants.NO_SEQNUM,
- STORAGE.getLastSequenceId(encodedRegionName, peerIdToDelete));
+ public void testListAllHFileRefs() throws Exception {
+ int nPeers = 10;
+ mockHFileRefs(storage, nPeers);
+ MigrationIterator>> iter = storage.listAllHFileRefs();
+ String previousPeerId = null;
+ for (int i = 0; i < nPeers; i++) {
+ Pair> pair = iter.next();
+ if (previousPeerId != null) {
+ assertEquals(-1, ZKUtil.checkExists(zk,
+ ZNodePaths.joinZNode(storage.getHfileRefsZNode(), previousPeerId)));
+ }
+ String peerId = pair.getFirst();
+ previousPeerId = peerId;
+ int index = Integer.parseInt(Iterables.getLast(Splitter.on('_').split(peerId)));
+ assertEquals(index, pair.getSecond().size());
}
+ assertNull(iter.next());
+ assertEquals(-1, ZKUtil.checkExists(zk, storage.getHfileRefsZNode()));
}
}
diff --git a/hbase-server/pom.xml b/hbase-server/pom.xml
index 0dba4aa98339..b61b0252a052 100644
--- a/hbase-server/pom.xml
+++ b/hbase-server/pom.xml
@@ -102,6 +102,12 @@
org.apache.hbase
hbase-replication
+
+ org.apache.hbase
+ hbase-replication
+ test-jar
+ test
+
org.apache.hbase
hbase-balancer
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 f3d4dfb292d3..3d59db245015 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
@@ -34,6 +34,9 @@
import java.net.InetAddress;
import java.net.InetSocketAddress;
import java.net.UnknownHostException;
+import java.time.Instant;
+import java.time.ZoneId;
+import java.time.format.DateTimeFormatter;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
@@ -59,6 +62,7 @@
import javax.servlet.http.HttpServlet;
import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.CatalogFamilyFormat;
@@ -170,6 +174,7 @@
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.MigrateReplicationQueueFromZkToTableProcedure;
import org.apache.hadoop.hbase.master.replication.RemovePeerProcedure;
import org.apache.hadoop.hbase.master.replication.ReplicationPeerManager;
import org.apache.hadoop.hbase.master.replication.ReplicationPeerModificationStateStore;
@@ -221,9 +226,12 @@
import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
import org.apache.hadoop.hbase.replication.ReplicationUtils;
import org.apache.hadoop.hbase.replication.SyncReplicationState;
+import org.apache.hadoop.hbase.replication.ZKReplicationQueueStorageForMigration;
import org.apache.hadoop.hbase.replication.master.ReplicationHFileCleaner;
import org.apache.hadoop.hbase.replication.master.ReplicationLogCleaner;
import org.apache.hadoop.hbase.replication.master.ReplicationSinkTrackerTableCreator;
+import org.apache.hadoop.hbase.replication.regionserver.ReplicationSyncUp;
+import org.apache.hadoop.hbase.replication.regionserver.ReplicationSyncUp.ReplicationSyncUpToolInfo;
import org.apache.hadoop.hbase.rsgroup.RSGroupAdminEndpoint;
import org.apache.hadoop.hbase.rsgroup.RSGroupBasedLoadBalancer;
import org.apache.hadoop.hbase.rsgroup.RSGroupInfoManager;
@@ -244,6 +252,7 @@
import org.apache.hadoop.hbase.util.HFileArchiveUtil;
import org.apache.hadoop.hbase.util.IdLock;
import org.apache.hadoop.hbase.util.JVMClusterUtil;
+import org.apache.hadoop.hbase.util.JsonMapper;
import org.apache.hadoop.hbase.util.ModifyRegionUtils;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.RetryCounter;
@@ -265,7 +274,9 @@
import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
+import org.apache.hbase.thirdparty.com.google.common.io.ByteStreams;
import org.apache.hbase.thirdparty.com.google.common.io.Closeables;
+import org.apache.hbase.thirdparty.com.google.gson.JsonParseException;
import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors;
import org.apache.hbase.thirdparty.com.google.protobuf.Service;
import org.apache.hbase.thirdparty.org.eclipse.jetty.server.Server;
@@ -788,8 +799,7 @@ private void initializeZKBasedSystemTrackers()
}
this.rsGroupInfoManager = RSGroupInfoManager.create(this);
- this.replicationPeerManager =
- ReplicationPeerManager.create(fileSystemManager.getFileSystem(), zooKeeper, conf, clusterId);
+ this.replicationPeerManager = ReplicationPeerManager.create(this, clusterId);
this.configurationManager.registerObserver(replicationPeerManager);
this.replicationPeerModificationStateStore =
new ReplicationPeerModificationStateStore(masterRegion);
@@ -1059,6 +1069,17 @@ private void finishActiveMasterInitialization() throws IOException, InterruptedE
this.balancer.initialize();
this.balancer.updateClusterMetrics(getClusterMetricsWithoutCoprocessor());
+ // try migrate replication data
+ ZKReplicationQueueStorageForMigration oldReplicationQueueStorage =
+ new ZKReplicationQueueStorageForMigration(zooKeeper, conf);
+ // check whether there are something to migrate and we haven't scheduled a migration procedure
+ // yet
+ if (
+ oldReplicationQueueStorage.hasData() && procedureExecutor.getProcedures().stream()
+ .allMatch(p -> !(p instanceof MigrateReplicationQueueFromZkToTableProcedure))
+ ) {
+ procedureExecutor.submitProcedure(new MigrateReplicationQueueFromZkToTableProcedure());
+ }
// start up all service threads.
startupTaskGroup.addTask("Initializing master service threads");
startServiceThreads();
@@ -1274,6 +1295,38 @@ private void finishActiveMasterInitialization() throws IOException, InterruptedE
status.setStatus("Initializing MOB Cleaner");
initMobCleaner();
+ // delete the stale data for replication sync up tool if necessary
+ status.setStatus("Cleanup ReplicationSyncUp status if necessary");
+ Path replicationSyncUpInfoFile =
+ new Path(new Path(dataRootDir, ReplicationSyncUp.INFO_DIR), ReplicationSyncUp.INFO_FILE);
+ if (dataFs.exists(replicationSyncUpInfoFile)) {
+ // info file is available, load the timestamp and use it to clean up stale data in replication
+ // queue storage.
+ byte[] data;
+ try (FSDataInputStream in = dataFs.open(replicationSyncUpInfoFile)) {
+ data = ByteStreams.toByteArray(in);
+ }
+ ReplicationSyncUpToolInfo info = null;
+ try {
+ info = JsonMapper.fromJson(Bytes.toString(data), ReplicationSyncUpToolInfo.class);
+ } catch (JsonParseException e) {
+ // usually this should be a partial file, which means the ReplicationSyncUp tool did not
+ // finish properly, so not a problem. Here we do not clean up the status as we do not know
+ // the reason why the tool did not finish properly, so let users clean the status up
+ // manually
+ LOG.warn("failed to parse replication sync up info file, ignore and continue...", e);
+ }
+ if (info != null) {
+ LOG.info("Remove last sequence ids and hfile references which are written before {}({})",
+ info.getStartTimeMs(), DateTimeFormatter.ISO_DATE_TIME.withZone(ZoneId.systemDefault())
+ .format(Instant.ofEpochMilli(info.getStartTimeMs())));
+ replicationPeerManager.getQueueStorage()
+ .removeLastSequenceIdsAndHFileRefsBefore(info.getStartTimeMs());
+ // delete the file after removing the stale data, so next time we do not need to do this
+ // again.
+ dataFs.delete(replicationSyncUpInfoFile, false);
+ }
+ }
status.setStatus("Calling postStartMaster coprocessors");
if (this.cpHost != null) {
// don't let cp initialization errors kill the master
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/FileCleanerDelegate.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/FileCleanerDelegate.java
index d37bb6202730..e08f53294336 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/FileCleanerDelegate.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/FileCleanerDelegate.java
@@ -50,7 +50,7 @@ default void preClean() {
}
/**
- * Used to do some cleanup work
+ * Will be called after cleaner run.
*/
default void postClean() {
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/GlobalProcedureInterface.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/GlobalProcedureInterface.java
new file mode 100644
index 000000000000..1ef168abfd8f
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/GlobalProcedureInterface.java
@@ -0,0 +1,29 @@
+/*
+ * 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;
+
+/**
+ * Procedure interface for global operations, such as migration.
+ */
+@InterfaceAudience.Private
+public interface GlobalProcedureInterface {
+
+ String getGlobalId();
+}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/GlobalQueue.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/GlobalQueue.java
new file mode 100644
index 000000000000..1633dc4856e7
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/GlobalQueue.java
@@ -0,0 +1,35 @@
+/*
+ * 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.hadoop.hbase.procedure2.LockStatus;
+import org.apache.hadoop.hbase.procedure2.Procedure;
+import org.apache.yetus.audience.InterfaceAudience;
+
+@InterfaceAudience.Private
+public class GlobalQueue extends Queue {
+
+ public GlobalQueue(String globalId, LockStatus lockStatus) {
+ super(globalId, lockStatus);
+ }
+
+ @Override
+ boolean requireExclusiveLock(Procedure> proc) {
+ return true;
+ }
+}
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 866f2f6f4032..fbf0eb8abf32 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
@@ -22,6 +22,7 @@
import java.util.List;
import java.util.function.Function;
import java.util.function.Supplier;
+import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableExistsException;
import org.apache.hadoop.hbase.TableName;
@@ -95,16 +96,20 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
(n, k) -> n.compareKey((String) k);
private final static AvlKeyComparator META_QUEUE_KEY_COMPARATOR =
(n, k) -> n.compareKey((TableName) k);
+ private final static AvlKeyComparator GLOBAL_QUEUE_KEY_COMPARATOR =
+ (n, k) -> n.compareKey((String) k);
private final FairQueue serverRunQueue = new FairQueue<>();
private final FairQueue tableRunQueue = new FairQueue<>();
private final FairQueue peerRunQueue = new FairQueue<>();
private final FairQueue metaRunQueue = new FairQueue<>();
+ private final FairQueue globalRunQueue = new FairQueue<>();
private final ServerQueue[] serverBuckets = new ServerQueue[128];
private TableQueue tableMap = null;
private PeerQueue peerMap = null;
private MetaQueue metaMap = null;
+ private GlobalQueue globalMap = null;
private final SchemaLocking locking;
@@ -128,6 +133,8 @@ protected void enqueue(final Procedure proc, final boolean addFront) {
doAdd(serverRunQueue, getServerQueue(spi.getServerName(), spi), proc, addFront);
} else if (isPeerProcedure(proc)) {
doAdd(peerRunQueue, getPeerQueue(getPeerId(proc)), proc, addFront);
+ } else if (isGlobalProcedure(proc)) {
+ doAdd(globalRunQueue, getGlobalQueue(getGlobalId(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
@@ -163,14 +170,19 @@ private > void doAdd(FairQueue fairq, Queue queue,
@Override
protected boolean queueHasRunnables() {
- return metaRunQueue.hasRunnables() || tableRunQueue.hasRunnables()
- || serverRunQueue.hasRunnables() || peerRunQueue.hasRunnables();
+ return globalRunQueue.hasRunnables() || metaRunQueue.hasRunnables()
+ || tableRunQueue.hasRunnables() || serverRunQueue.hasRunnables()
+ || peerRunQueue.hasRunnables();
}
@Override
protected Procedure dequeue() {
- // meta procedure is always the first priority
- Procedure> pollResult = doPoll(metaRunQueue);
+ // pull global first
+ Procedure> pollResult = doPoll(globalRunQueue);
+ // then meta procedure
+ if (pollResult == null) {
+ pollResult = doPoll(metaRunQueue);
+ }
// 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.
@@ -268,6 +280,14 @@ private void clearQueue() {
clear(peerMap, peerRunQueue, PEER_QUEUE_KEY_COMPARATOR);
peerMap = null;
+ // Remove Meta
+ clear(metaMap, metaRunQueue, META_QUEUE_KEY_COMPARATOR);
+ metaMap = null;
+
+ // Remove Global
+ clear(globalMap, globalRunQueue, GLOBAL_QUEUE_KEY_COMPARATOR);
+ globalMap = null;
+
assert size() == 0 : "expected queue size to be 0, got " + size();
}
@@ -300,6 +320,7 @@ protected int queueSize() {
count += queueSize(tableMap);
count += queueSize(peerMap);
count += queueSize(metaMap);
+ count += queueSize(globalMap);
return count;
}
@@ -502,6 +523,51 @@ private static boolean isMetaProcedure(Procedure> proc) {
return proc instanceof MetaProcedureInterface;
}
+ // ============================================================================
+ // Global Queue Lookup Helpers
+ // ============================================================================
+ private GlobalQueue getGlobalQueue(String globalId) {
+ GlobalQueue node = AvlTree.get(globalMap, globalId, GLOBAL_QUEUE_KEY_COMPARATOR);
+ if (node != null) {
+ return node;
+ }
+ node = new GlobalQueue(globalId, locking.getGlobalLock(globalId));
+ globalMap = AvlTree.insert(globalMap, node);
+ return node;
+ }
+
+ private void removeGlobalQueue(String globalId) {
+ globalMap = AvlTree.remove(globalMap, globalId, GLOBAL_QUEUE_KEY_COMPARATOR);
+ locking.removeGlobalLock(globalId);
+ }
+
+ private void tryCleanupGlobalQueue(String globalId, Procedure> procedure) {
+ schedLock();
+ try {
+ GlobalQueue queue = AvlTree.get(globalMap, globalId, GLOBAL_QUEUE_KEY_COMPARATOR);
+ if (queue == null) {
+ return;
+ }
+
+ final LockAndQueue lock = locking.getGlobalLock(globalId);
+ if (queue.isEmpty() && lock.tryExclusiveLock(procedure)) {
+ removeFromRunQueue(globalRunQueue, queue,
+ () -> "clean up global queue after " + procedure + " completed");
+ removeGlobalQueue(globalId);
+ }
+ } finally {
+ schedUnlock();
+ }
+ }
+
+ private static boolean isGlobalProcedure(Procedure> proc) {
+ return proc instanceof GlobalProcedureInterface;
+ }
+
+ private static String getGlobalId(Procedure> proc) {
+ return ((GlobalProcedureInterface) proc).getGlobalId();
+ }
+
// ============================================================================
// Table Locking Helpers
// ============================================================================
@@ -1006,6 +1072,51 @@ public void wakeMetaExclusiveLock(Procedure> procedure) {
}
}
+ // ============================================================================
+ // Global Locking Helpers
+ // ============================================================================
+ /**
+ * Try to acquire the share lock on global.
+ * @see #wakeGlobalExclusiveLock(Procedure, String)
+ * @param procedure the procedure trying to acquire the lock
+ * @return true if the procedure has to wait for global to be available
+ */
+ public boolean waitGlobalExclusiveLock(Procedure> procedure, String globalId) {
+ schedLock();
+ try {
+ final LockAndQueue lock = locking.getGlobalLock(globalId);
+ if (lock.tryExclusiveLock(procedure)) {
+ removeFromRunQueue(globalRunQueue, getGlobalQueue(globalId),
+ () -> procedure + " held shared lock");
+ return false;
+ }
+ waitProcedure(lock, procedure);
+ logLockedResource(LockedResourceType.GLOBAL, HConstants.EMPTY_STRING);
+ return true;
+ } finally {
+ schedUnlock();
+ }
+ }
+
+ /**
+ * Wake the procedures waiting for global.
+ * @see #waitGlobalExclusiveLock(Procedure, String)
+ * @param procedure the procedure releasing the lock
+ */
+ public void wakeGlobalExclusiveLock(Procedure> procedure, String globalId) {
+ schedLock();
+ try {
+ final LockAndQueue lock = locking.getGlobalLock(globalId);
+ lock.releaseExclusiveLock(procedure);
+ addToRunQueue(globalRunQueue, getGlobalQueue(globalId),
+ () -> procedure + " released shared lock");
+ int waitingCount = wakeWaitingProcedures(lock);
+ wakePollIfNeeded(waitingCount);
+ } finally {
+ schedUnlock();
+ }
+ }
+
/**
* For debugging. Expensive.
*/
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SchemaLocking.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SchemaLocking.java
index 13419ac455ca..853d13b0c93b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SchemaLocking.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SchemaLocking.java
@@ -53,6 +53,7 @@ class SchemaLocking {
// Single map for all regions irrespective of tables. Key is encoded region name.
private final Map regionLocks = new HashMap<>();
private final Map peerLocks = new HashMap<>();
+ private final Map globalLocks = new HashMap<>();
private final LockAndQueue metaLock;
public SchemaLocking(Function> procedureRetriever) {
@@ -94,6 +95,10 @@ LockAndQueue getMetaLock() {
return metaLock;
}
+ LockAndQueue getGlobalLock(String globalId) {
+ return getLock(globalLocks, globalId);
+ }
+
LockAndQueue removeRegionLock(String encodedRegionName) {
return regionLocks.remove(encodedRegionName);
}
@@ -114,6 +119,10 @@ LockAndQueue removePeerLock(String peerId) {
return peerLocks.remove(peerId);
}
+ LockAndQueue removeGlobalLock(String globalId) {
+ return globalLocks.remove(globalId);
+ }
+
private LockedResource createLockedResource(LockedResourceType resourceType, String resourceName,
LockAndQueue queue) {
LockType lockType;
@@ -164,6 +173,8 @@ List getLocks() {
addToLockedResources(lockedResources, peerLocks, Function.identity(), LockedResourceType.PEER);
addToLockedResources(lockedResources, ImmutableMap.of(TableName.META_TABLE_NAME, metaLock),
tn -> tn.getNameAsString(), LockedResourceType.META);
+ addToLockedResources(lockedResources, globalLocks, Function.identity(),
+ LockedResourceType.GLOBAL);
return lockedResources;
}
@@ -191,6 +202,10 @@ LockedResource getLockResource(LockedResourceType resourceType, String resourceN
break;
case META:
queue = metaLock;
+ break;
+ case GLOBAL:
+ queue = globalLocks.get(resourceName);
+ break;
default:
queue = null;
break;
@@ -216,7 +231,8 @@ public String toString() {
+ filterUnlocked(this.namespaceLocks) + ", tableLocks=" + filterUnlocked(this.tableLocks)
+ ", regionLocks=" + filterUnlocked(this.regionLocks) + ", peerLocks="
+ filterUnlocked(this.peerLocks) + ", metaLocks="
- + filterUnlocked(ImmutableMap.of(TableName.META_TABLE_NAME, metaLock));
+ + filterUnlocked(ImmutableMap.of(TableName.META_TABLE_NAME, metaLock)) + ", globalLocks="
+ + filterUnlocked(globalLocks);
}
private String filterUnlocked(Map, LockAndQueue> locks) {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java
index 0bd0f3ba0c7e..97976756d828 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java
@@ -36,7 +36,8 @@
import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
import org.apache.hadoop.hbase.master.assignment.RegionStateNode;
import org.apache.hadoop.hbase.master.assignment.TransitRegionStateProcedure;
-import org.apache.hadoop.hbase.master.replication.ClaimReplicationQueuesProcedure;
+import org.apache.hadoop.hbase.master.replication.AssignReplicationQueuesProcedure;
+import org.apache.hadoop.hbase.master.replication.MigrateReplicationQueueFromZkToTableProcedure;
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
import org.apache.hadoop.hbase.monitoring.TaskMonitor;
import org.apache.hadoop.hbase.procedure2.Procedure;
@@ -52,6 +53,7 @@
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ServerCrashState;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos;
/**
* Handle crashed server. This is a port to ProcedureV2 of what used to be euphemistically called
@@ -240,15 +242,43 @@ protected Flow executeFromState(MasterProcedureEnv env, ServerCrashState state)
}
assignRegions(env, regionsOnCrashedServer);
}
- setNextState(ServerCrashState.SERVER_CRASH_CLAIM_REPLICATION_QUEUES);
+ // If there is no replication peer, we do not need to enter the claim queues stage.
+ // This is also very important that now we will later initialize ReplicationQueueStorage
+ // so if there is no replication peer added yet, the storage can not be accessed.
+ // And there will be no race because:
+ // 1. For adding replication peer, if the peer storage has not been updated yet, the crash
+ // region server will not have any replication queues for this peer, so it is safe to skip
+ // claiming.
+ // 2. For removing replication peer, it it has already updated the peer storage, then
+ // there is no way to rollback and region servers are already started to close and delete
+ // replication queues, so it is also safe to skip claiming.
+ if (env.getReplicationPeerManager().listPeers(null).isEmpty()) {
+ setNextState(ServerCrashState.SERVER_CRASH_FINISH);
+ } else {
+ setNextState(ServerCrashState.SERVER_CRASH_CLAIM_REPLICATION_QUEUES);
+ }
break;
case SERVER_CRASH_HANDLE_RIT2:
// Noop. Left in place because we used to call handleRIT here for a second time
// but no longer necessary since HBASE-20634.
- setNextState(ServerCrashState.SERVER_CRASH_CLAIM_REPLICATION_QUEUES);
+ if (env.getReplicationPeerManager().listPeers(null).isEmpty()) {
+ setNextState(ServerCrashState.SERVER_CRASH_FINISH);
+ } else {
+ setNextState(ServerCrashState.SERVER_CRASH_CLAIM_REPLICATION_QUEUES);
+ }
break;
case SERVER_CRASH_CLAIM_REPLICATION_QUEUES:
- addChildProcedure(new ClaimReplicationQueuesProcedure(serverName));
+ if (
+ env.getMasterServices().getProcedures().stream()
+ .filter(p -> p instanceof MigrateReplicationQueueFromZkToTableProcedure)
+ .anyMatch(p -> !p.isFinished())
+ ) {
+ LOG.info("There is a pending {}, will retry claim replication queue later",
+ MigrateReplicationQueueFromZkToTableProcedure.class.getSimpleName());
+ suspend(10_000, true);
+ return Flow.NO_MORE_STATE;
+ }
+ addChildProcedure(new AssignReplicationQueuesProcedure(serverName));
setNextState(ServerCrashState.SERVER_CRASH_FINISH);
break;
case SERVER_CRASH_FINISH:
@@ -413,6 +443,13 @@ protected void releaseLock(final MasterProcedureEnv env) {
env.getProcedureScheduler().wakeServerExclusiveLock(this, getServerName());
}
+ @Override
+ protected synchronized boolean setTimeoutFailure(MasterProcedureEnv env) {
+ setState(ProcedureProtos.ProcedureState.RUNNABLE);
+ env.getProcedureScheduler().addFront(this);
+ return false;
+ }
+
@Override
public void toStringClassDetails(StringBuilder sb) {
sb.append(getProcName());
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/region/MasterRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/region/MasterRegion.java
index 86c231144581..e45b6271f7b9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/region/MasterRegion.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/region/MasterRegion.java
@@ -380,7 +380,7 @@ public static MasterRegion create(MasterRegionParams params) throws IOException
params.archivedWalSuffix(), params.rollPeriodMs(), params.flushSize());
walRoller.start();
- WALFactory walFactory = new WALFactory(conf, server.getServerName().toString(), server, false);
+ WALFactory walFactory = new WALFactory(conf, server.getServerName(), server, false);
Path tableDir = CommonFSUtils.getTableDir(rootDir, td.getTableName());
Path initializingFlag = new Path(tableDir, INITIALIZING_FLAG);
Path initializedFlag = new Path(tableDir, INITIALIZED_FLAG);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AbstractPeerNoLockProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AbstractPeerNoLockProcedure.java
index 660f9968573d..1f0a89f20762 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AbstractPeerNoLockProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AbstractPeerNoLockProcedure.java
@@ -98,10 +98,7 @@ protected final ProcedureSuspendedException suspend(Configuration conf,
}
long backoff = retryCounter.getBackoffTimeAndIncrementAttempts();
backoffConsumer.accept(backoff);
- setTimeout(Math.toIntExact(backoff));
- setState(ProcedureProtos.ProcedureState.WAITING_TIMEOUT);
- skipPersistence();
- throw new ProcedureSuspendedException();
+ throw suspend(Math.toIntExact(backoff), false);
}
protected final void resetRetry() {
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
index 6d0acee76caa..1d02fab5f194 100644
--- 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
@@ -21,7 +21,6 @@
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.master.procedure.ProcedurePrepareLatch;
import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
import org.apache.hadoop.hbase.replication.ReplicationException;
@@ -45,6 +44,8 @@ public class AddPeerProcedure extends ModifyPeerProcedure {
private boolean enabled;
+ private boolean cleanerDisabled;
+
public AddPeerProcedure() {
}
@@ -84,15 +85,24 @@ protected ReplicationPeerConfig getNewPeerConfig() {
@Override
protected void releaseLatch(MasterProcedureEnv env) {
+ if (cleanerDisabled) {
+ env.getReplicationPeerManager().getReplicationLogCleanerBarrier().enable();
+ }
if (peerConfig.isSyncReplication()) {
env.getReplicationPeerManager().releaseSyncReplicationPeerLock();
}
- ProcedurePrepareLatch.releaseLatch(latch, this);
+ super.releaseLatch(env);
}
@Override
protected void prePeerModification(MasterProcedureEnv env)
throws IOException, ReplicationException, ProcedureSuspendedException {
+ if (!env.getReplicationPeerManager().getReplicationLogCleanerBarrier().disable()) {
+ throw suspend(env.getMasterConfiguration(),
+ backoff -> LOG.warn("LogCleaner is run at the same time when adding peer {}, sleep {} secs",
+ peerId, backoff / 1000));
+ }
+ cleanerDisabled = true;
MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
if (cpHost != null) {
cpHost.preAddReplicationPeer(peerId, peerConfig);
@@ -128,9 +138,14 @@ protected void postPeerModification(MasterProcedureEnv env)
@Override
protected void afterReplay(MasterProcedureEnv env) {
if (getCurrentState() == getInitialState()) {
- // will try to acquire the lock when executing the procedure, no need to acquire it here
+ // do not need to disable log cleaner or acquire lock if we are in the initial state, later
+ // when executing the procedure we will try to disable and acquire.
return;
}
+ if (!env.getReplicationPeerManager().getReplicationLogCleanerBarrier().disable()) {
+ throw new IllegalStateException("can not disable log cleaner, this should not happen");
+ }
+ cleanerDisabled = true;
if (peerConfig.isSyncReplication()) {
if (!env.getReplicationPeerManager().tryAcquireSyncReplicationPeerLock()) {
throw new IllegalStateException(
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AssignReplicationQueuesProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AssignReplicationQueuesProcedure.java
new file mode 100644
index 000000000000..b547c87009dd
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AssignReplicationQueuesProcedure.java
@@ -0,0 +1,241 @@
+/*
+ * 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.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.master.MasterFileSystem;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.master.procedure.ServerProcedureInterface;
+import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
+import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
+import org.apache.hadoop.hbase.procedure2.ProcedureUtil;
+import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
+import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
+import org.apache.hadoop.hbase.replication.ReplicationException;
+import org.apache.hadoop.hbase.replication.ReplicationGroupOffset;
+import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
+import org.apache.hadoop.hbase.replication.ReplicationQueueId;
+import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
+import org.apache.hadoop.hbase.replication.regionserver.ReplicationSyncUp;
+import org.apache.hadoop.hbase.util.RetryCounter;
+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.AssignReplicationQueuesState;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.AssignReplicationQueuesStateData;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos;
+
+@InterfaceAudience.Private
+public class AssignReplicationQueuesProcedure
+ extends StateMachineProcedure
+ implements ServerProcedureInterface {
+
+ private static final Logger LOG = LoggerFactory.getLogger(AssignReplicationQueuesProcedure.class);
+
+ private ServerName crashedServer;
+
+ private RetryCounter retryCounter;
+
+ public AssignReplicationQueuesProcedure() {
+ }
+
+ public AssignReplicationQueuesProcedure(ServerName crashedServer) {
+ this.crashedServer = crashedServer;
+ }
+
+ @Override
+ public ServerName getServerName() {
+ return crashedServer;
+ }
+
+ @Override
+ public boolean hasMetaTableRegion() {
+ return false;
+ }
+
+ @Override
+ public ServerOperationType getServerOperationType() {
+ return ServerOperationType.CLAIM_REPLICATION_QUEUES;
+ }
+
+ private void addMissingQueues(MasterProcedureEnv env) throws ReplicationException {
+ ReplicationQueueStorage storage = env.getReplicationPeerManager().getQueueStorage();
+
+ Set existingQueuePeerIds = new HashSet<>();
+ List queueIds = storage.listAllQueueIds(crashedServer);
+ for (Iterator iter = queueIds.iterator(); iter.hasNext();) {
+ ReplicationQueueId queueId = iter.next();
+ if (!queueId.isRecovered()) {
+ existingQueuePeerIds.add(queueId.getPeerId());
+ }
+ }
+ List peers = env.getReplicationPeerManager().listPeers(null);
+ for (ReplicationPeerDescription peer : peers) {
+ if (!existingQueuePeerIds.contains(peer.getPeerId())) {
+ ReplicationQueueId queueId = new ReplicationQueueId(crashedServer, peer.getPeerId());
+ LOG.debug("Add replication queue {} for claiming", queueId);
+ env.getReplicationPeerManager().getQueueStorage().setOffset(queueId,
+ crashedServer.toString(), ReplicationGroupOffset.BEGIN, Collections.emptyMap());
+ }
+ }
+ }
+
+ private Flow claimQueues(MasterProcedureEnv env) throws ReplicationException, IOException {
+ Set existingPeerIds = env.getReplicationPeerManager().listPeers(null).stream()
+ .map(ReplicationPeerDescription::getPeerId).collect(Collectors.toSet());
+ ReplicationQueueStorage storage = env.getReplicationPeerManager().getQueueStorage();
+ // filter out replication queue for deleted peers
+ List queueIds = storage.listAllQueueIds(crashedServer).stream()
+ .filter(q -> existingPeerIds.contains(q.getPeerId())).collect(Collectors.toList());
+ if (queueIds.isEmpty()) {
+ LOG.debug("Finish claiming replication queues for {}", crashedServer);
+ // we are done
+ return Flow.NO_MORE_STATE;
+ }
+ LOG.debug("There are {} replication queues need to be claimed for {}", queueIds.size(),
+ crashedServer);
+ List targetServers =
+ env.getMasterServices().getServerManager().getOnlineServersList();
+ if (targetServers.isEmpty()) {
+ throw new ReplicationException("no region server available");
+ }
+ Collections.shuffle(targetServers);
+ for (int i = 0, n = Math.min(queueIds.size(), targetServers.size()); i < n; i++) {
+ addChildProcedure(
+ new ClaimReplicationQueueRemoteProcedure(queueIds.get(i), targetServers.get(i)));
+ }
+ retryCounter = null;
+ return Flow.HAS_MORE_STATE;
+ }
+
+ // check whether ReplicationSyncUp has already done the work for us, if so, we should skip
+ // claiming the replication queues and deleting them instead.
+ private boolean shouldSkip(MasterProcedureEnv env) throws IOException {
+ MasterFileSystem mfs = env.getMasterFileSystem();
+ Path syncUpDir = new Path(mfs.getRootDir(), ReplicationSyncUp.INFO_DIR);
+ return mfs.getFileSystem().exists(new Path(syncUpDir, crashedServer.getServerName()));
+ }
+
+ private void removeQueues(MasterProcedureEnv env) throws ReplicationException, IOException {
+ ReplicationQueueStorage storage = env.getReplicationPeerManager().getQueueStorage();
+ for (ReplicationQueueId queueId : storage.listAllQueueIds(crashedServer)) {
+ storage.removeQueue(queueId);
+ }
+ MasterFileSystem mfs = env.getMasterFileSystem();
+ Path syncUpDir = new Path(mfs.getRootDir(), ReplicationSyncUp.INFO_DIR);
+ // remove the region server record file
+ mfs.getFileSystem().delete(new Path(syncUpDir, crashedServer.getServerName()), false);
+ }
+
+ @Override
+ protected Flow executeFromState(MasterProcedureEnv env, AssignReplicationQueuesState state)
+ throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException {
+ try {
+ switch (state) {
+ case ASSIGN_REPLICATION_QUEUES_ADD_MISSING_QUEUES:
+ if (shouldSkip(env)) {
+ setNextState(AssignReplicationQueuesState.ASSIGN_REPLICATION_QUEUES_REMOVE_QUEUES);
+ return Flow.HAS_MORE_STATE;
+ } else {
+ addMissingQueues(env);
+ retryCounter = null;
+ setNextState(AssignReplicationQueuesState.ASSIGN_REPLICATION_QUEUES_CLAIM);
+ return Flow.HAS_MORE_STATE;
+ }
+ case ASSIGN_REPLICATION_QUEUES_CLAIM:
+ if (shouldSkip(env)) {
+ retryCounter = null;
+ setNextState(AssignReplicationQueuesState.ASSIGN_REPLICATION_QUEUES_REMOVE_QUEUES);
+ return Flow.HAS_MORE_STATE;
+ } else {
+ return claimQueues(env);
+ }
+ case ASSIGN_REPLICATION_QUEUES_REMOVE_QUEUES:
+ removeQueues(env);
+ return Flow.NO_MORE_STATE;
+ default:
+ throw new UnsupportedOperationException("unhandled state=" + state);
+ }
+ } catch (Exception e) {
+ if (retryCounter == null) {
+ retryCounter = ProcedureUtil.createRetryCounter(env.getMasterConfiguration());
+ }
+ long backoff = retryCounter.getBackoffTimeAndIncrementAttempts();
+ LOG.warn("Failed to claim replication queues for {}, suspend {}secs {}; {};", crashedServer,
+ backoff / 1000, e);
+ setTimeout(Math.toIntExact(backoff));
+ setState(ProcedureProtos.ProcedureState.WAITING_TIMEOUT);
+ skipPersistence();
+ throw new ProcedureSuspendedException();
+ }
+ }
+
+ @Override
+ protected synchronized boolean setTimeoutFailure(MasterProcedureEnv env) {
+ setState(ProcedureProtos.ProcedureState.RUNNABLE);
+ env.getProcedureScheduler().addFront(this);
+ return false;
+ }
+
+ @Override
+ protected void rollbackState(MasterProcedureEnv env, AssignReplicationQueuesState state)
+ throws IOException, InterruptedException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ protected AssignReplicationQueuesState getState(int stateId) {
+ return AssignReplicationQueuesState.forNumber(stateId);
+ }
+
+ @Override
+ protected int getStateId(AssignReplicationQueuesState state) {
+ return state.getNumber();
+ }
+
+ @Override
+ protected AssignReplicationQueuesState getInitialState() {
+ return AssignReplicationQueuesState.ASSIGN_REPLICATION_QUEUES_ADD_MISSING_QUEUES;
+ }
+
+ @Override
+ protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException {
+ super.serializeStateData(serializer);
+ serializer.serialize(AssignReplicationQueuesStateData.newBuilder()
+ .setCrashedServer(ProtobufUtil.toServerName(crashedServer)).build());
+ }
+
+ @Override
+ protected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException {
+ super.deserializeStateData(serializer);
+ AssignReplicationQueuesStateData proto =
+ serializer.deserialize(AssignReplicationQueuesStateData.class);
+ crashedServer = ProtobufUtil.toServerName(proto.getCrashedServer());
+ }
+
+}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ClaimReplicationQueueRemoteProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ClaimReplicationQueueRemoteProcedure.java
index 9ef97d1fff62..d3aeeba541a2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ClaimReplicationQueueRemoteProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ClaimReplicationQueueRemoteProcedure.java
@@ -19,15 +19,22 @@
import java.io.IOException;
import java.util.Optional;
+import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.master.MasterFileSystem;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
import org.apache.hadoop.hbase.master.procedure.RSProcedureDispatcher.ServerOperation;
import org.apache.hadoop.hbase.master.procedure.ServerProcedureInterface;
import org.apache.hadoop.hbase.master.procedure.ServerRemoteProcedure;
+import org.apache.hadoop.hbase.procedure2.Procedure;
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.replication.ReplicationQueueId;
import org.apache.hadoop.hbase.replication.regionserver.ClaimReplicationQueueCallable;
+import org.apache.hadoop.hbase.replication.regionserver.ReplicationSyncUp;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -43,34 +50,59 @@ public class ClaimReplicationQueueRemoteProcedure extends ServerRemoteProcedure
private static final Logger LOG =
LoggerFactory.getLogger(ClaimReplicationQueueRemoteProcedure.class);
- private ServerName crashedServer;
-
- private String queue;
+ private ReplicationQueueId queueId;
public ClaimReplicationQueueRemoteProcedure() {
}
- public ClaimReplicationQueueRemoteProcedure(ServerName crashedServer, String queue,
- ServerName targetServer) {
- this.crashedServer = crashedServer;
- this.queue = queue;
+ public ClaimReplicationQueueRemoteProcedure(ReplicationQueueId queueId, ServerName targetServer) {
+ this.queueId = queueId;
this.targetServer = targetServer;
}
+ // check whether ReplicationSyncUp has already done the work for us, if so, we should skip
+ // claiming the replication queues and deleting them instead.
+ private boolean shouldSkip(MasterProcedureEnv env) throws IOException {
+ MasterFileSystem mfs = env.getMasterFileSystem();
+ Path syncUpDir = new Path(mfs.getRootDir(), ReplicationSyncUp.INFO_DIR);
+ return mfs.getFileSystem().exists(new Path(syncUpDir, getServerName().getServerName()));
+ }
+
+ @Override
+ protected synchronized Procedure[] execute(MasterProcedureEnv env)
+ throws ProcedureYieldException, ProcedureSuspendedException, InterruptedException {
+ try {
+ if (shouldSkip(env)) {
+ LOG.info("Skip claiming {} because replication sync up has already done it for us",
+ getServerName());
+ return null;
+ }
+ } catch (IOException e) {
+ LOG.warn("failed to check whether we should skip claiming {} due to replication sync up",
+ getServerName(), e);
+ // just finish the procedure here, as the AssignReplicationQueuesProcedure will reschedule
+ return null;
+ }
+ return super.execute(env);
+ }
+
@Override
public Optional remoteCallBuild(MasterProcedureEnv env, ServerName remote) {
assert targetServer.equals(remote);
+ ClaimReplicationQueueRemoteParameter.Builder builder = ClaimReplicationQueueRemoteParameter
+ .newBuilder().setCrashedServer(ProtobufUtil.toServerName(queueId.getServerName()))
+ .setQueue(queueId.getPeerId());
+ queueId.getSourceServerName()
+ .ifPresent(sourceServer -> builder.setSourceServer(ProtobufUtil.toServerName(sourceServer)));
return Optional.of(new ServerOperation(this, getProcId(), ClaimReplicationQueueCallable.class,
- ClaimReplicationQueueRemoteParameter.newBuilder()
- .setCrashedServer(ProtobufUtil.toServerName(crashedServer)).setQueue(queue).build()
- .toByteArray()));
+ builder.build().toByteArray()));
}
@Override
public ServerName getServerName() {
// return crashed server here, as we are going to recover its replication queues so we should
// use its scheduler queue instead of the one for the target server.
- return crashedServer;
+ return queueId.getServerName();
}
@Override
@@ -86,8 +118,7 @@ public ServerOperationType getServerOperationType() {
@Override
protected void complete(MasterProcedureEnv env, Throwable error) {
if (error != null) {
- LOG.warn("Failed to claim replication queue {} of crashed server on server {} ", queue,
- crashedServer, targetServer, error);
+ LOG.warn("Failed to claim replication queue {} on server {} ", queueId, targetServer, error);
this.succ = false;
} else {
this.succ = true;
@@ -111,17 +142,26 @@ protected boolean waitInitialized(MasterProcedureEnv env) {
@Override
protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException {
- serializer.serialize(ClaimReplicationQueueRemoteStateData.newBuilder()
- .setCrashedServer(ProtobufUtil.toServerName(crashedServer)).setQueue(queue)
- .setTargetServer(ProtobufUtil.toServerName(targetServer)).build());
+ ClaimReplicationQueueRemoteStateData.Builder builder = ClaimReplicationQueueRemoteStateData
+ .newBuilder().setCrashedServer(ProtobufUtil.toServerName(queueId.getServerName()))
+ .setQueue(queueId.getPeerId()).setTargetServer(ProtobufUtil.toServerName(targetServer));
+ queueId.getSourceServerName()
+ .ifPresent(sourceServer -> builder.setSourceServer(ProtobufUtil.toServerName(sourceServer)));
+ serializer.serialize(builder.build());
}
@Override
protected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException {
ClaimReplicationQueueRemoteStateData data =
serializer.deserialize(ClaimReplicationQueueRemoteStateData.class);
- crashedServer = ProtobufUtil.toServerName(data.getCrashedServer());
- queue = data.getQueue();
targetServer = ProtobufUtil.toServerName(data.getTargetServer());
+ ServerName crashedServer = ProtobufUtil.toServerName(data.getCrashedServer());
+ String queue = data.getQueue();
+ if (data.hasSourceServer()) {
+ queueId = new ReplicationQueueId(crashedServer, queue,
+ ProtobufUtil.toServerName(data.getSourceServer()));
+ } else {
+ queueId = new ReplicationQueueId(crashedServer, queue);
+ }
}
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ClaimReplicationQueuesProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ClaimReplicationQueuesProcedure.java
index 5eb6608f4ee6..747d352d2aa5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ClaimReplicationQueuesProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ClaimReplicationQueuesProcedure.java
@@ -19,8 +19,10 @@
import java.io.IOException;
import java.util.Collections;
+import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
+import java.util.Set;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
import org.apache.hadoop.hbase.master.procedure.ServerProcedureInterface;
@@ -30,7 +32,9 @@
import org.apache.hadoop.hbase.procedure2.ProcedureUtil;
import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
import org.apache.hadoop.hbase.replication.ReplicationException;
-import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
+import org.apache.hadoop.hbase.replication.ReplicationGroupOffset;
+import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
+import org.apache.hadoop.hbase.replication.ReplicationQueueId;
import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
import org.apache.hadoop.hbase.util.RetryCounter;
import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
@@ -44,7 +48,10 @@
/**
* Used to assign the replication queues of a dead server to other region servers.
+ * @deprecated Use {@link AssignReplicationQueuesProcedure} instead, kept only for keeping
+ * compatibility.
*/
+@Deprecated
@InterfaceAudience.Private
public class ClaimReplicationQueuesProcedure extends Procedure
implements ServerProcedureInterface {
@@ -82,22 +89,36 @@ protected Procedure[] execute(MasterProcedureEnv env)
throws ProcedureYieldException, ProcedureSuspendedException, InterruptedException {
ReplicationQueueStorage storage = env.getReplicationPeerManager().getQueueStorage();
try {
- List queues = storage.getAllQueues(crashedServer);
+ List queues = storage.listAllQueueIds(crashedServer);
+ Set existQueuePeerIds = new HashSet<>();
// this is for upgrading to the new region replication framework, where we will delete the
- // legacy region_replica_replication peer directly, without deleting the replication queues,
- // as it may still be used by region servers which have not been upgraded yet.
- for (Iterator iter = queues.iterator(); iter.hasNext();) {
- ReplicationQueueInfo queue = new ReplicationQueueInfo(iter.next());
- if (queue.getPeerId().equals(ServerRegionReplicaUtil.REGION_REPLICA_REPLICATION_PEER)) {
+ // legacy region_replica_replication peer directly, without deleting the replication queues
+ for (Iterator iter = queues.iterator(); iter.hasNext();) {
+ ReplicationQueueId queueId = iter.next();
+ if (queueId.getPeerId().equals(ServerRegionReplicaUtil.REGION_REPLICA_REPLICATION_PEER)) {
LOG.info("Found replication queue {} for legacy region replication peer, "
- + "skipping claiming and removing...", queue.getQueueId());
+ + "skipping claiming and removing...", queueId);
iter.remove();
- storage.removeQueue(crashedServer, queue.getQueueId());
+ storage.removeQueue(queueId);
+ } else if (!queueId.isRecovered()) {
+ existQueuePeerIds.add(queueId.getPeerId());
+ }
+ }
+ List peers = env.getReplicationPeerManager().listPeers(null);
+ // TODO: the implementation is not enough yet, if there are retries, we need to know whether
+ // the replication queue for the given peer has been claimed or not, otherwise this logic will
+ // introduce redundant replication queues for the same peer. Add this logic to make some UTs
+ // pass first.
+ for (ReplicationPeerDescription peer : peers) {
+ if (!existQueuePeerIds.contains(peer.getPeerId())) {
+ ReplicationQueueId queueId = new ReplicationQueueId(crashedServer, peer.getPeerId());
+ env.getReplicationPeerManager().getQueueStorage().setOffset(queueId,
+ crashedServer.toString(), ReplicationGroupOffset.BEGIN, Collections.emptyMap());
+ queues.add(queueId);
}
}
if (queues.isEmpty()) {
LOG.debug("Finish claiming replication queues for {}", crashedServer);
- storage.removeReplicatorIfQueueIsEmpty(crashedServer);
// we are done
return null;
}
@@ -112,8 +133,7 @@ protected Procedure[] execute(MasterProcedureEnv env)
ClaimReplicationQueueRemoteProcedure[] procs =
new ClaimReplicationQueueRemoteProcedure[Math.min(queues.size(), targetServers.size())];
for (int i = 0; i < procs.length; i++) {
- procs[i] = new ClaimReplicationQueueRemoteProcedure(crashedServer, queues.get(i),
- targetServers.get(i));
+ procs[i] = new ClaimReplicationQueueRemoteProcedure(queues.get(i), targetServers.get(i));
}
return procs;
} catch (ReplicationException e) {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/MigrateReplicationQueueFromZkToTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/MigrateReplicationQueueFromZkToTableProcedure.java
new file mode 100644
index 000000000000..b7c4e33ef858
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/MigrateReplicationQueueFromZkToTableProcedure.java
@@ -0,0 +1,330 @@
+/*
+ * 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 static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MigrateReplicationQueueFromZkToTableState.MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_DISABLE_CLEANER;
+import static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MigrateReplicationQueueFromZkToTableState.MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_DISABLE_PEER;
+import static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MigrateReplicationQueueFromZkToTableState.MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_ENABLE_CLEANER;
+import static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MigrateReplicationQueueFromZkToTableState.MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_ENABLE_PEER;
+import static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MigrateReplicationQueueFromZkToTableState.MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_MIGRATE;
+import static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MigrateReplicationQueueFromZkToTableState.MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_PREPARE;
+import static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MigrateReplicationQueueFromZkToTableState.MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_WAIT_UPGRADING;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.function.LongConsumer;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.master.procedure.GlobalProcedureInterface;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.master.procedure.PeerProcedureInterface;
+import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
+import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
+import org.apache.hadoop.hbase.procedure2.ProcedureUtil;
+import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
+import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
+import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
+import org.apache.hadoop.hbase.replication.ZKReplicationQueueStorageForMigration;
+import org.apache.hadoop.hbase.util.FutureUtils;
+import org.apache.hadoop.hbase.util.IdLock;
+import org.apache.hadoop.hbase.util.RetryCounter;
+import org.apache.hadoop.hbase.util.VersionInfo;
+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.MasterProcedureProtos.MigrateReplicationQueueFromZkToTableState;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MigrateReplicationQueueFromZkToTableStateData;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos;
+
+/**
+ * A procedure for migrating replication queue data from zookeeper to hbase:replication table.
+ */
+@InterfaceAudience.Private
+public class MigrateReplicationQueueFromZkToTableProcedure
+ extends StateMachineProcedure
+ implements GlobalProcedureInterface {
+
+ private static final Logger LOG =
+ LoggerFactory.getLogger(MigrateReplicationQueueFromZkToTableProcedure.class);
+
+ private static final int MIN_MAJOR_VERSION = 3;
+
+ private List disabledPeerIds;
+
+ private CompletableFuture> future;
+
+ private ExecutorService executor;
+
+ private RetryCounter retryCounter;
+
+ @Override
+ public String getGlobalId() {
+ return getClass().getSimpleName();
+ }
+
+ private ProcedureSuspendedException suspend(Configuration conf, LongConsumer backoffConsumer)
+ throws ProcedureSuspendedException {
+ if (retryCounter == null) {
+ retryCounter = ProcedureUtil.createRetryCounter(conf);
+ }
+ long backoff = retryCounter.getBackoffTimeAndIncrementAttempts();
+ backoffConsumer.accept(backoff);
+ throw suspend(Math.toIntExact(backoff), true);
+ }
+
+ private void resetRetry() {
+ retryCounter = null;
+ }
+
+ private ExecutorService getExecutorService() {
+ if (executor == null) {
+ executor = Executors.newCachedThreadPool(new ThreadFactoryBuilder()
+ .setNameFormat(getClass().getSimpleName() + "-%d").setDaemon(true).build());
+ }
+ return executor;
+ }
+
+ private void shutdownExecutorService() {
+ if (executor != null) {
+ executor.shutdown();
+ executor = null;
+ }
+ }
+
+ private void disableReplicationLogCleaner(MasterProcedureEnv env)
+ throws ProcedureSuspendedException {
+ if (!env.getReplicationPeerManager().getReplicationLogCleanerBarrier().disable()) {
+ // it is not likely that we can reach here as we will schedule this procedure immediately
+ // after master restarting, where ReplicationLogCleaner should have not started its first run
+ // yet. But anyway, let's make the code more robust. And it is safe to wait a bit here since
+ // there will be no data in the new replication queue storage before we execute this procedure
+ // so ReplicationLogCleaner will quit immediately without doing anything.
+ throw suspend(env.getMasterConfiguration(),
+ backoff -> LOG.info(
+ "Can not disable replication log cleaner, sleep {} secs and retry later",
+ backoff / 1000));
+ }
+ resetRetry();
+ }
+
+ private void enableReplicationLogCleaner(MasterProcedureEnv env) {
+ env.getReplicationPeerManager().getReplicationLogCleanerBarrier().enable();
+ }
+
+ private void waitUntilNoPeerProcedure(MasterProcedureEnv env) throws ProcedureSuspendedException {
+ long peerProcCount;
+ try {
+ peerProcCount = env.getMasterServices().getProcedures().stream()
+ .filter(p -> p instanceof PeerProcedureInterface).filter(p -> !p.isFinished()).count();
+ } catch (IOException e) {
+ throw suspend(env.getMasterConfiguration(),
+ backoff -> LOG.warn("failed to check peer procedure status, sleep {} secs and retry later",
+ backoff / 1000, e));
+ }
+ if (peerProcCount > 0) {
+ throw suspend(env.getMasterConfiguration(),
+ backoff -> LOG.info(
+ "There are still {} pending peer procedures, sleep {} secs and retry later",
+ peerProcCount, backoff / 1000));
+ }
+ resetRetry();
+ LOG.info("No pending peer procedures found, continue...");
+ }
+
+ @Override
+ protected Flow executeFromState(MasterProcedureEnv env,
+ MigrateReplicationQueueFromZkToTableState state)
+ throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException {
+ switch (state) {
+ case MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_DISABLE_CLEANER:
+ disableReplicationLogCleaner(env);
+ setNextState(MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_PREPARE);
+ return Flow.HAS_MORE_STATE;
+ case MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_PREPARE:
+ waitUntilNoPeerProcedure(env);
+ List peers = env.getReplicationPeerManager().listPeers(null);
+ if (peers.isEmpty()) {
+ LOG.info("No active replication peer found, delete old replication queue data and quit");
+ ZKReplicationQueueStorageForMigration oldStorage =
+ new ZKReplicationQueueStorageForMigration(env.getMasterServices().getZooKeeper(),
+ env.getMasterConfiguration());
+ try {
+ oldStorage.deleteAllData();
+ } catch (KeeperException e) {
+ throw suspend(env.getMasterConfiguration(),
+ backoff -> LOG.warn(
+ "failed to delete old replication queue data, sleep {} secs and retry later",
+ backoff / 1000, e));
+ }
+ setNextState(MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_ENABLE_CLEANER);
+ return Flow.HAS_MORE_STATE;
+ }
+ // here we do not care the peers which have already been disabled, as later we do not need
+ // to enable them
+ disabledPeerIds = peers.stream().filter(ReplicationPeerDescription::isEnabled)
+ .map(ReplicationPeerDescription::getPeerId).collect(Collectors.toList());
+ setNextState(MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_DISABLE_PEER);
+ resetRetry();
+ return Flow.HAS_MORE_STATE;
+ case MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_DISABLE_PEER:
+ for (String peerId : disabledPeerIds) {
+ addChildProcedure(new DisablePeerProcedure(peerId));
+ }
+ setNextState(MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_MIGRATE);
+ return Flow.HAS_MORE_STATE;
+ case MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_MIGRATE:
+ if (future != null) {
+ // should have finished when we arrive here
+ assert future.isDone();
+ try {
+ future.get();
+ } catch (Exception e) {
+ future = null;
+ throw suspend(env.getMasterConfiguration(),
+ backoff -> LOG.warn("failed to migrate queue data, sleep {} secs and retry later",
+ backoff / 1000, e));
+ }
+ shutdownExecutorService();
+ setNextState(MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_WAIT_UPGRADING);
+ resetRetry();
+ return Flow.HAS_MORE_STATE;
+ }
+ future = env.getReplicationPeerManager()
+ .migrateQueuesFromZk(env.getMasterServices().getZooKeeper(), getExecutorService());
+ FutureUtils.addListener(future, (r, e) -> {
+ // should acquire procedure execution lock to make sure that the procedure executor has
+ // finished putting this procedure to the WAITING_TIMEOUT state, otherwise there could be
+ // race and cause unexpected result
+ IdLock procLock =
+ env.getMasterServices().getMasterProcedureExecutor().getProcExecutionLock();
+ IdLock.Entry lockEntry;
+ try {
+ lockEntry = procLock.getLockEntry(getProcId());
+ } catch (IOException ioe) {
+ LOG.error("Error while acquiring execution lock for procedure {}"
+ + " when trying to wake it up, aborting...", ioe);
+ env.getMasterServices().abort("Can not acquire procedure execution lock", e);
+ return;
+ }
+ try {
+ setTimeoutFailure(env);
+ } finally {
+ procLock.releaseLockEntry(lockEntry);
+ }
+ });
+ // here we set timeout to -1 so the ProcedureExecutor will not schedule a Timer for us
+ setTimeout(-1);
+ setState(ProcedureProtos.ProcedureState.WAITING_TIMEOUT);
+ // skip persistence is a must now since when restarting, if the procedure is in
+ // WAITING_TIMEOUT state and has -1 as timeout, it will block there forever...
+ skipPersistence();
+ throw new ProcedureSuspendedException();
+ case MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_WAIT_UPGRADING:
+ long rsWithLowerVersion =
+ env.getMasterServices().getServerManager().getOnlineServers().values().stream()
+ .filter(sm -> VersionInfo.getMajorVersion(sm.getVersion()) < MIN_MAJOR_VERSION).count();
+ if (rsWithLowerVersion == 0) {
+ setNextState(MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_ENABLE_PEER);
+ return Flow.HAS_MORE_STATE;
+ } else {
+ throw suspend(env.getMasterConfiguration(),
+ backoff -> LOG.warn(
+ "There are still {} region servers which have a major version"
+ + " less than {}, sleep {} secs and check later",
+ rsWithLowerVersion, MIN_MAJOR_VERSION, backoff / 1000));
+ }
+ case MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_ENABLE_PEER:
+ for (String peerId : disabledPeerIds) {
+ addChildProcedure(new EnablePeerProcedure(peerId));
+ }
+ setNextState(MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_ENABLE_CLEANER);
+ return Flow.HAS_MORE_STATE;
+ case MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_ENABLE_CLEANER:
+ enableReplicationLogCleaner(env);
+ return Flow.NO_MORE_STATE;
+ default:
+ throw new UnsupportedOperationException("unhandled state=" + state);
+ }
+ }
+
+ @Override
+ protected synchronized boolean setTimeoutFailure(MasterProcedureEnv env) {
+ setState(ProcedureProtos.ProcedureState.RUNNABLE);
+ env.getProcedureScheduler().addFront(this);
+ return false;
+ }
+
+ @Override
+ protected void rollbackState(MasterProcedureEnv env,
+ MigrateReplicationQueueFromZkToTableState state) throws IOException, InterruptedException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ protected MigrateReplicationQueueFromZkToTableState getState(int stateId) {
+ return MigrateReplicationQueueFromZkToTableState.forNumber(stateId);
+ }
+
+ @Override
+ protected int getStateId(MigrateReplicationQueueFromZkToTableState state) {
+ return state.getNumber();
+ }
+
+ @Override
+ protected MigrateReplicationQueueFromZkToTableState getInitialState() {
+ return MIGRATE_REPLICATION_QUEUE_FROM_ZK_TO_TABLE_DISABLE_CLEANER;
+ }
+
+ @Override
+ protected void afterReplay(MasterProcedureEnv env) {
+ if (getCurrentState() == getInitialState()) {
+ // do not need to disable log cleaner or acquire lock if we are in the initial state, later
+ // when executing the procedure we will try to disable and acquire.
+ return;
+ }
+ if (!env.getReplicationPeerManager().getReplicationLogCleanerBarrier().disable()) {
+ throw new IllegalStateException("can not disable log cleaner, this should not happen");
+ }
+ }
+
+ @Override
+ protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException {
+ super.serializeStateData(serializer);
+ MigrateReplicationQueueFromZkToTableStateData.Builder builder =
+ MigrateReplicationQueueFromZkToTableStateData.newBuilder();
+ if (disabledPeerIds != null) {
+ builder.addAllDisabledPeerId(disabledPeerIds);
+ }
+ serializer.serialize(builder.build());
+ }
+
+ @Override
+ protected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException {
+ super.deserializeStateData(serializer);
+ MigrateReplicationQueueFromZkToTableStateData data =
+ serializer.deserialize(MigrateReplicationQueueFromZkToTableStateData.class);
+ disabledPeerIds = data.getDisabledPeerIdList().stream().collect(Collectors.toList());
+ }
+}
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
index 3af902e1d8a4..79bed1503bec 100644
--- 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
@@ -19,6 +19,7 @@
import java.io.IOException;
import java.io.InterruptedIOException;
+import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.client.TableDescriptor;
@@ -27,6 +28,7 @@
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch;
import org.apache.hadoop.hbase.master.procedure.ReopenTableRegionsProcedure;
+import org.apache.hadoop.hbase.procedure2.Procedure;
import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
import org.apache.hadoop.hbase.replication.ReplicationException;
import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
@@ -74,7 +76,7 @@ protected abstract void prePeerModification(MasterProcedureEnv env)
* update the peer storage.
*/
protected abstract void postPeerModification(MasterProcedureEnv env)
- throws IOException, ReplicationException;
+ throws IOException, ReplicationException, ProcedureSuspendedException;
protected void releaseLatch(MasterProcedureEnv env) {
ProcedurePrepareLatch.releaseLatch(latch, this);
@@ -152,12 +154,36 @@ protected void reopenRegions(MasterProcedureEnv env) throws IOException {
}
}
+ private boolean shouldFailForMigrating(MasterProcedureEnv env) throws IOException {
+ long parentProcId = getParentProcId();
+ if (
+ parentProcId != Procedure.NO_PROC_ID && env.getMasterServices().getMasterProcedureExecutor()
+ .getProcedure(parentProcId) instanceof MigrateReplicationQueueFromZkToTableProcedure
+ ) {
+ // this is scheduled by MigrateReplicationQueueFromZkToTableProcedure, should not fail it
+ return false;
+ }
+ return env.getMasterServices().getProcedures().stream()
+ .filter(p -> p instanceof MigrateReplicationQueueFromZkToTableProcedure)
+ .anyMatch(p -> !p.isFinished());
+ }
+
@Override
protected Flow executeFromState(MasterProcedureEnv env, PeerModificationState state)
throws ProcedureSuspendedException, InterruptedException {
switch (state) {
case PRE_PEER_MODIFICATION:
try {
+ if (shouldFailForMigrating(env)) {
+ LOG.info("There is a pending {}, give up execution of {}",
+ MigrateReplicationQueueFromZkToTableProcedure.class.getSimpleName(),
+ getClass().getName());
+ setFailure("master-" + getPeerOperationType().name().toLowerCase() + "-peer",
+ new DoNotRetryIOException("There is a pending "
+ + MigrateReplicationQueueFromZkToTableProcedure.class.getSimpleName()));
+ releaseLatch(env);
+ return Flow.NO_MORE_STATE;
+ }
checkPeerModificationEnabled(env);
prePeerModification(env);
} catch (IOException e) {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/OfflineTableReplicationQueueStorage.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/OfflineTableReplicationQueueStorage.java
new file mode 100644
index 000000000000..9faca74f710d
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/OfflineTableReplicationQueueStorage.java
@@ -0,0 +1,382 @@
+/*
+ * 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.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.ClientSideRegionScanner;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
+import org.apache.hadoop.hbase.replication.ReplicationException;
+import org.apache.hadoop.hbase.replication.ReplicationGroupOffset;
+import org.apache.hadoop.hbase.replication.ReplicationQueueData;
+import org.apache.hadoop.hbase.replication.ReplicationQueueId;
+import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
+import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
+import org.apache.hadoop.hbase.replication.TableReplicationQueueStorage;
+import org.apache.hadoop.hbase.replication.ZKReplicationQueueStorageForMigration;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList;
+import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableMap;
+
+@InterfaceAudience.Private
+public class OfflineTableReplicationQueueStorage implements ReplicationQueueStorage {
+
+ private final Map> offsets =
+ new HashMap<>();
+
+ private final Map> lastSequenceIds = new HashMap<>();
+
+ private final Map> hfileRefs = new HashMap<>();
+
+ private void loadRegionInfo(FileSystem fs, Path regionDir,
+ NavigableMap startKey2RegionInfo) throws IOException {
+ RegionInfo hri = HRegionFileSystem.loadRegionInfoFileContent(fs, regionDir);
+ // TODO: we consider that the there will not be too many regions for hbase:replication table, so
+ // here we just iterate over all the regions to find out the overlapped ones. Can be optimized
+ // later.
+ Iterator> iter = startKey2RegionInfo.entrySet().iterator();
+ while (iter.hasNext()) {
+ Map.Entry entry = iter.next();
+ if (hri.isOverlap(entry.getValue())) {
+ if (hri.getRegionId() > entry.getValue().getRegionId()) {
+ // we are newer, remove the old hri, we can not break here as if hri is a merged region,
+ // we need to remove all its parent regions.
+ iter.remove();
+ } else {
+ // we are older, just return, skip the below add
+ return;
+ }
+ }
+
+ }
+ startKey2RegionInfo.put(hri.getStartKey(), hri);
+ }
+
+ private void loadOffsets(Result result) {
+ NavigableMap map =
+ result.getFamilyMap(TableReplicationQueueStorage.QUEUE_FAMILY);
+ if (map == null || map.isEmpty()) {
+ return;
+ }
+ Map offsetMap = new HashMap<>();
+ map.forEach((k, v) -> {
+ String walGroup = Bytes.toString(k);
+ ReplicationGroupOffset offset = ReplicationGroupOffset.parse(Bytes.toString(v));
+ offsetMap.put(walGroup, offset);
+ });
+ ReplicationQueueId queueId = ReplicationQueueId.parse(Bytes.toString(result.getRow()));
+ offsets.put(queueId, offsetMap);
+ }
+
+ private void loadLastSequenceIds(Result result) {
+ NavigableMap map =
+ result.getFamilyMap(TableReplicationQueueStorage.LAST_SEQUENCE_ID_FAMILY);
+ if (map == null || map.isEmpty()) {
+ return;
+ }
+ Map lastSeqIdMap = new HashMap<>();
+ map.forEach((k, v) -> {
+ String encodedRegionName = Bytes.toString(k);
+ long lastSeqId = Bytes.toLong(v);
+ lastSeqIdMap.put(encodedRegionName, lastSeqId);
+ });
+ String peerId = Bytes.toString(result.getRow());
+ lastSequenceIds.put(peerId, lastSeqIdMap);
+ }
+
+ private void loadHFileRefs(Result result) {
+ NavigableMap map =
+ result.getFamilyMap(TableReplicationQueueStorage.HFILE_REF_FAMILY);
+ if (map == null || map.isEmpty()) {
+ return;
+ }
+ Set refs = new HashSet<>();
+ map.keySet().forEach(ref -> refs.add(Bytes.toString(ref)));
+ String peerId = Bytes.toString(result.getRow());
+ hfileRefs.put(peerId, refs);
+ }
+
+ private void loadReplicationQueueData(Configuration conf, TableName tableName)
+ throws IOException {
+ Path rootDir = CommonFSUtils.getRootDir(conf);
+ Path tableDir = CommonFSUtils.getTableDir(rootDir, tableName);
+ FileSystem fs = tableDir.getFileSystem(conf);
+ FileStatus[] regionDirs =
+ CommonFSUtils.listStatus(fs, tableDir, new FSUtils.RegionDirFilter(fs));
+ if (regionDirs == null) {
+ return;
+ }
+ NavigableMap startKey2RegionInfo = new TreeMap<>(Bytes.BYTES_COMPARATOR);
+ for (FileStatus regionDir : regionDirs) {
+ loadRegionInfo(fs, regionDir.getPath(), startKey2RegionInfo);
+ }
+ TableDescriptor td = ReplicationStorageFactory.createReplicationQueueTableDescriptor(tableName);
+ for (RegionInfo hri : startKey2RegionInfo.values()) {
+ try (ClientSideRegionScanner scanner =
+ new ClientSideRegionScanner(conf, fs, rootDir, td, hri, new Scan(), null)) {
+ for (;;) {
+ Result result = scanner.next();
+ if (result == null) {
+ break;
+ }
+ loadOffsets(result);
+ loadLastSequenceIds(result);
+ loadHFileRefs(result);
+ }
+ }
+ }
+ }
+
+ public OfflineTableReplicationQueueStorage(Configuration conf, TableName tableName)
+ throws IOException {
+ loadReplicationQueueData(conf, tableName);
+ }
+
+ @Override
+ public synchronized void setOffset(ReplicationQueueId queueId, String walGroup,
+ ReplicationGroupOffset offset, Map lastSeqIds) throws ReplicationException {
+ Map offsetMap = offsets.get(queueId);
+ if (offsetMap == null) {
+ offsetMap = new HashMap<>();
+ offsets.put(queueId, offsetMap);
+ }
+ offsetMap.put(walGroup, offset);
+ Map lastSeqIdsMap = lastSequenceIds.get(queueId.getPeerId());
+ if (lastSeqIdsMap == null) {
+ lastSeqIdsMap = new HashMap<>();
+ lastSequenceIds.put(queueId.getPeerId(), lastSeqIdsMap);
+ }
+ for (Map.Entry entry : lastSeqIds.entrySet()) {
+ Long oldSeqId = lastSeqIdsMap.get(entry.getKey());
+ if (oldSeqId == null || oldSeqId < entry.getValue()) {
+ lastSeqIdsMap.put(entry.getKey(), entry.getValue());
+ }
+ }
+ }
+
+ @Override
+ public synchronized Map getOffsets(ReplicationQueueId queueId)
+ throws ReplicationException {
+ Map offsetMap = offsets.get(queueId);
+ if (offsetMap == null) {
+ return Collections.emptyMap();
+ }
+ return ImmutableMap.copyOf(offsetMap);
+ }
+
+ @Override
+ public synchronized List listAllQueueIds(String peerId)
+ throws ReplicationException {
+ return offsets.keySet().stream().filter(rqi -> rqi.getPeerId().equals(peerId))
+ .collect(Collectors.toList());
+ }
+
+ @Override
+ public synchronized List listAllQueueIds(ServerName serverName)
+ throws ReplicationException {
+ return offsets.keySet().stream().filter(rqi -> rqi.getServerName().equals(serverName))
+ .collect(Collectors.toList());
+ }
+
+ @Override
+ public synchronized List listAllQueueIds(String peerId, ServerName serverName)
+ throws ReplicationException {
+ return offsets.keySet().stream()
+ .filter(rqi -> rqi.getPeerId().equals(peerId) && rqi.getServerName().equals(serverName))
+ .collect(Collectors.toList());
+ }
+
+ @Override
+ public synchronized List listAllQueues() throws ReplicationException {
+ return offsets.entrySet().stream()
+ .map(e -> new ReplicationQueueData(e.getKey(), ImmutableMap.copyOf(e.getValue())))
+ .collect(Collectors.toList());
+ }
+
+ @Override
+ public synchronized List listAllReplicators() throws ReplicationException {
+ return offsets.keySet().stream().map(ReplicationQueueId::getServerName).distinct()
+ .collect(Collectors.toList());
+ }
+
+ @Override
+ public synchronized Map claimQueue(ReplicationQueueId queueId,
+ ServerName targetServerName) throws ReplicationException {
+ Map offsetMap = offsets.remove(queueId);
+ if (offsetMap == null) {
+ return Collections.emptyMap();
+ }
+ offsets.put(queueId.claim(targetServerName), offsetMap);
+ return ImmutableMap.copyOf(offsetMap);
+ }
+
+ @Override
+ public synchronized void removeQueue(ReplicationQueueId queueId) throws ReplicationException {
+ offsets.remove(queueId);
+ }
+
+ @Override
+ public synchronized void removeAllQueues(String peerId) throws ReplicationException {
+ Iterator iter = offsets.keySet().iterator();
+ while (iter.hasNext()) {
+ if (iter.next().getPeerId().equals(peerId)) {
+ iter.remove();
+ }
+ }
+ }
+
+ @Override
+ public synchronized long getLastSequenceId(String encodedRegionName, String peerId)
+ throws ReplicationException {
+ Map lastSeqIdMap = lastSequenceIds.get(peerId);
+ if (lastSeqIdMap == null) {
+ return HConstants.NO_SEQNUM;
+ }
+ Long lastSeqId = lastSeqIdMap.get(encodedRegionName);
+ return lastSeqId != null ? lastSeqId.longValue() : HConstants.NO_SEQNUM;
+ }
+
+ @Override
+ public synchronized void setLastSequenceIds(String peerId, Map lastSeqIds)
+ throws ReplicationException {
+ Map lastSeqIdMap = lastSequenceIds.get(peerId);
+ if (lastSeqIdMap == null) {
+ lastSeqIdMap = new HashMap<>();
+ lastSequenceIds.put(peerId, lastSeqIdMap);
+ }
+ lastSeqIdMap.putAll(lastSeqIds);
+ }
+
+ @Override
+ public synchronized void removeLastSequenceIds(String peerId) throws ReplicationException {
+ lastSequenceIds.remove(peerId);
+ }
+
+ @Override
+ public synchronized void removeLastSequenceIds(String peerId, List encodedRegionNames)
+ throws ReplicationException {
+ Map lastSeqIdMap = lastSequenceIds.get(peerId);
+ if (lastSeqIdMap == null) {
+ return;
+ }
+ for (String encodedRegionName : encodedRegionNames) {
+ lastSeqIdMap.remove(encodedRegionName);
+ }
+ }
+
+ @Override
+ public synchronized void removePeerFromHFileRefs(String peerId) throws ReplicationException {
+ hfileRefs.remove(peerId);
+ }
+
+ @Override
+ public synchronized void addHFileRefs(String peerId, List> pairs)
+ throws ReplicationException {
+ Set refs = hfileRefs.get(peerId);
+ if (refs == null) {
+ refs = new HashSet<>();
+ hfileRefs.put(peerId, refs);
+ }
+ for (Pair pair : pairs) {
+ refs.add(pair.getSecond().getName());
+ }
+ }
+
+ @Override
+ public synchronized void removeHFileRefs(String peerId, List files)
+ throws ReplicationException {
+ Set refs = hfileRefs.get(peerId);
+ if (refs == null) {
+ return;
+ }
+ refs.removeAll(files);
+ }
+
+ @Override
+ public synchronized List getAllPeersFromHFileRefsQueue() throws ReplicationException {
+ return ImmutableList.copyOf(hfileRefs.keySet());
+ }
+
+ @Override
+ public synchronized List getReplicableHFiles(String peerId) throws ReplicationException {
+ Set refs = hfileRefs.get(peerId);
+ if (refs == null) {
+ return Collections.emptyList();
+ }
+ return ImmutableList.copyOf(refs);
+ }
+
+ @Override
+ public synchronized Set getAllHFileRefs() throws ReplicationException {
+ return hfileRefs.values().stream().flatMap(Set::stream).collect(Collectors.toSet());
+ }
+
+ @Override
+ public boolean hasData() throws ReplicationException {
+ return true;
+ }
+
+ @Override
+ public void batchUpdateQueues(ServerName serverName, List datas)
+ throws ReplicationException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public void batchUpdateLastSequenceIds(
+ List lastPushedSeqIds)
+ throws ReplicationException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public void batchUpdateHFileRefs(String peerId, List hfileRefs)
+ throws ReplicationException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public void removeLastSequenceIdsAndHFileRefsBefore(long ts) throws ReplicationException {
+ throw new UnsupportedOperationException();
+ }
+}
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
index 2042e8468497..2fadc3fd6642 100644
--- 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
@@ -18,10 +18,17 @@
package org.apache.hadoop.hbase.master.replication;
import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
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.Procedure;
+import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
+import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
import org.apache.hadoop.hbase.replication.ReplicationException;
import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
import org.apache.yetus.audience.InterfaceAudience;
@@ -40,6 +47,8 @@ public class RemovePeerProcedure extends ModifyPeerProcedure {
private ReplicationPeerConfig peerConfig;
+ private List ongoingAssignReplicationQueuesProcIds = Collections.emptyList();
+
public RemovePeerProcedure() {
}
@@ -64,15 +73,43 @@ protected void prePeerModification(MasterProcedureEnv env) throws IOException {
@Override
protected void updatePeerStorage(MasterProcedureEnv env) throws ReplicationException {
env.getReplicationPeerManager().removePeer(peerId);
+ // record ongoing AssignReplicationQueuesProcedures after we update the peer storage
+ ongoingAssignReplicationQueuesProcIds = env.getMasterServices().getMasterProcedureExecutor()
+ .getProcedures().stream().filter(p -> p instanceof AssignReplicationQueuesProcedure)
+ .filter(p -> !p.isFinished()).map(Procedure::getProcId).collect(Collectors.toList());
}
private void removeRemoteWALs(MasterProcedureEnv env) throws IOException {
env.getMasterServices().getSyncReplicationReplayWALManager().removePeerRemoteWALs(peerId);
}
+ private void checkAssignReplicationQueuesFinished(MasterProcedureEnv env)
+ throws ProcedureSuspendedException {
+ if (ongoingAssignReplicationQueuesProcIds.isEmpty()) {
+ LOG.info("No ongoing assign replication queues procedures when removing peer {}, move on",
+ peerId);
+ }
+ ProcedureExecutor procExec =
+ env.getMasterServices().getMasterProcedureExecutor();
+ long[] unfinishedProcIds =
+ ongoingAssignReplicationQueuesProcIds.stream().map(procExec::getProcedure)
+ .filter(p -> p != null && !p.isFinished()).mapToLong(Procedure::getProcId).toArray();
+ if (unfinishedProcIds.length == 0) {
+ LOG.info(
+ "All assign replication queues procedures are finished when removing peer {}, move on",
+ peerId);
+ } else {
+ throw suspend(env.getMasterConfiguration(), backoff -> LOG.info(
+ "There are still {} pending assign replication queues procedures {} when removing peer {}, sleep {} secs",
+ unfinishedProcIds.length, Arrays.toString(unfinishedProcIds), peerId, backoff / 1000));
+ }
+ }
+
@Override
protected void postPeerModification(MasterProcedureEnv env)
- throws IOException, ReplicationException {
+ throws IOException, ReplicationException, ProcedureSuspendedException {
+ checkAssignReplicationQueuesFinished(env);
+
if (peerConfig.isSyncReplication()) {
removeRemoteWALs(env);
}
@@ -94,6 +131,7 @@ protected void serializeStateData(ProcedureStateSerializer serializer) throws IO
if (peerConfig != null) {
builder.setPeerConfig(ReplicationPeerConfigUtil.convert(peerConfig));
}
+ builder.addAllOngoingAssignReplicationQueuesProcIds(ongoingAssignReplicationQueuesProcIds);
serializer.serialize(builder.build());
}
@@ -104,5 +142,6 @@ protected void deserializeStateData(ProcedureStateSerializer serializer) throws
if (data.hasPeerConfig()) {
this.peerConfig = ReplicationPeerConfigUtil.convert(data.getPeerConfig());
}
+ ongoingAssignReplicationQueuesProcIds = data.getOngoingAssignReplicationQueuesProcIdsList();
}
}
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
index bfb7b7c10c08..8b01225e553e 100644
--- 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
@@ -17,18 +17,23 @@
*/
package org.apache.hadoop.hbase.master.replication;
+import com.google.errorprone.annotations.RestrictedApi;
import java.io.IOException;
import java.net.URI;
import java.util.ArrayList;
import java.util.Collection;
import java.util.EnumSet;
+import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
+import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutorService;
import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
import java.util.regex.Pattern;
import java.util.stream.Collectors;
import org.apache.commons.lang3.StringUtils;
@@ -42,19 +47,33 @@
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil;
import org.apache.hadoop.hbase.conf.ConfigurationObserver;
+import org.apache.hadoop.hbase.master.MasterServices;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.master.procedure.ProcedureSyncWait;
+import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
import org.apache.hadoop.hbase.replication.BaseReplicationEndpoint;
import org.apache.hadoop.hbase.replication.HBaseReplicationEndpoint;
import org.apache.hadoop.hbase.replication.ReplicationEndpoint;
import org.apache.hadoop.hbase.replication.ReplicationException;
+import org.apache.hadoop.hbase.replication.ReplicationGroupOffset;
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.ReplicationQueueData;
+import org.apache.hadoop.hbase.replication.ReplicationQueueId;
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.replication.SyncReplicationState;
+import org.apache.hadoop.hbase.replication.ZKReplicationQueueStorageForMigration;
+import org.apache.hadoop.hbase.replication.ZKReplicationQueueStorageForMigration.MigrationIterator;
+import org.apache.hadoop.hbase.replication.ZKReplicationQueueStorageForMigration.ZkLastPushedSeqId;
+import org.apache.hadoop.hbase.replication.ZKReplicationQueueStorageForMigration.ZkReplicationQueueData;
+import org.apache.hadoop.hbase.replication.master.ReplicationLogCleanerBarrier;
+import org.apache.hadoop.hbase.util.FutureUtils;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
import org.apache.hadoop.hbase.zookeeper.ZKConfig;
import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
@@ -96,6 +115,9 @@ public class ReplicationPeerManager implements ConfigurationObserver {
// Only allow to add one sync replication peer concurrently
private final Semaphore syncReplicationPeerLock = new Semaphore(1);
+ private final ReplicationLogCleanerBarrier replicationLogCleanerBarrier =
+ new ReplicationLogCleanerBarrier();
+
private final String clusterId;
private volatile Configuration conf;
@@ -105,9 +127,20 @@ public class ReplicationPeerManager implements ConfigurationObserver {
private final ZKWatcher zk;
+ @FunctionalInterface
+ interface ReplicationQueueStorageInitializer {
+
+ void initialize() throws IOException;
+ }
+
+ private final ReplicationQueueStorageInitializer queueStorageInitializer;
+
+ // we will mock this class in UT so leave the constructor as package private and not mark the
+ // class as final, since mockito can not mock a final class
ReplicationPeerManager(FileSystem fs, ZKWatcher zk, ReplicationPeerStorage peerStorage,
ReplicationQueueStorage queueStorage, ConcurrentMap peers,
- Configuration conf, String clusterId) {
+ Configuration conf, String clusterId,
+ ReplicationQueueStorageInitializer queueStorageInitializer) {
this.fs = fs;
this.zk = zk;
this.peerStorage = peerStorage;
@@ -115,27 +148,27 @@ public class ReplicationPeerManager implements ConfigurationObserver {
this.peers = peers;
this.conf = conf;
this.clusterId = clusterId;
+ this.queueStorageInitializer = queueStorageInitializer;
}
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);
- }
- }
+ List queueIds = queueStorage.listAllQueueIds(peerId);
+ if (!queueIds.isEmpty()) {
+ throw new DoNotRetryIOException("There are still " + queueIds.size()
+ + " undeleted queue(s) for peerId: " + peerId + ", first is " + queueIds.get(0));
}
if (queueStorage.getAllPeersFromHFileRefsQueue().contains(peerId)) {
throw new DoNotRetryIOException("Undeleted queue for peer " + peerId + " in hfile-refs");
}
}
+ private void initializeQueueStorage() throws IOException {
+ queueStorageInitializer.initialize();
+ }
+
void preAddPeer(String peerId, ReplicationPeerConfig peerConfig)
- throws DoNotRetryIOException, ReplicationException {
+ throws ReplicationException, IOException {
if (peerId.contains("-")) {
throw new DoNotRetryIOException("Found invalid peer name: " + peerId);
}
@@ -146,6 +179,9 @@ void preAddPeer(String peerId, ReplicationPeerConfig peerConfig)
if (peers.containsKey(peerId)) {
throw new DoNotRetryIOException("Replication peer " + peerId + " already exists");
}
+
+ // lazy create table
+ initializeQueueStorage();
// 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
@@ -365,8 +401,8 @@ public void removeAllQueues(String peerId) throws ReplicationException {
// 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.removeAllQueues(peerId);
+ queueStorage.removeAllQueues(peerId);
}
public void removeAllQueuesAndHFileRefs(String peerId) throws ReplicationException {
@@ -568,14 +604,69 @@ public List getSerialPeerIdsBelongsTo(TableName tableName) {
.collect(Collectors.toList());
}
+ @RestrictedApi(explanation = "Should only be called in tests", link = "",
+ allowedOnPath = ".*/src/test/.*")
+ public ReplicationPeerStorage getPeerStorage() {
+ return peerStorage;
+ }
+
public ReplicationQueueStorage getQueueStorage() {
return queueStorage;
}
- public static ReplicationPeerManager create(FileSystem fs, ZKWatcher zk, Configuration conf,
- String clusterId) throws ReplicationException {
+ private static Pair
+ createReplicationQueueStorage(MasterServices services) throws IOException {
+ Configuration conf = services.getConfiguration();
+ TableName replicationQueueTableName =
+ TableName.valueOf(conf.get(ReplicationStorageFactory.REPLICATION_QUEUE_TABLE_NAME,
+ ReplicationStorageFactory.REPLICATION_QUEUE_TABLE_NAME_DEFAULT.getNameAsString()));
+ ReplicationQueueStorageInitializer initializer;
+ if (services.getTableDescriptors().exists(replicationQueueTableName)) {
+ // no need to create the table
+ initializer = () -> {
+ };
+ } else {
+ // lazy create the replication table.
+ initializer = new ReplicationQueueStorageInitializer() {
+
+ private volatile boolean created = false;
+
+ @Override
+ public void initialize() throws IOException {
+ if (created) {
+ return;
+ }
+ synchronized (this) {
+ if (created) {
+ return;
+ }
+ if (services.getTableDescriptors().exists(replicationQueueTableName)) {
+ created = true;
+ return;
+ }
+ long procId = services.createSystemTable(ReplicationStorageFactory
+ .createReplicationQueueTableDescriptor(replicationQueueTableName));
+ ProcedureExecutor procExec = services.getMasterProcedureExecutor();
+ ProcedureSyncWait.waitFor(procExec.getEnvironment(), TimeUnit.MINUTES.toMillis(1),
+ "Creating table " + replicationQueueTableName, () -> procExec.isFinished(procId));
+ }
+ }
+ };
+ }
+ return Pair.newPair(ReplicationStorageFactory.getReplicationQueueStorage(
+ services.getConnection(), conf, replicationQueueTableName), initializer);
+ }
+
+ public static ReplicationPeerManager create(MasterServices services, String clusterId)
+ throws ReplicationException, IOException {
+ Configuration conf = services.getConfiguration();
+ FileSystem fs = services.getMasterFileSystem().getFileSystem();
+ ZKWatcher zk = services.getZooKeeper();
ReplicationPeerStorage peerStorage =
ReplicationStorageFactory.getReplicationPeerStorage(fs, zk, conf);
+ Pair pair =
+ createReplicationQueueStorage(services);
+ ReplicationQueueStorage queueStorage = pair.getFirst();
ConcurrentMap peers = new ConcurrentHashMap<>();
for (String peerId : peerStorage.listPeerIds()) {
ReplicationPeerConfig peerConfig = peerStorage.getPeerConfig(peerId);
@@ -585,7 +676,24 @@ public static ReplicationPeerManager create(FileSystem fs, ZKWatcher zk, Configu
) {
// we do not use this endpoint for region replication any more, see HBASE-26233
LOG.info("Legacy region replication peer found, removing: {}", peerConfig);
- peerStorage.removePeer(peerId);
+ // do it asynchronous to not block the start up of HMaster
+ new Thread("Remove legacy replication peer " + peerId) {
+
+ @Override
+ public void run() {
+ try {
+ // need to delete two times to make sure we delete all the queues, see the comments in
+ // above
+ // removeAllQueues method for more details.
+ queueStorage.removeAllQueues(peerId);
+ queueStorage.removeAllQueues(peerId);
+ // delete queue first and then peer, because we use peer as a flag.
+ peerStorage.removePeer(peerId);
+ } catch (Exception e) {
+ LOG.warn("Failed to delete legacy replication peer {}", peerId);
+ }
+ }
+ }.start();
continue;
}
peerConfig = ReplicationPeerConfigUtil.updateReplicationBasePeerConfigs(conf, peerConfig);
@@ -594,8 +702,8 @@ public static ReplicationPeerManager create(FileSystem fs, ZKWatcher zk, Configu
SyncReplicationState state = peerStorage.getPeerSyncReplicationState(peerId);
peers.put(peerId, new ReplicationPeerDescription(peerId, enabled, peerConfig, state));
}
- return new ReplicationPeerManager(fs, zk, peerStorage,
- ReplicationStorageFactory.getReplicationQueueStorage(zk, conf), peers, conf, clusterId);
+ return new ReplicationPeerManager(fs, zk, peerStorage, queueStorage, peers, conf, clusterId,
+ pair.getSecond());
}
/**
@@ -617,9 +725,110 @@ public void releaseSyncReplicationPeerLock() {
syncReplicationPeerLock.release();
}
+ public ReplicationLogCleanerBarrier getReplicationLogCleanerBarrier() {
+ return replicationLogCleanerBarrier;
+ }
+
@Override
public void onConfigurationChange(Configuration conf) {
this.conf = conf;
this.peerStorage = ReplicationStorageFactory.getReplicationPeerStorage(fs, zk, conf);
}
+
+ private ReplicationQueueData convert(ZkReplicationQueueData zkData) {
+ Map groupOffsets = new HashMap<>();
+ zkData.getWalOffsets().forEach((wal, offset) -> {
+ String walGroup = AbstractFSWALProvider.getWALPrefixFromWALName(wal);
+ groupOffsets.compute(walGroup, (k, oldOffset) -> {
+ if (oldOffset == null) {
+ return new ReplicationGroupOffset(wal, offset);
+ }
+ // we should record the first wal's offset
+ long oldWalTs = AbstractFSWALProvider.getTimestamp(oldOffset.getWal());
+ long walTs = AbstractFSWALProvider.getTimestamp(wal);
+ if (walTs < oldWalTs) {
+ return new ReplicationGroupOffset(wal, offset);
+ }
+ return oldOffset;
+ });
+ });
+ return new ReplicationQueueData(zkData.getQueueId(), ImmutableMap.copyOf(groupOffsets));
+ }
+
+ private void migrateQueues(ZKReplicationQueueStorageForMigration oldQueueStorage)
+ throws Exception {
+ MigrationIterator>> iter =
+ oldQueueStorage.listAllQueues();
+ for (;;) {
+ Pair> pair = iter.next();
+ if (pair == null) {
+ return;
+ }
+ queueStorage.batchUpdateQueues(pair.getFirst(),
+ pair.getSecond().stream().filter(data -> peers.containsKey(data.getQueueId().getPeerId()))
+ .map(this::convert).collect(Collectors.toList()));
+ }
+ }
+
+ private void migrateLastPushedSeqIds(ZKReplicationQueueStorageForMigration oldQueueStorage)
+ throws Exception {
+ MigrationIterator> iter = oldQueueStorage.listAllLastPushedSeqIds();
+ for (;;) {
+ List list = iter.next();
+ if (list == null) {
+ return;
+ }
+ queueStorage.batchUpdateLastSequenceIds(list.stream()
+ .filter(data -> peers.containsKey(data.getPeerId())).collect(Collectors.toList()));
+ }
+ }
+
+ private void migrateHFileRefs(ZKReplicationQueueStorageForMigration oldQueueStorage)
+ throws Exception {
+ MigrationIterator>> iter = oldQueueStorage.listAllHFileRefs();
+ for (;;) {
+ Pair> pair = iter.next();
+ if (pair == null) {
+ return;
+ }
+ if (peers.containsKey(pair.getFirst())) {
+ queueStorage.batchUpdateHFileRefs(pair.getFirst(), pair.getSecond());
+ }
+ }
+ }
+
+ private interface ExceptionalRunnable {
+ void run() throws Exception;
+ }
+
+ private CompletableFuture> runAsync(ExceptionalRunnable task, ExecutorService executor) {
+ CompletableFuture> future = new CompletableFuture<>();
+ executor.execute(() -> {
+ try {
+ task.run();
+ future.complete(null);
+ } catch (Exception e) {
+ future.completeExceptionally(e);
+ }
+ });
+ return future;
+ }
+
+ /**
+ * Submit the migration tasks to the given {@code executor}.
+ */
+ CompletableFuture> migrateQueuesFromZk(ZKWatcher zookeeper, ExecutorService executor) {
+ // the replication queue table creation is asynchronous and will be triggered by addPeer, so
+ // here we need to manually initialize it since we will not call addPeer.
+ try {
+ initializeQueueStorage();
+ } catch (IOException e) {
+ return FutureUtils.failedFuture(e);
+ }
+ ZKReplicationQueueStorageForMigration oldStorage =
+ new ZKReplicationQueueStorageForMigration(zookeeper, conf);
+ return CompletableFuture.allOf(runAsync(() -> migrateQueues(oldStorage), executor),
+ runAsync(() -> migrateLastPushedSeqIds(oldStorage), executor),
+ runAsync(() -> migrateHFileRefs(oldStorage), executor));
+ }
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/TransitPeerSyncReplicationStateProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/TransitPeerSyncReplicationStateProcedure.java
index ed0760c69924..df6078d64bed 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/TransitPeerSyncReplicationStateProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/TransitPeerSyncReplicationStateProcedure.java
@@ -28,6 +28,7 @@
import org.apache.hadoop.hbase.master.procedure.ReopenTableRegionsProcedure;
import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
+import org.apache.hadoop.hbase.procedure2.StateMachineProcedure.Flow;
import org.apache.hadoop.hbase.replication.ReplicationException;
import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
import org.apache.hadoop.hbase.replication.ReplicationUtils;
@@ -236,6 +237,19 @@ protected Flow executeFromState(MasterProcedureEnv env,
switch (state) {
case PRE_PEER_SYNC_REPLICATION_STATE_TRANSITION:
try {
+ if (
+ env.getMasterServices().getProcedures().stream()
+ .filter(p -> p instanceof MigrateReplicationQueueFromZkToTableProcedure)
+ .anyMatch(p -> !p.isFinished())
+ ) {
+ LOG.info("There is a pending {}, give up execution of {}",
+ MigrateReplicationQueueFromZkToTableProcedure.class.getSimpleName(),
+ getClass().getSimpleName());
+ setFailure("master-transit-peer-sync-replication-state",
+ new DoNotRetryIOException("There is a pending "
+ + MigrateReplicationQueueFromZkToTableProcedure.class.getSimpleName()));
+ return Flow.NO_MORE_STATE;
+ }
checkPeerModificationEnabled(env);
preTransit(env);
} catch (IOException e) {
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 6daae10b726f..1bdf6a225c62 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
@@ -1733,7 +1733,7 @@ public boolean isOnline() {
* be hooked up to WAL.
*/
private void setupWALAndReplication() throws IOException {
- WALFactory factory = new WALFactory(conf, serverName.toString(), this, true);
+ WALFactory factory = new WALFactory(conf, serverName, this, true);
// TODO Replication make assumptions here based on the default filesystem impl
Path oldLogDir = new Path(walRootDir, HConstants.HREGION_OLDLOGDIR_NAME);
String logName = AbstractFSWALProvider.getWALDirectoryName(this.serverName.toString());
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java
index dd8c9c551270..6c7fc504b5fd 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java
@@ -982,12 +982,12 @@ synchronized public void run() {
lastRan = currentTime;
- final WALProvider provider = regionServer.getWalFactory().getWALProvider();
- final WALProvider metaProvider = regionServer.getWalFactory().getMetaWALProvider();
- numWALFiles = (provider == null ? 0 : provider.getNumLogFiles())
- + (metaProvider == null ? 0 : metaProvider.getNumLogFiles());
- walFileSize = (provider == null ? 0 : provider.getLogFileSize())
- + (metaProvider == null ? 0 : metaProvider.getLogFileSize());
+ List providers = regionServer.getWalFactory().getAllWALProviders();
+ for (WALProvider provider : providers) {
+ numWALFiles += provider.getNumLogFiles();
+ walFileSize += provider.getLogFileSize();
+ }
+
// Copy over computed values so that no thread sees half computed values.
numStores = tempNumStores;
numStoreFiles = tempNumStoreFiles;
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java
index dc4c3a0a6bcb..8df65487c676 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java
@@ -32,6 +32,7 @@
import java.io.InterruptedIOException;
import java.lang.management.MemoryType;
import java.net.URLEncoder;
+import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Comparator;
@@ -441,8 +442,9 @@ protected AbstractFSWAL(final FileSystem fs, final Abortable abortable, final Pa
}
// If prefix is null||empty then just name it wal
- this.walFilePrefix =
- prefix == null || prefix.isEmpty() ? "wal" : URLEncoder.encode(prefix, "UTF8");
+ this.walFilePrefix = prefix == null || prefix.isEmpty()
+ ? "wal"
+ : URLEncoder.encode(prefix, StandardCharsets.UTF_8.name());
// we only correctly differentiate suffices when numeric ones start with '.'
if (suffix != null && !(suffix.isEmpty()) && !(suffix.startsWith(WAL_FILE_NAME_DELIMITER))) {
throw new IllegalArgumentException("WAL suffix must start with '" + WAL_FILE_NAME_DELIMITER
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ReplicationOffsetUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ReplicationOffsetUtil.java
new file mode 100644
index 000000000000..052c5542d47a
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ReplicationOffsetUtil.java
@@ -0,0 +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.
+ */
+package org.apache.hadoop.hbase.replication;
+
+import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
+import org.apache.yetus.audience.InterfaceAudience;
+
+@InterfaceAudience.Private
+public final class ReplicationOffsetUtil {
+
+ private ReplicationOffsetUtil() {
+ }
+
+ public static boolean shouldReplicate(ReplicationGroupOffset offset, String wal) {
+ // if no offset or the offset is just a place marker, replicate
+ if (offset == null || offset == ReplicationGroupOffset.BEGIN) {
+ return true;
+ }
+ // otherwise, compare the timestamp
+ long walTs = AbstractFSWALProvider.getTimestamp(wal);
+ long startWalTs = AbstractFSWALProvider.getTimestamp(offset.getWal());
+ if (walTs < startWalTs) {
+ return false;
+ } else if (walTs > startWalTs) {
+ return true;
+ }
+ // if the timestamp equals, usually it means we should include this wal but there is a special
+ // case, a negative offset means the wal has already been fully replicated, so here we should
+ // check the offset.
+ return offset.getOffset() >= 0;
+ }
+}
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 819e4c5e54ac..00e875f8be56 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
@@ -19,23 +19,28 @@
import java.io.IOException;
import java.util.Collections;
+import java.util.Map;
import java.util.Set;
-import org.apache.hadoop.conf.Configuration;
+import org.apache.commons.io.IOUtils;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.Server;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.master.cleaner.BaseHFileCleanerDelegate;
import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
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.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.base.Predicate;
import org.apache.hbase.thirdparty.com.google.common.collect.Iterables;
+import org.apache.hbase.thirdparty.org.apache.commons.collections4.MapUtils;
/**
* Implementation of a file cleaner that checks if a hfile is still scheduled for replication before
@@ -44,15 +49,20 @@
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
public class ReplicationHFileCleaner extends BaseHFileCleanerDelegate {
private static final Logger LOG = LoggerFactory.getLogger(ReplicationHFileCleaner.class);
- private ZKWatcher zkw;
+ private Connection conn;
+ private boolean shareConn;
private ReplicationQueueStorage rqs;
private boolean stopped = false;
@Override
public Iterable getDeletableFiles(Iterable files) {
- // all members of this class are null if replication is disabled,
- // so we cannot filter the files
- if (this.getConf() == null) {
+ if (
+ !(getConf().getBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY,
+ HConstants.REPLICATION_BULKLOAD_ENABLE_DEFAULT))
+ ) {
+ LOG.warn(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY + " is not enabled. Better to remove "
+ + ReplicationHFileCleaner.class + " from " + HFileCleaner.MASTER_HFILE_CLEANER_PLUGINS
+ + " configuration.");
return files;
}
@@ -88,51 +98,34 @@ public boolean apply(FileStatus file) {
}
@Override
- public void setConf(Configuration config) {
- // If either replication or replication of bulk load hfiles is disabled, keep all members null
- if (
- !(config.getBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY,
- HConstants.REPLICATION_BULKLOAD_ENABLE_DEFAULT))
- ) {
- LOG.warn(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY + " is not enabled. Better to remove "
- + ReplicationHFileCleaner.class + " from " + HFileCleaner.MASTER_HFILE_CLEANER_PLUGINS
- + " configuration.");
- return;
- }
- // Make my own Configuration. Then I'll have my own connection to zk that
- // I can close myself when time comes.
- Configuration conf = new Configuration(config);
+ public void init(Map params) {
+ super.init(params);
try {
- setConf(conf, new ZKWatcher(conf, "replicationHFileCleaner", null));
+ if (MapUtils.isNotEmpty(params)) {
+ Object master = params.get(HMaster.MASTER);
+ if (master != null && master instanceof Server) {
+ conn = ((Server) master).getConnection();
+ shareConn = true;
+ }
+ }
+ if (conn == null) {
+ conn = ConnectionFactory.createConnection(getConf());
+ }
+ this.rqs = ReplicationStorageFactory.getReplicationQueueStorage(conn, getConf());
} catch (IOException e) {
LOG.error("Error while configuring " + this.getClass().getName(), e);
}
}
- @InterfaceAudience.Private
- public void setConf(Configuration conf, ZKWatcher zk) {
- super.setConf(conf);
- try {
- initReplicationQueueStorage(conf, zk);
- } catch (Exception e) {
- LOG.error("Error while configuring " + this.getClass().getName(), e);
- }
- }
-
- private void initReplicationQueueStorage(Configuration conf, ZKWatcher zk) {
- this.zkw = zk;
- this.rqs = ReplicationStorageFactory.getReplicationQueueStorage(zk, conf);
- }
-
@Override
public void stop(String why) {
if (this.stopped) {
return;
}
this.stopped = true;
- if (this.zkw != null) {
- LOG.info("Stopping " + this.zkw);
- this.zkw.close();
+ if (!shareConn && this.conn != null) {
+ LOG.info("Stopping " + this.conn);
+ IOUtils.closeQuietly(conn);
}
}
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 54e600e09ada..3ab52da6158e 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
@@ -17,20 +17,29 @@
*/
package org.apache.hadoop.hbase.replication.master;
-import java.io.IOException;
import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
import java.util.Map;
import java.util.Set;
-import org.apache.hadoop.conf.Configuration;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.master.cleaner.BaseLogCleanerDelegate;
+import org.apache.hadoop.hbase.master.procedure.ServerCrashProcedure;
+import org.apache.hadoop.hbase.master.replication.ReplicationPeerManager;
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.hadoop.hbase.replication.ReplicationGroupOffset;
+import org.apache.hadoop.hbase.replication.ReplicationOffsetUtil;
+import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
+import org.apache.hadoop.hbase.replication.ReplicationQueueData;
+import org.apache.hadoop.hbase.replication.ReplicationQueueId;
+import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -42,34 +51,137 @@
/**
* Implementation of a log cleaner that checks if a log is still scheduled for replication before
* deleting it when its TTL is over.
+ *
+ * The logic is a bit complicated after we switch to use table based replication queue storage, see
+ * the design doc in HBASE-27109 and the comments in HBASE-27214 for more details.
*/
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
public class ReplicationLogCleaner extends BaseLogCleanerDelegate {
private static final Logger LOG = LoggerFactory.getLogger(ReplicationLogCleaner.class);
- private ZKWatcher zkw = null;
- private boolean shareZK = false;
- private ReplicationQueueStorage queueStorage;
+ private Set notFullyDeadServers;
+ private Set peerIds;
+ // ServerName -> PeerId -> WalGroup -> Offset
+ // Here the server name is the source server name, so we can make sure that there is only one
+ // queue for a given peer, that why we can use a String peerId as key instead of
+ // ReplicationQueueId.
+ private Map>> replicationOffsets;
+ private ReplicationPeerManager rpm;
+ private Supplier> getNotFullyDeadServers;
+
+ private boolean canFilter;
private boolean stopped = false;
- private Set wals;
- private long readZKTimestamp = 0;
@Override
public void preClean() {
- readZKTimestamp = EnvironmentEdgeManager.currentTime();
+ if (this.getConf() == null) {
+ return;
+ }
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 = queueStorage.getAllWALs();
+ if (!rpm.getQueueStorage().hasData()) {
+ return;
+ }
} catch (ReplicationException e) {
- LOG.warn("Failed to read zookeeper, skipping checking deletable files");
- wals = null;
+ LOG.error("Error occurred while executing queueStorage.hasData()", e);
+ return;
+ }
+ canFilter = rpm.getReplicationLogCleanerBarrier().start();
+ if (canFilter) {
+ notFullyDeadServers = getNotFullyDeadServers.get();
+ peerIds = rpm.listPeers(null).stream().map(ReplicationPeerDescription::getPeerId)
+ .collect(Collectors.toSet());
+ // must get the not fully dead servers first and then get the replication queue data, in this
+ // way we can make sure that, we should have added the missing replication queues for the dead
+ // region servers recorded in the above set, otherwise the logic in the
+ // filterForDeadRegionServer method may lead us delete wal still in use.
+ List allQueueData;
+ try {
+ allQueueData = rpm.getQueueStorage().listAllQueues();
+ } catch (ReplicationException e) {
+ LOG.error("Can not list all replication queues, give up cleaning", e);
+ rpm.getReplicationLogCleanerBarrier().stop();
+ canFilter = false;
+ notFullyDeadServers = null;
+ peerIds = null;
+ return;
+ }
+ replicationOffsets = new HashMap<>();
+ for (ReplicationQueueData queueData : allQueueData) {
+ ReplicationQueueId queueId = queueData.getId();
+ ServerName serverName = queueId.getServerWALsBelongTo();
+ Map> peerId2Offsets =
+ replicationOffsets.computeIfAbsent(serverName, k -> new HashMap<>());
+ Map offsets =
+ peerId2Offsets.computeIfAbsent(queueId.getPeerId(), k -> new HashMap<>());
+ offsets.putAll(queueData.getOffsets());
+ }
+ } else {
+ LOG.info("Skip replication log cleaner because an AddPeerProcedure is running");
}
}
@Override
public void postClean() {
- // release memory
- wals = null;
+ if (canFilter) {
+ rpm.getReplicationLogCleanerBarrier().stop();
+ canFilter = false;
+ // release memory
+ notFullyDeadServers = null;
+ peerIds = null;
+ replicationOffsets = null;
+ }
+ }
+
+ private boolean shouldDelete(ReplicationGroupOffset offset, FileStatus file) {
+ return !ReplicationOffsetUtil.shouldReplicate(offset, file.getPath().getName());
+ }
+
+ private boolean filterForLiveRegionServer(ServerName serverName, FileStatus file) {
+ Map> peerId2Offsets =
+ replicationOffsets.get(serverName);
+ if (peerId2Offsets == null) {
+ // if there are replication queues missing, we can not delete the wal
+ return false;
+ }
+ for (String peerId : peerIds) {
+ Map offsets = peerId2Offsets.get(peerId);
+ // if no replication queue for a peer, we can not delete the wal
+ if (offsets == null) {
+ return false;
+ }
+ String walGroupId = AbstractFSWALProvider.getWALPrefixFromWALName(file.getPath().getName());
+ ReplicationGroupOffset offset = offsets.get(walGroupId);
+ // if a replication queue still need to replicate this wal, we can not delete it
+ if (!shouldDelete(offset, file)) {
+ return false;
+ }
+ }
+ // if all replication queues have already finished replicating this wal, we can delete it.
+ return true;
+ }
+
+ private boolean filterForDeadRegionServer(ServerName serverName, FileStatus file) {
+ Map> peerId2Offsets =
+ replicationOffsets.get(serverName);
+ if (peerId2Offsets == null) {
+ // no replication queue for this dead rs, we can delete all wal files for it
+ return true;
+ }
+ for (String peerId : peerIds) {
+ Map offsets = peerId2Offsets.get(peerId);
+ if (offsets == null) {
+ // for dead server, we only care about existing replication queues, as we will delete a
+ // queue after we finish replicating it.
+ continue;
+ }
+ String walGroupId = AbstractFSWALProvider.getWALPrefixFromWALName(file.getPath().getName());
+ ReplicationGroupOffset offset = offsets.get(walGroupId);
+ // if a replication queue still need to replicate this wal, we can not delete it
+ if (!shouldDelete(offset, file)) {
+ return false;
+ }
+ }
+ // if all replication queues have already finished replicating this wal, we can delete it.
+ return true;
}
@Override
@@ -79,10 +191,12 @@ public Iterable getDeletableFiles(Iterable files) {
if (this.getConf() == null) {
return files;
}
-
- if (wals == null) {
+ if (!canFilter) {
+ // We can not delete anything if there are AddPeerProcedure running at the same time
+ // See HBASE-27214 for more details.
return Collections.emptyList();
}
+
return Iterables.filter(files, new Predicate() {
@Override
public boolean apply(FileStatus file) {
@@ -91,63 +205,56 @@ public boolean apply(FileStatus file) {
if (file == null) {
return false;
}
- String wal = file.getPath().getName();
- boolean logInReplicationQueue = wals.contains(wal);
- if (logInReplicationQueue) {
- LOG.debug("Found up in ZooKeeper, NOT deleting={}", wal);
+ if (peerIds.isEmpty()) {
+ // no peer, can always delete
+ return true;
+ }
+ // not a valid wal file name, delete
+ if (!AbstractFSWALProvider.validateWALFilename(file.getPath().getName())) {
+ return true;
+ }
+ // meta wal is always deletable as we will never replicate it
+ if (AbstractFSWALProvider.isMetaFile(file.getPath())) {
+ return true;
+ }
+ ServerName serverName =
+ AbstractFSWALProvider.parseServerNameFromWALName(file.getPath().getName());
+ if (notFullyDeadServers.contains(serverName)) {
+ return filterForLiveRegionServer(serverName, file);
+ } else {
+ return filterForDeadRegionServer(serverName, file);
}
- return !logInReplicationQueue && (file.getModificationTime() < readZKTimestamp);
}
});
}
+ private Set getNotFullyDeadServers(MasterServices services) {
+ List onlineServers = services.getServerManager().getOnlineServersList();
+ return Stream.concat(onlineServers.stream(),
+ services.getMasterProcedureExecutor().getProcedures().stream()
+ .filter(p -> p instanceof ServerCrashProcedure).filter(p -> !p.isFinished())
+ .map(p -> ((ServerCrashProcedure) p).getServerName()))
+ .collect(Collectors.toSet());
+ }
+
@Override
public void init(Map params) {
super.init(params);
- try {
- if (MapUtils.isNotEmpty(params)) {
- Object master = params.get(HMaster.MASTER);
- if (master != null && master instanceof HMaster) {
- zkw = ((HMaster) master).getZooKeeper();
- shareZK = true;
- }
- }
- if (zkw == null) {
- zkw = new ZKWatcher(getConf(), "replicationLogCleaner", null);
+ if (MapUtils.isNotEmpty(params)) {
+ Object master = params.get(HMaster.MASTER);
+ if (master != null && master instanceof MasterServices) {
+ MasterServices m = (MasterServices) master;
+ rpm = m.getReplicationPeerManager();
+ getNotFullyDeadServers = () -> getNotFullyDeadServers(m);
+ return;
}
- this.queueStorage = ReplicationStorageFactory.getReplicationQueueStorage(zkw, getConf());
- } catch (IOException e) {
- LOG.error("Error while configuring " + this.getClass().getName(), e);
}
- }
-
- @InterfaceAudience.Private
- public void setConf(Configuration conf, ZKWatcher zk) {
- super.setConf(conf);
- try {
- this.zkw = zk;
- this.queueStorage = ReplicationStorageFactory.getReplicationQueueStorage(zk, conf);
- } catch (Exception e) {
- LOG.error("Error while configuring " + this.getClass().getName(), e);
- }
- }
-
- @InterfaceAudience.Private
- public void setConf(Configuration conf, ZKWatcher zk,
- ReplicationQueueStorage replicationQueueStorage) {
- super.setConf(conf);
- this.zkw = zk;
- this.queueStorage = replicationQueueStorage;
+ throw new IllegalArgumentException("Missing " + HMaster.MASTER + " parameter");
}
@Override
public void stop(String why) {
- if (this.stopped) return;
this.stopped = true;
- if (!shareZK && this.zkw != null) {
- LOG.info("Stopping " + this.zkw);
- this.zkw.close();
- }
}
@Override
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleanerBarrier.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleanerBarrier.java
new file mode 100644
index 000000000000..d87565187280
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleanerBarrier.java
@@ -0,0 +1,85 @@
+/*
+ * 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.yetus.audience.InterfaceAudience;
+
+/**
+ * A barrier to guard the execution of {@link ReplicationLogCleaner}.
+ *
+ * The reason why we introduce this class is because there could be race between
+ * {@link org.apache.hadoop.hbase.master.replication.AddPeerProcedure} and
+ * {@link ReplicationLogCleaner}. See HBASE-27214 for more details.
+ */
+@InterfaceAudience.Private
+public class ReplicationLogCleanerBarrier {
+
+ private enum State {
+ // the cleaner is not running
+ NOT_RUNNING,
+ // the cleaner is running
+ RUNNING,
+ // the cleaner is disabled
+ DISABLED
+ }
+
+ private State state = State.NOT_RUNNING;
+
+ // we could have multiple AddPeerProcedure running at the same time, so here we need to do
+ // reference counting.
+ private int numberDisabled = 0;
+
+ public synchronized boolean start() {
+ if (state == State.NOT_RUNNING) {
+ state = State.RUNNING;
+ return true;
+ }
+ if (state == State.DISABLED) {
+ return false;
+ }
+ throw new IllegalStateException("Unexpected state " + state);
+ }
+
+ public synchronized void stop() {
+ if (state != State.RUNNING) {
+ throw new IllegalStateException("Unexpected state " + state);
+ }
+ state = State.NOT_RUNNING;
+ }
+
+ public synchronized boolean disable() {
+ if (state == State.RUNNING) {
+ return false;
+ }
+ if (state == State.NOT_RUNNING) {
+ state = State.DISABLED;
+ }
+ numberDisabled++;
+ return true;
+ }
+
+ public synchronized void enable() {
+ if (state != State.DISABLED) {
+ throw new IllegalStateException("Unexpected state " + state);
+ }
+ numberDisabled--;
+ if (numberDisabled == 0) {
+ state = State.NOT_RUNNING;
+ }
+ }
+}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ClaimReplicationQueueCallable.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ClaimReplicationQueueCallable.java
index b9a7be813af8..2b7e14f9f7aa 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ClaimReplicationQueueCallable.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ClaimReplicationQueueCallable.java
@@ -20,6 +20,7 @@
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.executor.EventType;
import org.apache.hadoop.hbase.procedure2.BaseRSProcedureCallable;
+import org.apache.hadoop.hbase.replication.ReplicationQueueId;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hbase.thirdparty.com.google.protobuf.InvalidProtocolBufferException;
@@ -30,9 +31,7 @@
@InterfaceAudience.Private
public class ClaimReplicationQueueCallable extends BaseRSProcedureCallable {
- private ServerName crashedServer;
-
- private String queue;
+ private ReplicationQueueId queueId;
@Override
public EventType getEventType() {
@@ -42,14 +41,20 @@ public EventType getEventType() {
@Override
protected void doCall() throws Exception {
PeerProcedureHandler handler = rs.getReplicationSourceService().getPeerProcedureHandler();
- handler.claimReplicationQueue(crashedServer, queue);
+ handler.claimReplicationQueue(queueId);
}
@Override
protected void initParameter(byte[] parameter) throws InvalidProtocolBufferException {
ClaimReplicationQueueRemoteParameter param =
ClaimReplicationQueueRemoteParameter.parseFrom(parameter);
- crashedServer = ProtobufUtil.toServerName(param.getCrashedServer());
- queue = param.getQueue();
+ ServerName crashedServer = ProtobufUtil.toServerName(param.getCrashedServer());
+ String queue = param.getQueue();
+ if (param.hasSourceServer()) {
+ ServerName sourceServer = ProtobufUtil.toServerName(param.getSourceServer());
+ queueId = new ReplicationQueueId(crashedServer, queue, sourceServer);
+ } else {
+ queueId = new ReplicationQueueId(crashedServer, queue);
+ }
}
}
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 4636e239904a..b284e3f6837f 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
@@ -19,9 +19,12 @@
import java.io.FileNotFoundException;
import java.io.IOException;
+import java.net.URLEncoder;
+import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
+import java.util.Comparator;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
@@ -32,7 +35,7 @@
import org.apache.hadoop.conf.Configured;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.client.Admin;
@@ -41,28 +44,33 @@
import org.apache.hadoop.hbase.client.replication.TableCFs;
import org.apache.hadoop.hbase.io.WALLink;
import org.apache.hadoop.hbase.procedure2.util.StringUtils;
+import org.apache.hadoop.hbase.replication.ReplicationException;
+import org.apache.hadoop.hbase.replication.ReplicationGroupOffset;
+import org.apache.hadoop.hbase.replication.ReplicationOffsetUtil;
import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
-import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
+import org.apache.hadoop.hbase.replication.ReplicationQueueData;
+import org.apache.hadoop.hbase.replication.ReplicationQueueId;
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.ZKDump;
-import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
+import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableMap;
import org.apache.hbase.thirdparty.com.google.common.util.concurrent.AtomicLongMap;
/**
+ *
* Provides information about the existing states of replication, replication peers and queues.
* Usage: hbase org.apache.hadoop.hbase.replication.regionserver.DumpReplicationQueues [args]
* Arguments: --distributed Polls each RS to dump information about the queue --hdfs Reports HDFS
- * usage by the replication queues (note: can be overestimated).
+ * usage by the replication queues (note: can be overestimated). In the new version, we
+ * reimplemented the DumpReplicationQueues tool to support obtaining information from replication
+ * table.
*/
@InterfaceAudience.Private
public class DumpReplicationQueues extends Configured implements Tool {
@@ -186,7 +194,7 @@ protected static void printUsage(final String className, final String message) {
System.err.println("General Options:");
System.err.println(" -h|--h|--help Show this help and exit.");
System.err.println(" --distributed Poll each RS and print its own replication queue. "
- + "Default only polls ZooKeeper");
+ + "Default only polls replication table.");
System.err.println(" --hdfs Use HDFS to calculate usage of WALs by replication."
+ " It could be overestimated if replicating to multiple peers."
+ " --distributed flag is also needed.");
@@ -202,13 +210,7 @@ private int dumpReplicationQueues(DumpOptions opts) throws Exception {
Connection connection = ConnectionFactory.createConnection(conf);
Admin admin = connection.getAdmin();
- ZKWatcher zkw =
- new ZKWatcher(conf, "DumpReplicationQueues" + EnvironmentEdgeManager.currentTime(),
- new WarnOnlyAbortable(), true);
-
try {
- // Our zk watcher
- LOG.info("Our Quorum: " + zkw.getQuorum());
List replicatedTableCFs = admin.listReplicatedTableCFs();
if (replicatedTableCFs.isEmpty()) {
LOG.info("No tables with a configured replication peer were found.");
@@ -230,19 +232,70 @@ private int dumpReplicationQueues(DumpOptions opts) throws Exception {
LOG.info("Found [--distributed], will poll each RegionServer.");
Set peerIds =
peers.stream().map((peer) -> peer.getPeerId()).collect(Collectors.toSet());
- System.out.println(dumpQueues(zkw, peerIds, opts.isHdfs()));
+ System.out.println(dumpQueues(connection, peerIds, opts.isHdfs(), conf));
System.out.println(dumpReplicationSummary());
} else {
- // use ZK instead
- System.out.print("Dumping replication znodes via ZooKeeper:");
- System.out.println(ZKDump.getReplicationZnodesDump(zkw));
+ // use replication table instead
+ System.out.println("Dumping replication info via replication table.");
+ System.out.println(dumpReplicationViaTable(connection, conf));
}
return (0);
} catch (IOException e) {
return (-1);
} finally {
- zkw.close();
+ connection.close();
+ }
+ }
+
+ public String dumpReplicationViaTable(Connection connection, Configuration conf)
+ throws ReplicationException, IOException {
+ StringBuilder sb = new StringBuilder();
+ ReplicationQueueStorage queueStorage =
+ ReplicationStorageFactory.getReplicationQueueStorage(connection, conf);
+
+ // The dump info format is as follows:
+ // peers:
+ // peers/1: zk1:2181:/hbase
+ // peers/1/peer-state: ENABLED
+ // rs:
+ // rs/rs1,16020,1664092120094/1/rs1%2C16020%2C1664092120094.1664096778778: 123
+ // rs/rs2,16020,1664092120094/2/rs1%2C16020%2C1664092120094.1664096778778: 321
+ // hfile-refs:
+ // hfile-refs/1/hfile1,hfile2
+ // hfile-refs/2/hfile3,hfile4
+ String peersKey = "peers";
+ sb.append(peersKey).append(": ").append("\n");
+ List repPeerDescs = connection.getAdmin().listReplicationPeers();
+ for (ReplicationPeerDescription repPeerDesc : repPeerDescs) {
+ sb.append(peersKey).append("/").append(repPeerDesc.getPeerId()).append(": ")
+ .append(repPeerDesc.getPeerConfig().getClusterKey()).append("\n");
+ sb.append(peersKey).append("/").append(repPeerDesc.getPeerId()).append("/peer-state: ")
+ .append(repPeerDesc.isEnabled() ? "ENABLED" : "DISABLED").append("\n");
+ }
+
+ List repQueueDataList = queueStorage.listAllQueues();
+ String rsKey = "rs";
+ sb.append(rsKey).append(": ").append("\n");
+ for (ReplicationQueueData repQueueData : repQueueDataList) {
+ String peerId = repQueueData.getId().getPeerId();
+ for (ImmutableMap.Entry entry : repQueueData.getOffsets()
+ .entrySet()) {
+ sb.append(rsKey).append("/").append(entry.getKey()).append("/").append(peerId).append("/")
+ .append(entry.getValue().getWal()).append(": ").append(entry.getValue().getOffset())
+ .append("\n");
+ }
+ }
+
+ List peerIds = queueStorage.getAllPeersFromHFileRefsQueue();
+ String hfileKey = "hfile-refs";
+ sb.append(hfileKey).append(": ").append("\n");
+ for (String peerId : peerIds) {
+ List hfiles = queueStorage.getReplicableHFiles(peerId);
+ sb.append(hfileKey).append("/").append(peerId).append("/").append(String.join(",", hfiles))
+ .append("\n");
}
+
+ return sb.toString();
}
public String dumpReplicationSummary() {
@@ -295,70 +348,103 @@ public String dumpPeersState(List peers) throws Exce
return sb.toString();
}
- public String dumpQueues(ZKWatcher zkw, Set peerIds, boolean hdfs) throws Exception {
- ReplicationQueueStorage queueStorage;
+ public String dumpQueues(Connection connection, Set peerIds, boolean hdfs,
+ Configuration conf) throws Exception {
StringBuilder sb = new StringBuilder();
+ ReplicationQueueStorage queueStorage =
+ ReplicationStorageFactory.getReplicationQueueStorage(connection, conf);
- queueStorage = ReplicationStorageFactory.getReplicationQueueStorage(zkw, getConf());
- Set liveRegionServers = ZKUtil.listChildrenNoWatch(zkw, zkw.getZNodePaths().rsZNode)
- .stream().map(ServerName::parseServerName).collect(Collectors.toSet());
+ Set liveRegionServers =
+ connection.getAdmin().getClusterMetrics().getLiveServerMetrics().keySet();
- // Loops each peer on each RS and dumps the queues
- List regionservers = queueStorage.getListOfReplicators();
- if (regionservers == null || regionservers.isEmpty()) {
+ List regionServers = queueStorage.listAllReplicators();
+ if (regionServers == null || regionServers.isEmpty()) {
return sb.toString();
}
- for (ServerName regionserver : regionservers) {
- List queueIds = queueStorage.getAllQueues(regionserver);
- if (!liveRegionServers.contains(regionserver)) {
- deadRegionServers.add(regionserver.getServerName());
+ for (ServerName regionServer : regionServers) {
+ List queueIds = queueStorage.listAllQueueIds(regionServer);
+
+ if (!liveRegionServers.contains(regionServer)) {
+ deadRegionServers.add(regionServer.getServerName());
}
- for (String queueId : queueIds) {
- ReplicationQueueInfo queueInfo = new ReplicationQueueInfo(queueId);
- List wals = queueStorage.getWALsInQueue(regionserver, queueId);
- Collections.sort(wals);
- if (!peerIds.contains(queueInfo.getPeerId())) {
- deletedQueues.add(regionserver + "/" + queueId);
- sb.append(formatQueue(regionserver, queueStorage, queueInfo, queueId, wals, true, hdfs));
+ for (ReplicationQueueId queueId : queueIds) {
+ List tmpWals = new ArrayList<>();
+ // wals
+ AbstractFSWALProvider
+ .getWALFiles(connection.getConfiguration(), queueId.getServerWALsBelongTo()).stream()
+ .map(Path::toString).forEach(tmpWals::add);
+
+ // old wals
+ AbstractFSWALProvider.getArchivedWALFiles(connection.getConfiguration(),
+ queueId.getServerWALsBelongTo(), URLEncoder
+ .encode(queueId.getServerWALsBelongTo().toString(), StandardCharsets.UTF_8.name()))
+ .stream().map(Path::toString).forEach(tmpWals::add);
+
+ Map offsets = queueStorage.getOffsets(queueId);
+ // filter out the wal files that should replicate
+ List wals = new ArrayList<>();
+ for (Map.Entry entry : offsets.entrySet()) {
+ ReplicationGroupOffset offset = entry.getValue();
+ for (String wal : tmpWals) {
+ if (ReplicationOffsetUtil.shouldReplicate(offset, wal)) {
+ wals.add(wal);
+ }
+ }
+ }
+ Collections.sort(wals, Comparator.comparingLong(AbstractFSWALProvider::getTimestamp));
+ if (!peerIds.contains(queueId.getPeerId())) {
+ deletedQueues.add(regionServer + "/" + queueId);
+ sb.append(formatQueue(regionServer, offsets, wals, queueId, true, hdfs));
} else {
- sb.append(formatQueue(regionserver, queueStorage, queueInfo, queueId, wals, false, hdfs));
+ sb.append(formatQueue(regionServer, offsets, wals, queueId, false, hdfs));
}
}
}
return sb.toString();
}
- private String formatQueue(ServerName regionserver, ReplicationQueueStorage queueStorage,
- ReplicationQueueInfo queueInfo, String queueId, List wals, boolean isDeleted,
- boolean hdfs) throws Exception {
+ private String formatQueue(ServerName regionServer, Map offsets,
+ List wals, ReplicationQueueId queueId, boolean isDeleted, boolean hdfs)
+ throws Exception {
StringBuilder sb = new StringBuilder();
- List deadServers;
-
- sb.append("Dumping replication queue info for RegionServer: [" + regionserver + "]" + "\n");
- sb.append(" Queue znode: " + queueId + "\n");
- sb.append(" PeerID: " + queueInfo.getPeerId() + "\n");
- sb.append(" Recovered: " + queueInfo.isQueueRecovered() + "\n");
- deadServers = queueInfo.getDeadRegionServers();
- if (deadServers.isEmpty()) {
- sb.append(" No dead RegionServers found in this queue." + "\n");
+ sb.append("Dumping replication queue info for RegionServer: [" + regionServer + "]" + "\n");
+ sb.append(" Queue id: " + queueId + "\n");
+ sb.append(" PeerID: " + queueId.getPeerId() + "\n");
+ sb.append(" Recovered: " + queueId.isRecovered() + "\n");
+ // In new version, we only record the first dead RegionServer in queueId.
+ if (queueId.getSourceServerName().isPresent()) {
+ sb.append(" Dead RegionServer: " + queueId.getSourceServerName().get() + "\n");
} else {
- sb.append(" Dead RegionServers: " + deadServers + "\n");
+ sb.append(" No dead RegionServer found in this queue." + "\n");
}
sb.append(" Was deleted: " + isDeleted + "\n");
sb.append(" Number of WALs in replication queue: " + wals.size() + "\n");
- peersQueueSize.addAndGet(queueInfo.getPeerId(), wals.size());
-
- for (String wal : wals) {
- 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");
+ peersQueueSize.addAndGet(queueId.getPeerId(), wals.size());
+
+ for (Map.Entry entry : offsets.entrySet()) {
+ String walGroup = entry.getKey();
+ ReplicationGroupOffset offset = entry.getValue();
+ for (String wal : wals) {
+ long position = 0;
+ if (offset.getWal().equals(wal)) {
+ position = offset.getOffset();
+ }
+ sb.append(
+ " Replication position for " + (walGroup != null ? walGroup + "/" + wal : wal) + ": ");
+ if (position == 0) {
+ sb.append("0 (not started or nothing to replicate)");
+ } else if (position > 0) {
+ sb.append(position);
+ }
+ sb.append("\n");
+ }
}
if (hdfs) {
FileSystem fs = FileSystem.get(getConf());
sb.append(" Total size of WALs on HDFS for this queue: "
- + StringUtils.humanSize(getTotalWALSize(fs, wals, regionserver)) + "\n");
+ + StringUtils.humanSize(getTotalWALSize(fs, wals, regionServer)) + "\n");
}
return sb.toString();
}
@@ -366,8 +452,7 @@ private String formatQueue(ServerName regionserver, ReplicationQueueStorage queu
/**
* return total size in bytes from a list of WALs
*/
- private long getTotalWALSize(FileSystem fs, List wals, ServerName server)
- throws IOException {
+ private long getTotalWALSize(FileSystem fs, List wals, ServerName server) {
long size = 0;
FileStatus fileStatus;
@@ -389,19 +474,4 @@ private long getTotalWALSize(FileSystem fs, List wals, ServerName server
totalSizeOfWALs += size;
return size;
}
-
- private static class WarnOnlyAbortable implements Abortable {
- @Override
- public void abort(String why, Throwable e) {
- LOG.warn("DumpReplicationQueue received abort, ignoring. Reason: " + why);
- if (LOG.isDebugEnabled()) {
- LOG.debug(e.toString(), e);
- }
- }
-
- @Override
- public boolean isAborted() {
- return false;
- }
- }
}
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
index 3df78c1d8313..b2cffd59fd4a 100644
--- 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
@@ -18,9 +18,9 @@
package org.apache.hadoop.hbase.replication.regionserver;
import java.io.IOException;
-import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.replication.ReplicationException;
+import org.apache.hadoop.hbase.replication.ReplicationQueueId;
import org.apache.yetus.audience.InterfaceAudience;
/**
@@ -42,6 +42,5 @@ public interface PeerProcedureHandler {
void transitSyncReplicationPeerState(String peerId, int stage, HRegionServer rs)
throws ReplicationException, IOException;
- void claimReplicationQueue(ServerName crashedServer, String queue)
- throws ReplicationException, IOException;
+ void claimReplicationQueue(ReplicationQueueId queueId) 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
index 0187de14f806..cd3db44d8fa3 100644
--- 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
@@ -20,7 +20,6 @@
import java.io.IOException;
import java.io.InterruptedIOException;
import java.util.concurrent.locks.Lock;
-import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.LogRoller;
import org.apache.hadoop.hbase.replication.ReplicationException;
@@ -28,6 +27,7 @@
import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
import org.apache.hadoop.hbase.replication.ReplicationPeerImpl;
import org.apache.hadoop.hbase.replication.ReplicationPeers;
+import org.apache.hadoop.hbase.replication.ReplicationQueueId;
import org.apache.hadoop.hbase.replication.ReplicationUtils;
import org.apache.hadoop.hbase.replication.SyncReplicationState;
import org.apache.hadoop.hbase.util.KeyLocker;
@@ -226,8 +226,8 @@ public void transitSyncReplicationPeerState(String peerId, int stage, HRegionSer
}
@Override
- public void claimReplicationQueue(ServerName crashedServer, String queue)
+ public void claimReplicationQueue(ReplicationQueueId queueId)
throws ReplicationException, IOException {
- replicationSourceManager.claimQueue(crashedServer, queue);
+ replicationSourceManager.claimQueue(queueId);
}
}
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 024248a3f8c9..e740a01dc4f7 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
@@ -17,23 +17,7 @@
*/
package org.apache.hadoop.hbase.replication.regionserver;
-import java.io.IOException;
-import java.util.List;
-import java.util.UUID;
-import java.util.concurrent.PriorityBlockingQueue;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileStatus;
-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.hadoop.hbase.replication.ReplicationPeer;
-import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
-import org.apache.hadoop.hbase.util.CommonFSUtils;
-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 another
@@ -42,124 +26,14 @@
@InterfaceAudience.Private
public class RecoveredReplicationSource extends ReplicationSource {
- private static final Logger LOG = LoggerFactory.getLogger(RecoveredReplicationSource.class);
-
- private String actualPeerId;
-
- @Override
- public void init(Configuration conf, FileSystem fs, ReplicationSourceManager manager,
- 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();
- }
-
@Override
protected RecoveredReplicationSourceShipper createNewShipper(String walGroupId) {
- return new RecoveredReplicationSourceShipper(conf, walGroupId, logQueue, this, queueStorage);
- }
-
- public void locateRecoveredPaths(String walGroupId) throws IOException {
- boolean hasPathChanged = false;
- PriorityBlockingQueue queue = logQueue.getQueue(walGroupId);
- PriorityBlockingQueue newPaths = new PriorityBlockingQueue