HBASE-20116 Optimize the region last pushed sequence id layout on zk

This commit is contained in:
huzheng 2018-03-20 10:13:15 +08:00
parent 17ac2fe9c1
commit 8ab7b20f48
3 changed files with 28 additions and 7 deletions

View File

@ -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.CollectionUtils;
@ -139,19 +140,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

View File

@ -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));
}

View File

@ -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);
}
}