You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by zh...@apache.org on 2018/04/09 07:22:21 UTC
[14/20] hbase git commit: HBASE-20116 Optimize the region last pushed
sequence id layout on zk
HBASE-20116 Optimize the region last pushed sequence id layout on zk
Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/2b9fed86
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/2b9fed86
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/2b9fed86
Branch: refs/heads/branch-2
Commit: 2b9fed863f19510007f4bce08082ee42fb867eac
Parents: 16a4dd6
Author: huzheng <op...@gmail.com>
Authored: Tue Mar 20 10:13:15 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Apr 9 15:18:44 2018 +0800
----------------------------------------------------------------------
.../replication/ZKReplicationQueueStorage.java | 22 ++++++++++++++------
.../replication/TestReplicationStateBasic.java | 3 ++-
.../TestZKReplicationQueueStorage.java | 10 +++++++++
3 files changed, 28 insertions(+), 7 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hbase/blob/2b9fed86/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java
----------------------------------------------------------------------
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
index 63f43e8..6c9752a 100644
--- 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
@@ -33,6 +33,7 @@ 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;
@@ -138,19 +139,28 @@ class ZKReplicationQueueStorage extends ZKReplicationStorageBase
* So the final znode path will be format like this:
*
* <pre>
- * /hbase/replication/regions/254/dd04e76a6966d4ffa908ed0586764767-100
+ * /hbase/replication/regions/e1/ff/dd04e76a6966d4ffa908ed0586764767-100
* </pre>
*
- * The 254 indicate the hash of encoded region name, the 100 indicate the peer id.
+ * The e1 indicate the first level hash of encoded region name, and the ff indicate the second
+ * level hash of encoded region name, the 100 indicate the peer id. <br>
+ * Note that here we use two-level hash because if only one-level hash (such as mod 65535), it
+ * will still lead to too many children under the /hbase/replication/regions znode.
* @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.
*/
- private String getSerialReplicationRegionPeerNode(String encodedRegionName, String peerId) {
- int hash = encodedRegionName.hashCode() & 0x0000FFFF;
- String hashPath = ZNodePaths.joinZNode(regionsZNode, String.valueOf(hash));
- return ZNodePaths.joinZNode(hashPath, String.format("%s-%s", encodedRegionName, peerId));
+ @VisibleForTesting
+ public 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.substring(0, 2)).append(ZNodePaths.ZNODE_PATH_SEPARATOR)
+ .append(encodedRegionName.substring(2, 4)).append(ZNodePaths.ZNODE_PATH_SEPARATOR)
+ .append(encodedRegionName).append("-").append(peerId).toString();
}
@Override
http://git-wip-us.apache.org/repos/asf/hbase/blob/2b9fed86/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
----------------------------------------------------------------------
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 21b09aa..3ed4121 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
@@ -285,7 +285,8 @@ public abstract class TestReplicationStateBasic {
ServerName serverName1 = ServerName.valueOf("127.0.0.1", 8000, 10000);
assertTrue(rqs.getAllQueues(serverName1).isEmpty());
String queue1 = "1";
- String region0 = "region0", region1 = "region1";
+ String region0 = "6b2c8f8555335cc9af74455b94516cbe",
+ region1 = "6ecd2e9e010499f8ddef97ee8f70834f";
for (int i = 0; i < 10; i++) {
rqs.addWAL(serverName1, queue1, getFileName("file1", i));
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/2b9fed86/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationQueueStorage.java
----------------------------------------------------------------------
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 c813870..28cdff1 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
@@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.util.Pair;
import org.apache.zookeeper.KeeperException;
import org.junit.After;
import org.junit.AfterClass;
+import org.junit.Assert;
import org.junit.BeforeClass;
import org.junit.ClassRule;
import org.junit.Test;
@@ -251,4 +252,13 @@ public class TestZKReplicationQueueStorage {
assertEquals(1, allHFileRefs.size());
assertThat(allHFileRefs, hasItems("test"));
}
+
+ @Test
+ public void testRegionsZNodeLayout() throws Exception {
+ String peerId = "1";
+ String encodedRegionName = "31d9792f4435b99d9fb1016f6fbc8dc7";
+ String expectedPath = "/hbase/replication/regions/31/d9/" + encodedRegionName + "-" + peerId;
+ String path = STORAGE.getSerialReplicationRegionPeerNode(encodedRegionName, peerId);
+ Assert.assertEquals(expectedPath, path);
+ }
}