From e57b806287f9e9af6b191c4585b0bc102a29f181 Mon Sep 17 00:00:00 2001 From: Bharath Vissapragada Date: Wed, 13 Nov 2019 12:54:36 -0800 Subject: [PATCH 01/10] HBASE-23281: Track meta region locations in masters This patch adds a simple cache that tracks the meta region replica locations. It keeps an eye on the region movements so that the cached locations are not stale. This information is used for servicing client RPCs for connections that use master based registry (HBASE-18095). The RPC end points will be added in a separate patch. --- .../hbase/shaded/protobuf/ProtobufUtil.java | 36 ++- .../hadoop/hbase/zookeeper/ZNodePaths.java | 14 +- .../apache/hadoop/hbase/master/HMaster.java | 14 +- .../hbase/master/MetaRegionLocationCache.java | 217 ++++++++++++++++++ .../client/TestMetaRegionLocationCache.java | 125 ++++++++++ .../hbase/zookeeper/MetaTableLocator.java | 22 +- 6 files changed, 404 insertions(+), 24 deletions(-) create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java index 94a2805b61cc..956ca66c95b3 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -93,6 +93,7 @@ import org.apache.hadoop.hbase.filter.ByteArrayComparable; import org.apache.hadoop.hbase.filter.Filter; import org.apache.hadoop.hbase.io.TimeRange; +import org.apache.hadoop.hbase.master.RegionState; import org.apache.hadoop.hbase.protobuf.ProtobufMagic; import org.apache.hadoop.hbase.protobuf.ProtobufMessageConverter; import org.apache.hadoop.hbase.quotas.QuotaScope; @@ -3051,6 +3052,39 @@ public static ProcedureDescription buildProcedureDescription(String signature, S return builder.build(); } + /** + * Get the Meta region servername from the passed data bytes. Can handle both old and new style + * server names. + * @param data protobuf serialized data with meta server name. + * @return Servername instance correpsonding to the serialized data. + * @throws DeserializationException if the data is invalid. + */ + public static ServerName parseMetaServerNameFrom(final byte[] data) + throws DeserializationException { + RegionState.State state = RegionState.State.OPEN; + ServerName serverName; + if (data != null && data.length > 0 && ProtobufUtil.isPBMagicPrefix(data)) { + try { + int prefixLen = ProtobufUtil.lengthOfPBMagic(); + ZooKeeperProtos.MetaRegionServer rl = + ZooKeeperProtos.MetaRegionServer.parser().parseFrom(data, prefixLen, + data.length - prefixLen); + if (rl.hasState()) { + state = RegionState.State.convert(rl.getState()); + } + HBaseProtos.ServerName sn = rl.getServer(); + serverName = ServerName.valueOf( + sn.getHostName(), sn.getPort(), sn.getStartCode()); + } catch (InvalidProtocolBufferException e) { + throw new DeserializationException("Unable to parse meta region location"); + } + } else { + // old style of meta region location? + serverName = parseServerNameFrom(data); + } + return serverName; + } + /** * Get a ServerName from the passed in data bytes. * @param data Data with a serialize server name in it; can handle the old style 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 c5e510fe4b9c..a1cc47fcc2b9 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 @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -182,6 +182,18 @@ public String getZNodeForReplica(int replicaId) { .orElseGet(() -> metaReplicaZNodes.get(DEFAULT_REPLICA_ID) + "-" + replicaId); } + /** + * Parses the meta replicaId from the passed path. + * @param path the name of the full path which includes baseZNode. + * @return replicaId + */ + public int getMetaReplicaIdFromPath(String path) { + // Extract the znode from path. The prefix is of the following format. + // baseZNode + PATH_SEPARATOR. + int prefixLen = baseZNode.length() + 1; + return getMetaReplicaIdFromZnode(path.substring(prefixLen)); + } + /** * Parse the meta replicaId from the passed znode * @param znode the name of the znode, does not include baseZNode 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 b444d3bad1a2..71c951174696 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -355,6 +355,12 @@ public void run() { // manager of assignment nodes in zookeeper private AssignmentManager assignmentManager; + /** + * Cache for the meta region replica's locations. Also tracks their changes to avoid stale + * cache entries. + */ + private final MetaRegionLocationCache metaRegionLocationCache; + // manager of replication private ReplicationPeerManager replicationPeerManager; @@ -522,7 +528,7 @@ public HMaster(final Configuration conf) } else { maintenanceMode = false; } - + metaRegionLocationCache = new MetaRegionLocationCache(this.zooKeeper); this.rsFatals = new MemoryBoundedLogMessageBuffer( conf.getLong("hbase.master.buffer.for.rs.fatals", 1 * 1024 * 1024)); LOG.info("hbase.rootdir={}, hbase.cluster.distributed={}", getDataRootDir(), @@ -3880,4 +3886,8 @@ public void runReplicationBarrierCleaner() { rbc.chore(); } } + + public MetaRegionLocationCache getMetaRegionLocationCache() { + return this.metaRegionLocationCache; + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java new file mode 100644 index 000000000000..c7a3d58415cc --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java @@ -0,0 +1,217 @@ +/* + * 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; + +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; +import java.util.concurrent.ConcurrentNavigableMap; +import org.apache.hadoop.hbase.HRegionLocation; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.client.RegionInfoBuilder; +import org.apache.hadoop.hbase.client.RegionReplicaUtil; +import org.apache.hadoop.hbase.exceptions.DeserializationException; +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.types.CopyOnWriteArrayMap; +import org.apache.hadoop.hbase.util.RetryCounter; +import org.apache.hadoop.hbase.util.RetryCounterFactory; +import org.apache.hadoop.hbase.zookeeper.ZKListener; +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.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A cache of meta region location metadata. Registers a listener on ZK to track changes to the + * meta table znodes. Clients are expected to retry if the meta information is stale. This class + * is thread-safe. + */ +@InterfaceAudience.Private +public class MetaRegionLocationCache extends ZKListener { + + private static final Logger LOG = LoggerFactory.getLogger(MetaRegionLocationCache.class); + + // Maximum number of times we retry when ZK operation times out. + private static final int MAX_ZK_META_FETCH_RETRIES = 10; + // Sleep interval ms between ZK operation retries. + private static final int SLEEP_INTERVAL_MS_BETWEEN_RETRIES = 1000; + private final RetryCounterFactory retryCounterFactory = + new RetryCounterFactory(MAX_ZK_META_FETCH_RETRIES, SLEEP_INTERVAL_MS_BETWEEN_RETRIES); + + private final ZKWatcher watcher; + // Cached meta region locations indexed by replica ID. + // CopyOnWriteArrayMap ensures synchronization during updates and a consistent snapshot during + // client requests. Even though CopyOnWriteArrayMap copies the data structure for every write, + // that should be OK since the size of the list is often small and mutations are not too often + // and we do not need to block client requests while mutations are in progress. + private final CopyOnWriteArrayMap cachedMetaLocations; + + private enum ZNodeOpType { + INIT, + CREATED, + CHANGED, + DELETED + }; + + MetaRegionLocationCache(ZKWatcher zkWatcher) { + super(zkWatcher); + watcher = zkWatcher; + cachedMetaLocations = new CopyOnWriteArrayMap<>(); + watcher.registerListener(this); + // Populate the initial snapshot of data from meta znodes. + // This is needed because stand-by masters can potentially start after the initial znode + // creation. + populateInitialMetaLocations(); + } + + private void populateInitialMetaLocations() { + RetryCounter retryCounter = retryCounterFactory.create(); + List znodes = null; + do { + try { + znodes = watcher.getMetaReplicaNodes(); + break; + } catch (KeeperException ke) { + LOG.debug("Error populating intial meta locations", ke); + try { + retryCounter.sleepUntilNextRetry(); + } catch (InterruptedException ie) { + LOG.error("Interrupted while populating intial meta locations", ie); + return; + } + if (!retryCounter.shouldRetry()) { + LOG.error("Error populating intial meta locations. Retries exhausted. Last error: ", ke); + break; + } + } + } while (retryCounter.shouldRetry()); + if (znodes == null) { + return; + } + for (String znode: znodes) { + String path = ZNodePaths.joinZNode(watcher.getZNodePaths().baseZNode, znode); + updateMetaLocation(path, ZNodeOpType.INIT); + } + } + + /** + * Gets the HRegionLocation for a given meta replica ID. Renews the watch on the znode for + * future updates. + * @param replicaId ReplicaID of the region. + * @return HRegionLocation for the meta replica. + * @throws KeeperException if there is any issue fetching/parsing the serialized data. + */ + private HRegionLocation getMetaRegionLocation(int replicaId) + throws KeeperException { + ServerName serverName = null; + try { + byte[] data = ZKUtil.getDataAndWatch(watcher, + watcher.getZNodePaths().getZNodeForReplica(replicaId)); + serverName = ProtobufUtil.parseMetaServerNameFrom(data); + } catch (DeserializationException e) { + throw ZKUtil.convert(e); + } + return new HRegionLocation(RegionReplicaUtil.getRegionInfoForReplica( + RegionInfoBuilder.FIRST_META_REGIONINFO, replicaId), serverName); + } + + private void updateMetaLocation(String path, ZNodeOpType opType) { + if (!isValidMetaZNode(path)) { + return; + } + LOG.debug("Updating meta znode for path {}: {}", path, opType.name()); + int replicaId = watcher.getZNodePaths().getMetaReplicaIdFromPath(path); + RetryCounter retryCounter = retryCounterFactory.create(); + HRegionLocation location = null; + do { + try { + if (opType == ZNodeOpType.DELETED) { + if (!ZKUtil.watchAndCheckExists(watcher, path)) { + // The path does not exist, we've set the watcher and we can break for now. + break; + } + // If it is a transient error and the node appears right away, we fetch the + // latest meta state. + } + location = getMetaRegionLocation(replicaId); + break; + } catch (KeeperException e) { + LOG.debug("Error getting meta location for path {}", path, e); + if (retryCounter.shouldRetry()) { + try { + retryCounter.sleepUntilNextRetry(); + } catch (InterruptedException ie) { + LOG.error("Interrupted while updating meta location for path {}", path, ie); + return; + } + } else { + LOG.error("Error getting meta location for path {}. Retries exhausted.", path, e); + } + } + } while (retryCounter.shouldRetry()); + if (location == null) { + cachedMetaLocations.remove(replicaId); + return; + } + cachedMetaLocations.put(replicaId, location); + } + + /** + * @return List of HRegionLocations for meta replica(s), null if the cache is empty. + */ + public Optional> getCachedMetaRegionLocations() { + ConcurrentNavigableMap snapshot = + cachedMetaLocations.tailMap(cachedMetaLocations.firstKey()); + if (snapshot == null || snapshot.isEmpty()) { + // This could be possible if the master has not successfully initialized yet or meta region + // is stuck in some weird state. + return Optional.empty(); + } + List result = new ArrayList<>(); + // Explicitly iterate instead of new ArrayList<>(snapshot.values()) because the underlying + // ArrayValueCollection does not implement toArray(). + snapshot.values().forEach(location -> result.add(location)); + return Optional.of(result); + } + + /** + * Helper to check if the given 'path' corresponds to a meta znode. This listener is only + * interested in changes to meta znodes. + */ + private boolean isValidMetaZNode(String path) { + return watcher.getZNodePaths().isAnyMetaReplicaZNode(path); + } + + @Override + public void nodeCreated(String path) { + updateMetaLocation(path, ZNodeOpType.CREATED); + } + + @Override + public void nodeDeleted(String path) { + updateMetaLocation(path, ZNodeOpType.DELETED); + } + + @Override + public void nodeDataChanged(String path) { + updateMetaLocation(path, ZNodeOpType.CHANGED); + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java new file mode 100644 index 000000000000..dce9fab4d5bb --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java @@ -0,0 +1,125 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.client; + +import static junit.framework.TestCase.assertTrue; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HRegionLocation; +import org.apache.hadoop.hbase.master.HMaster; +import org.apache.hadoop.hbase.master.RegionState; +import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer; +import org.apache.hadoop.hbase.testclassification.MasterTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.util.JVMClusterUtil; +import org.apache.hadoop.hbase.zookeeper.MetaTableLocator; +import org.apache.hadoop.hbase.zookeeper.ZKWatcher; +import org.apache.hadoop.hbase.zookeeper.ZNodePaths; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category({SmallTests.class, MasterTests.class }) +public class TestMetaRegionLocationCache { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestAsyncMetaRegionLocator.class); + + private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); + private static AsyncRegistry REGISTRY; + + @BeforeClass + public static void setUp() throws Exception { + TEST_UTIL.getConfiguration().set(BaseLoadBalancer.TABLES_ON_MASTER, "none"); + TEST_UTIL.getConfiguration().setInt(HConstants.META_REPLICAS_NUM, 3); + TEST_UTIL.startMiniCluster(3); + REGISTRY = AsyncRegistryFactory.getRegistry(TEST_UTIL.getConfiguration()); + RegionReplicaTestHelper.waitUntilAllMetaReplicasHavingRegionLocation( + TEST_UTIL.getConfiguration(), REGISTRY, 3); + TEST_UTIL.getAdmin().balancerSwitch(false, true); + } + + @AfterClass + public static void cleanUp() throws Exception { + IOUtils.closeQuietly(REGISTRY); + TEST_UTIL.shutdownMiniCluster(); + } + + private List getCurrentMetaLocations(ZKWatcher zk) throws Exception { + List result = new ArrayList<>(); + for (String znode: zk.getMetaReplicaNodes()) { + String path = ZNodePaths.joinZNode(zk.getZNodePaths().baseZNode, znode); + int replicaId = zk.getZNodePaths().getMetaReplicaIdFromPath(path); + RegionState state = MetaTableLocator.getMetaRegionState(zk, replicaId); + result.add(new HRegionLocation(state.getRegion(), state.getServerName())); + } + return result; + } + + // Verifies that the cached meta locations in the given master are in sync with what is in ZK. + private void verifyCachedMetaLocations(HMaster master) throws Exception { + List metaHRLs = + master.getMetaRegionLocationCache().getCachedMetaRegionLocations().get(); + assertTrue(metaHRLs != null); + assertFalse(metaHRLs.isEmpty()); + ZKWatcher zk = master.getZooKeeper(); + List metaZnodes = zk.getMetaReplicaNodes(); + assertEquals(metaZnodes.size(), metaHRLs.size()); + List actualHRLs = getCurrentMetaLocations(zk); + Collections.sort(metaHRLs); + Collections.sort(actualHRLs); + assertEquals(actualHRLs, metaHRLs); + } + + @Test public void testInitialMetaLocations() throws Exception { + verifyCachedMetaLocations(TEST_UTIL.getMiniHBaseCluster().getMaster()); + } + + @Test public void testStandByMetaLocations() throws Exception { + HMaster standBy = TEST_UTIL.getMiniHBaseCluster().startMaster().getMaster(); + verifyCachedMetaLocations(standBy); + } + + /* + * Shuffles the meta region replicas around the cluster and makes sure the cache is not stale. + */ + @Test public void testMetaLocationsChange() throws Exception { + List currentMetaLocs = + getCurrentMetaLocations(TEST_UTIL.getMiniHBaseCluster().getMaster().getZooKeeper()); + // Move these replicas to random servers. + for (HRegionLocation location: currentMetaLocs) { + RegionReplicaTestHelper.moveRegion(TEST_UTIL, location); + } + RegionReplicaTestHelper.waitUntilAllMetaReplicasHavingRegionLocation( + TEST_UTIL.getConfiguration(), REGISTRY, 3); + for (JVMClusterUtil.MasterThread masterThread: + TEST_UTIL.getMiniHBaseCluster().getMasterThreads()) { + verifyCachedMetaLocations(masterThread.getMaster()); + } + } +} \ No newline at end of file diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java index 0cebc762fd73..c6dd89f97c3b 100644 --- a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java +++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java @@ -274,30 +274,12 @@ public static RegionState getMetaRegionState(ZKWatcher zkw) throws KeeperExcepti * @throws KeeperException if a ZooKeeper operation fails */ public static RegionState getMetaRegionState(ZKWatcher zkw, int replicaId) - throws KeeperException { + throws KeeperException { RegionState.State state = RegionState.State.OPEN; ServerName serverName = null; try { byte[] data = ZKUtil.getData(zkw, zkw.getZNodePaths().getZNodeForReplica(replicaId)); - if (data != null && data.length > 0 && ProtobufUtil.isPBMagicPrefix(data)) { - try { - int prefixLen = ProtobufUtil.lengthOfPBMagic(); - ZooKeeperProtos.MetaRegionServer rl = - ZooKeeperProtos.MetaRegionServer.parser().parseFrom(data, prefixLen, - data.length - prefixLen); - if (rl.hasState()) { - state = RegionState.State.convert(rl.getState()); - } - HBaseProtos.ServerName sn = rl.getServer(); - serverName = ServerName.valueOf( - sn.getHostName(), sn.getPort(), sn.getStartCode()); - } catch (InvalidProtocolBufferException e) { - throw new DeserializationException("Unable to parse meta region location"); - } - } else { - // old style of meta region location? - serverName = ProtobufUtil.parseServerNameFrom(data); - } + serverName = ProtobufUtil.parseMetaServerNameFrom(data); } catch (DeserializationException e) { throw ZKUtil.convert(e); } catch (InterruptedException e) { From ad53427575d4a6bc287f0ca6b1ad4504a023f581 Mon Sep 17 00:00:00 2001 From: Bharath Vissapragada Date: Thu, 14 Nov 2019 15:41:03 -0800 Subject: [PATCH 02/10] Fix checkstyle and findbugs issues and review comments. --- .../hbase/shaded/protobuf/ProtobufUtil.java | 14 ++++++++++---- .../hbase/master/MetaRegionLocationCache.java | 13 +++++-------- .../client/TestMetaRegionLocationCache.java | 5 ++--- .../hbase/zookeeper/MetaTableLocator.java | 18 +++--------------- 4 files changed, 20 insertions(+), 30 deletions(-) diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java index 956ca66c95b3..bc5ae99612c9 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java @@ -80,6 +80,7 @@ import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.client.RegionLoadStats; +import org.apache.hadoop.hbase.client.RegionReplicaUtil; import org.apache.hadoop.hbase.client.RegionStatesCount; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Scan; @@ -3053,13 +3054,14 @@ public static ProcedureDescription buildProcedureDescription(String signature, S } /** - * Get the Meta region servername from the passed data bytes. Can handle both old and new style + * Get the Meta region state from the passed data bytes. Can handle both old and new style * server names. * @param data protobuf serialized data with meta server name. - * @return Servername instance correpsonding to the serialized data. + * @param replicaId replica ID for this region + * @return RegionState instance corresponding to the serialized data. * @throws DeserializationException if the data is invalid. */ - public static ServerName parseMetaServerNameFrom(final byte[] data) + public static RegionState parseMetaRegionStateFrom(final byte[] data, int replicaId) throws DeserializationException { RegionState.State state = RegionState.State.OPEN; ServerName serverName; @@ -3082,7 +3084,11 @@ public static ServerName parseMetaServerNameFrom(final byte[] data) // old style of meta region location? serverName = parseServerNameFrom(data); } - return serverName; + if (serverName == null) { + state = RegionState.State.OFFLINE; + } + return new RegionState(RegionReplicaUtil.getRegionInfoForReplica( + RegionInfoBuilder.FIRST_META_REGIONINFO, replicaId), state, serverName); } /** diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java index c7a3d58415cc..3ee25482d3d6 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java @@ -26,7 +26,6 @@ import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.client.RegionReplicaUtil; import org.apache.hadoop.hbase.exceptions.DeserializationException; -import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.types.CopyOnWriteArrayMap; import org.apache.hadoop.hbase.util.RetryCounter; import org.apache.hadoop.hbase.util.RetryCounterFactory; @@ -38,6 +37,7 @@ import org.apache.zookeeper.KeeperException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; /** * A cache of meta region location metadata. Registers a listener on ZK to track changes to the @@ -56,7 +56,6 @@ public class MetaRegionLocationCache extends ZKListener { private final RetryCounterFactory retryCounterFactory = new RetryCounterFactory(MAX_ZK_META_FETCH_RETRIES, SLEEP_INTERVAL_MS_BETWEEN_RETRIES); - private final ZKWatcher watcher; // Cached meta region locations indexed by replica ID. // CopyOnWriteArrayMap ensures synchronization during updates and a consistent snapshot during // client requests. Even though CopyOnWriteArrayMap copies the data structure for every write, @@ -73,7 +72,6 @@ private enum ZNodeOpType { MetaRegionLocationCache(ZKWatcher zkWatcher) { super(zkWatcher); - watcher = zkWatcher; cachedMetaLocations = new CopyOnWriteArrayMap<>(); watcher.registerListener(this); // Populate the initial snapshot of data from meta znodes. @@ -121,16 +119,15 @@ private void populateInitialMetaLocations() { */ private HRegionLocation getMetaRegionLocation(int replicaId) throws KeeperException { - ServerName serverName = null; + RegionState metaRegionState; try { byte[] data = ZKUtil.getDataAndWatch(watcher, watcher.getZNodePaths().getZNodeForReplica(replicaId)); - serverName = ProtobufUtil.parseMetaServerNameFrom(data); + metaRegionState = ProtobufUtil.parseMetaRegionStateFrom(data, replicaId); } catch (DeserializationException e) { throw ZKUtil.convert(e); } - return new HRegionLocation(RegionReplicaUtil.getRegionInfoForReplica( - RegionInfoBuilder.FIRST_META_REGIONINFO, replicaId), serverName); + return new HRegionLocation(metaRegionState.getRegion(), metaRegionState.getServerName()); } private void updateMetaLocation(String path, ZNodeOpType opType) { @@ -177,7 +174,7 @@ private void updateMetaLocation(String path, ZNodeOpType opType) { /** * @return List of HRegionLocations for meta replica(s), null if the cache is empty. */ - public Optional> getCachedMetaRegionLocations() { + public Optional> getMetaRegionLocations() { ConcurrentNavigableMap snapshot = cachedMetaLocations.tailMap(cachedMetaLocations.firstKey()); if (snapshot == null || snapshot.isEmpty()) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java index dce9fab4d5bb..2f66148cf71d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java @@ -21,7 +21,6 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; import java.util.List; import org.apache.commons.io.IOUtils; @@ -48,7 +47,7 @@ public class TestMetaRegionLocationCache { @ClassRule public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestAsyncMetaRegionLocator.class); + HBaseClassTestRule.forClass(TestMetaRegionLocationCache.class); private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static AsyncRegistry REGISTRY; @@ -84,7 +83,7 @@ private List getCurrentMetaLocations(ZKWatcher zk) throws Excep // Verifies that the cached meta locations in the given master are in sync with what is in ZK. private void verifyCachedMetaLocations(HMaster master) throws Exception { List metaHRLs = - master.getMetaRegionLocationCache().getCachedMetaRegionLocations().get(); + master.getMetaRegionLocationCache().getMetaRegionLocations().get(); assertTrue(metaHRLs != null); assertFalse(metaHRLs.isEmpty()); ZKWatcher zk = master.getZooKeeper(); diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java index c6dd89f97c3b..d28d8f9df9af 100644 --- a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java +++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java @@ -34,12 +34,7 @@ import org.apache.zookeeper.KeeperException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; - -import org.apache.hbase.thirdparty.com.google.protobuf.InvalidProtocolBufferException; - import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.MetaRegionServer; /** @@ -275,23 +270,16 @@ public static RegionState getMetaRegionState(ZKWatcher zkw) throws KeeperExcepti */ public static RegionState getMetaRegionState(ZKWatcher zkw, int replicaId) throws KeeperException { - RegionState.State state = RegionState.State.OPEN; - ServerName serverName = null; + RegionState regionState = null; try { byte[] data = ZKUtil.getData(zkw, zkw.getZNodePaths().getZNodeForReplica(replicaId)); - serverName = ProtobufUtil.parseMetaServerNameFrom(data); + regionState = ProtobufUtil.parseMetaRegionStateFrom(data, replicaId); } catch (DeserializationException e) { throw ZKUtil.convert(e); } catch (InterruptedException e) { Thread.currentThread().interrupt(); } - if (serverName == null) { - state = RegionState.State.OFFLINE; - } - return new RegionState( - RegionReplicaUtil.getRegionInfoForReplica( - RegionInfoBuilder.FIRST_META_REGIONINFO, replicaId), - state, serverName); + return regionState; } /** From d318470732c2c167b130b49e144db51284ed4105 Mon Sep 17 00:00:00 2001 From: Bharath Vissapragada Date: Thu, 14 Nov 2019 22:27:38 -0800 Subject: [PATCH 03/10] [checkstyle] Remove some more unusued imports --- .../apache/hadoop/hbase/master/MetaRegionLocationCache.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java index 3ee25482d3d6..610e6779b496 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java @@ -22,9 +22,6 @@ import java.util.Optional; import java.util.concurrent.ConcurrentNavigableMap; import org.apache.hadoop.hbase.HRegionLocation; -import org.apache.hadoop.hbase.ServerName; -import org.apache.hadoop.hbase.client.RegionInfoBuilder; -import org.apache.hadoop.hbase.client.RegionReplicaUtil; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.types.CopyOnWriteArrayMap; import org.apache.hadoop.hbase.util.RetryCounter; From f5b6b48cfc2a54dd59089746b6e19480a8ef5b7b Mon Sep 17 00:00:00 2001 From: Bharath Vissapragada Date: Tue, 19 Nov 2019 10:38:37 -0800 Subject: [PATCH 04/10] Address Nick's comments. --- .../hbase/master/MetaRegionLocationCache.java | 86 ++++++++++++------- .../client/TestMetaRegionLocationCache.java | 6 +- .../hbase/protobuf/TestProtobufUtil.java | 29 ++++++- .../hadoop/hbase/zookeeper/ZKWatcher.java | 31 +++++-- 4 files changed, 109 insertions(+), 43 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java index 610e6779b496..7eadffe19126 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java @@ -39,25 +39,32 @@ /** * A cache of meta region location metadata. Registers a listener on ZK to track changes to the * meta table znodes. Clients are expected to retry if the meta information is stale. This class - * is thread-safe. + * is thread-safe (a single instance of this class can be shared by multiple threads without race + * conditions). */ @InterfaceAudience.Private public class MetaRegionLocationCache extends ZKListener { private static final Logger LOG = LoggerFactory.getLogger(MetaRegionLocationCache.class); - // Maximum number of times we retry when ZK operation times out. + /** + * Maximum number of times we retry when ZK operation times out. + */ private static final int MAX_ZK_META_FETCH_RETRIES = 10; - // Sleep interval ms between ZK operation retries. + /** + * Sleep interval ms between ZK operation retries. + */ private static final int SLEEP_INTERVAL_MS_BETWEEN_RETRIES = 1000; private final RetryCounterFactory retryCounterFactory = new RetryCounterFactory(MAX_ZK_META_FETCH_RETRIES, SLEEP_INTERVAL_MS_BETWEEN_RETRIES); - // Cached meta region locations indexed by replica ID. - // CopyOnWriteArrayMap ensures synchronization during updates and a consistent snapshot during - // client requests. Even though CopyOnWriteArrayMap copies the data structure for every write, - // that should be OK since the size of the list is often small and mutations are not too often - // and we do not need to block client requests while mutations are in progress. + /** + * Cached meta region locations indexed by replica ID. + * CopyOnWriteArrayMap ensures synchronization during updates and a consistent snapshot during + * client requests. Even though CopyOnWriteArrayMap copies the data structure for every write, + * that should be OK since the size of the list is often small and mutations are not too often + * and we do not need to block client requests while mutations are in progress. + */ private final CopyOnWriteArrayMap cachedMetaLocations; private enum ZNodeOpType { @@ -77,28 +84,36 @@ private enum ZNodeOpType { populateInitialMetaLocations(); } + /** + * Populates the current snapshot of meta locations from ZK. If no meta znodes exist, it registers + * a watcher on base znode to check for any CREATE/DELETE events on the children. + */ private void populateInitialMetaLocations() { RetryCounter retryCounter = retryCounterFactory.create(); List znodes = null; - do { + while (retryCounter.shouldRetry()) { try { - znodes = watcher.getMetaReplicaNodes(); + znodes = watcher.getMetaReplicaNodesAndWatch(); break; } catch (KeeperException ke) { - LOG.debug("Error populating intial meta locations", ke); + LOG.debug("Error populating initial meta locations", ke); + if (!retryCounter.shouldRetry()) { + // Retries exhausted and watchers not set. This is not a desirable state since the cache + // could remain stale forever. Propagate the exception. + watcher.abort("Error populating meta locations", ke); + return; + } try { retryCounter.sleepUntilNextRetry(); } catch (InterruptedException ie) { - LOG.error("Interrupted while populating intial meta locations", ie); + Thread.currentThread().interrupt(); return; } - if (!retryCounter.shouldRetry()) { - LOG.error("Error populating intial meta locations. Retries exhausted. Last error: ", ke); - break; - } } - } while (retryCounter.shouldRetry()); + } if (znodes == null) { + // No meta znodes exist at this point but we registered a watcher on the base znode to listen + // for updates. They will be handled via nodeChildrenChanged(). return; } for (String znode: znodes) { @@ -135,7 +150,7 @@ private void updateMetaLocation(String path, ZNodeOpType opType) { int replicaId = watcher.getZNodePaths().getMetaReplicaIdFromPath(path); RetryCounter retryCounter = retryCounterFactory.create(); HRegionLocation location = null; - do { + while (retryCounter.shouldRetry()) { try { if (opType == ZNodeOpType.DELETED) { if (!ZKUtil.watchAndCheckExists(watcher, path)) { @@ -149,18 +164,18 @@ private void updateMetaLocation(String path, ZNodeOpType opType) { break; } catch (KeeperException e) { LOG.debug("Error getting meta location for path {}", path, e); - if (retryCounter.shouldRetry()) { - try { - retryCounter.sleepUntilNextRetry(); - } catch (InterruptedException ie) { - LOG.error("Interrupted while updating meta location for path {}", path, ie); - return; - } - } else { - LOG.error("Error getting meta location for path {}. Retries exhausted.", path, e); + if (!retryCounter.shouldRetry()) { + LOG.warn("Error getting meta location for path {}. Retries exhausted.", path, e); + break; + } + try { + retryCounter.sleepUntilNextRetry(); + } catch (InterruptedException ie) { + Thread.currentThread().interrupt(); + return; } } - } while (retryCounter.shouldRetry()); + } if (location == null) { cachedMetaLocations.remove(replicaId); return; @@ -169,12 +184,13 @@ private void updateMetaLocation(String path, ZNodeOpType opType) { } /** - * @return List of HRegionLocations for meta replica(s), null if the cache is empty. + * @return Optional list of HRegionLocations for meta replica(s), null if the cache is empty. + * */ public Optional> getMetaRegionLocations() { ConcurrentNavigableMap snapshot = cachedMetaLocations.tailMap(cachedMetaLocations.firstKey()); - if (snapshot == null || snapshot.isEmpty()) { + if (snapshot.isEmpty()) { // This could be possible if the master has not successfully initialized yet or meta region // is stuck in some weird state. return Optional.empty(); @@ -208,4 +224,14 @@ public void nodeDeleted(String path) { public void nodeDataChanged(String path) { updateMetaLocation(path, ZNodeOpType.CHANGED); } + + @Override + public void nodeChildrenChanged(String path) { + if (!path.equals(watcher.getZNodePaths().baseZNode)) { + return; + } + // Can get triggered for *any* children change, but that is OK. It does not happen once the + // initial set of meta znodes are populated. + populateInitialMetaLocations(); + } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java index 2f66148cf71d..2eb943d1bb62 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java @@ -17,7 +17,6 @@ */ package org.apache.hadoop.hbase.client; -import static junit.framework.TestCase.assertTrue; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import java.util.ArrayList; @@ -30,7 +29,6 @@ import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.master.RegionState; -import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.JVMClusterUtil; @@ -54,7 +52,6 @@ public class TestMetaRegionLocationCache { @BeforeClass public static void setUp() throws Exception { - TEST_UTIL.getConfiguration().set(BaseLoadBalancer.TABLES_ON_MASTER, "none"); TEST_UTIL.getConfiguration().setInt(HConstants.META_REPLICAS_NUM, 3); TEST_UTIL.startMiniCluster(3); REGISTRY = AsyncRegistryFactory.getRegistry(TEST_UTIL.getConfiguration()); @@ -84,7 +81,6 @@ private List getCurrentMetaLocations(ZKWatcher zk) throws Excep private void verifyCachedMetaLocations(HMaster master) throws Exception { List metaHRLs = master.getMetaRegionLocationCache().getMetaRegionLocations().get(); - assertTrue(metaHRLs != null); assertFalse(metaHRLs.isEmpty()); ZKWatcher zk = master.getZooKeeper(); List metaZnodes = zk.getMetaReplicaNodes(); @@ -121,4 +117,4 @@ private void verifyCachedMetaLocations(HMaster master) throws Exception { verifyCachedMetaLocations(masterThread.getMaster()); } } -} \ No newline at end of file +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestProtobufUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestProtobufUtil.java index ff29df838b5d..a1de329136bd 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestProtobufUtil.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestProtobufUtil.java @@ -19,7 +19,6 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; - import com.google.protobuf.ByteString; import java.io.IOException; import java.nio.ByteBuffer; @@ -29,13 +28,17 @@ import org.apache.hadoop.hbase.CellComparatorImpl; import org.apache.hadoop.hbase.ExtendedCellBuilderFactory; import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.client.Append; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Increment; import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.io.TimeRange; +import org.apache.hadoop.hbase.master.RegionState; import org.apache.hadoop.hbase.protobuf.generated.CellProtos; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Column; @@ -51,11 +54,12 @@ import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.MetaRegionServer; /** * Class to test ProtobufUtil. */ -@Category({MiscTests.class, SmallTests.class}) +@Category({ MiscTests.class, SmallTests.class}) public class TestProtobufUtil { @ClassRule @@ -348,4 +352,25 @@ public void testToCell() throws Exception { ProtobufUtil.toCell(ExtendedCellBuilderFactory.create(CellBuilderType.SHALLOW_COPY), cell); assertTrue(CellComparatorImpl.COMPARATOR.compare(offheapKV, newOffheapKV) == 0); } + + @Test + public void testMetaRegionState() throws Exception { + ServerName serverName = ServerName.valueOf("localhost", 1234, 5678); + for (RegionState.State state: RegionState.State.values()) { + RegionState regionState = + new RegionState(RegionInfoBuilder.FIRST_META_REGIONINFO, state, serverName); + MetaRegionServer metars = MetaRegionServer.newBuilder() + .setServer(org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil.toServerName(serverName)) + .setRpcVersion(HConstants.RPC_CURRENT_VERSION) + .setState(state.convert()).build(); + // Serialize + byte[] data = ProtobufUtil.prependPBMagic(metars.toByteArray()); + ProtobufUtil.prependPBMagic(data); + // Deserialize + RegionState regionStateNew = + org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil.parseMetaRegionStateFrom(data, 1); + assertEquals(regionState.getServerName(), regionStateNew.getServerName()); + assertEquals(regionState.getState(), regionStateNew.getState()); + } + } } diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKWatcher.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKWatcher.java index ce00af40495f..a71c39a9d752 100644 --- a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKWatcher.java +++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKWatcher.java @@ -384,13 +384,32 @@ public String prefix(final String str) { */ public List getMetaReplicaNodes() throws KeeperException { List childrenOfBaseNode = ZKUtil.listChildrenNoWatch(this, znodePaths.baseZNode); + return filterMetaReplicaNodes(childrenOfBaseNode); + } + + /** + * Same as {@link #getMetaReplicaNodes()} except that this also registers a watcher on base znode + * for subsequent CREATE/DELETE operations on child nodes. + */ + public List getMetaReplicaNodesAndWatch() throws KeeperException { + List childrenOfBaseNode = + ZKUtil.listChildrenAndWatchForNewChildren(this, znodePaths.baseZNode); + return filterMetaReplicaNodes(childrenOfBaseNode); + } + + /** + * @param nodes Input list of znodes + * @return Filtered list of znodes from nodes that belong to meta replica(s). + */ + private List filterMetaReplicaNodes(List nodes) { + if (nodes == null || nodes.isEmpty()) { + return new ArrayList<>(); + } List metaReplicaNodes = new ArrayList<>(2); - if (childrenOfBaseNode != null) { - String pattern = conf.get("zookeeper.znode.metaserver","meta-region-server"); - for (String child : childrenOfBaseNode) { - if (child.startsWith(pattern)) { - metaReplicaNodes.add(child); - } + String pattern = conf.get("zookeeper.znode.metaserver","meta-region-server"); + for (String child : nodes) { + if (child.startsWith(pattern)) { + metaReplicaNodes.add(child); } } return metaReplicaNodes; From ac5cc6221d638a8344d6c1b9970a65444a224af9 Mon Sep 17 00:00:00 2001 From: Bharath Vissapragada Date: Thu, 21 Nov 2019 22:47:34 -0800 Subject: [PATCH 05/10] Address Nick's comments - Part 2. - More test coverage - Better interrupted exception handling. --- .../apache/hadoop/hbase/master/HMaster.java | 2 +- .../hbase/master/MetaRegionLocationCache.java | 20 +++---- .../client/TestMetaRegionLocationCache.java | 58 +++++++++++++++++++ .../master/TestCloseAnOpeningRegion.java | 4 +- .../master/TestClusterRestartFailover.java | 2 +- .../TestRegionsRecoveryConfigManager.java | 4 +- .../master/TestShutdownBackupMaster.java | 2 +- .../TestOpenRegionProcedureBackoff.java | 2 +- .../TestOpenRegionProcedureHang.java | 2 +- ...RegionAssignedToMultipleRegionServers.java | 2 +- .../TestReportOnlineRegionsRace.java | 2 +- ...rtRegionStateTransitionFromDeadServer.java | 2 +- .../TestReportRegionStateTransitionRetry.java | 2 +- .../assignment/TestSCPGetRegionsRace.java | 2 +- .../TestWakeUpUnexpectedProcedure.java | 2 +- .../TestRegisterPeerWorkerWhenRestarting.java | 2 +- .../hbase/protobuf/TestProtobufUtil.java | 17 ++++-- .../TestRegionServerReportForDuty.java | 2 +- .../TestReplicationProcedureRetry.java | 2 +- .../hadoop/hbase/zookeeper/ZKWatcher.java | 8 +-- 20 files changed, 99 insertions(+), 40 deletions(-) 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 71c951174696..7674db88520f 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 @@ -515,7 +515,7 @@ public void doGet(HttpServletRequest request, * the master becomes the active one. */ public HMaster(final Configuration conf) - throws IOException, KeeperException { + throws InterruptedException, IOException { super(conf); TraceUtil.initTracer(conf); try { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java index 7eadffe19126..72022ccb3b42 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java @@ -74,7 +74,7 @@ private enum ZNodeOpType { DELETED }; - MetaRegionLocationCache(ZKWatcher zkWatcher) { + public MetaRegionLocationCache(ZKWatcher zkWatcher) throws InterruptedException { super(zkWatcher); cachedMetaLocations = new CopyOnWriteArrayMap<>(); watcher.registerListener(this); @@ -88,12 +88,12 @@ private enum ZNodeOpType { * Populates the current snapshot of meta locations from ZK. If no meta znodes exist, it registers * a watcher on base znode to check for any CREATE/DELETE events on the children. */ - private void populateInitialMetaLocations() { + private void populateInitialMetaLocations() throws InterruptedException { RetryCounter retryCounter = retryCounterFactory.create(); List znodes = null; while (retryCounter.shouldRetry()) { try { - znodes = watcher.getMetaReplicaNodesAndWatch(); + znodes = watcher.getMetaReplicaNodesAndWatchChildren(); break; } catch (KeeperException ke) { LOG.debug("Error populating initial meta locations", ke); @@ -103,12 +103,7 @@ private void populateInitialMetaLocations() { watcher.abort("Error populating meta locations", ke); return; } - try { - retryCounter.sleepUntilNextRetry(); - } catch (InterruptedException ie) { - Thread.currentThread().interrupt(); - return; - } + retryCounter.sleepUntilNextRetry(); } } if (znodes == null) { @@ -232,6 +227,11 @@ public void nodeChildrenChanged(String path) { } // Can get triggered for *any* children change, but that is OK. It does not happen once the // initial set of meta znodes are populated. - populateInitialMetaLocations(); + try { + populateInitialMetaLocations(); + } catch (InterruptedException ie) { + LOG.warn("Interrupted while initializing meta region cache", ie); + Thread.currentThread().interrupt(); + } } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java index 2eb943d1bb62..5d6d0f7f77fa 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java @@ -23,16 +23,21 @@ import java.util.Collections; import java.util.List; import org.apache.commons.io.IOUtils; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; +import org.apache.hadoop.hbase.MultithreadedTestUtil; +import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.master.HMaster; +import org.apache.hadoop.hbase.master.MetaRegionLocationCache; import org.apache.hadoop.hbase.master.RegionState; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.JVMClusterUtil; import org.apache.hadoop.hbase.zookeeper.MetaTableLocator; +import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hbase.zookeeper.ZKWatcher; import org.apache.hadoop.hbase.zookeeper.ZNodePaths; import org.junit.AfterClass; @@ -117,4 +122,57 @@ private void verifyCachedMetaLocations(HMaster master) throws Exception { verifyCachedMetaLocations(masterThread.getMaster()); } } + + /** + * Tests MetaRegionLocationCache's init procedure to make sure that it correctly watches the base + * znode for notifications. + */ + @Test public void testMetaRegionLocationCache() throws Exception { + final String parentZnodeName = "/randomznodename"; + Configuration conf = new Configuration(TEST_UTIL.getConfiguration()); + conf.set(HConstants.ZOOKEEPER_ZNODE_PARENT, parentZnodeName); + ServerName sn = ServerName.valueOf("localhost", 1234, 5678); + try (ZKWatcher zkWatcher = new ZKWatcher(conf, null, null, true)) { + // A thread that repeatedly creates and drops an unrelated child znode. This is to simulate + // some ZK activity in the background. + MultithreadedTestUtil.TestContext ctx = new MultithreadedTestUtil.TestContext(conf); + ctx.addThread(new MultithreadedTestUtil.RepeatingTestThread(ctx) { + @Override public void doAnAction() throws Exception { + final String testZnode = parentZnodeName + "/child"; + ZKUtil.createNodeIfNotExistsAndWatch(zkWatcher, testZnode, testZnode.getBytes()); + ZKUtil.deleteNode(zkWatcher, testZnode); + } + }); + ctx.startThreads(); + try { + MetaRegionLocationCache metaCache = new MetaRegionLocationCache(zkWatcher); + // meta znodes do not exist at this point, cache should be empty. + assertFalse(metaCache.getMetaRegionLocations().isPresent()); + // Set the meta locations for a random meta replicas, simulating an active hmaster meta + // assignment. + for (int i = 0; i < 3; i++) { + // Updates the meta znodes. + MetaTableLocator.setMetaLocation(zkWatcher, sn, i, RegionState.State.OPEN); + } + // Wait until the meta cache is populated. + int iters = 0; + while (iters++ < 10) { + if (metaCache.getMetaRegionLocations().isPresent() + && metaCache.getMetaRegionLocations().get().size() == 3) { + break; + } + Thread.sleep(1000); + } + List metaLocations = metaCache.getMetaRegionLocations().get(); + assertEquals(3, metaLocations.size()); + for (HRegionLocation location : metaLocations) { + assertEquals(sn, location.getServerName()); + } + } finally { + // clean up. + ctx.stop(); + ZKUtil.deleteChildrenRecursively(zkWatcher, parentZnodeName); + } + } + } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCloseAnOpeningRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCloseAnOpeningRegion.java index ba4d53510faa..9d1d0b25023a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCloseAnOpeningRegion.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCloseAnOpeningRegion.java @@ -65,7 +65,7 @@ public class TestCloseAnOpeningRegion { public static final class MockHMaster extends HMaster { - public MockHMaster(Configuration conf) throws IOException, KeeperException { + public MockHMaster(Configuration conf) throws InterruptedException, IOException { super(conf); } @@ -141,4 +141,4 @@ public void test() throws IOException, InterruptedException { table.put(new Put(Bytes.toBytes(0)).addColumn(CF, Bytes.toBytes("cq"), Bytes.toBytes(0))); } } -} \ No newline at end of file +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClusterRestartFailover.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClusterRestartFailover.java index a6844fcac091..aa516991b584 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClusterRestartFailover.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClusterRestartFailover.java @@ -147,7 +147,7 @@ private void setupTable() throws Exception { public static final class HMasterForTest extends HMaster { - public HMasterForTest(Configuration conf) throws IOException, KeeperException { + public HMasterForTest(Configuration conf) throws InterruptedException, IOException { super(conf); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionsRecoveryConfigManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionsRecoveryConfigManager.java index 22554d355b9a..017d8392119c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionsRecoveryConfigManager.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionsRecoveryConfigManager.java @@ -120,7 +120,7 @@ public void testChoreSchedule() throws Exception { // Make it public so that JVMClusterUtil can access it. public static class TestHMaster extends HMaster { - public TestHMaster(Configuration conf) throws IOException, KeeperException { + public TestHMaster(Configuration conf) throws InterruptedException, IOException { super(conf); } } @@ -144,4 +144,4 @@ public boolean isStopped() { } -} \ No newline at end of file +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestShutdownBackupMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestShutdownBackupMaster.java index d3a85209ccac..61d2e5ebc9fb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestShutdownBackupMaster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestShutdownBackupMaster.java @@ -56,7 +56,7 @@ public class TestShutdownBackupMaster { public static final class MockHMaster extends HMaster { - public MockHMaster(Configuration conf) throws IOException, KeeperException { + public MockHMaster(Configuration conf) throws InterruptedException, IOException { super(conf); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestOpenRegionProcedureBackoff.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestOpenRegionProcedureBackoff.java index ca0384e893eb..cfcc6053e327 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestOpenRegionProcedureBackoff.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestOpenRegionProcedureBackoff.java @@ -71,7 +71,7 @@ void persistToMeta(RegionStateNode regionNode) throws IOException { public static final class HMasterForTest extends HMaster { - public HMasterForTest(Configuration conf) throws IOException, KeeperException { + public HMasterForTest(Configuration conf) throws InterruptedException, IOException { super(conf); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestOpenRegionProcedureHang.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestOpenRegionProcedureHang.java index 0463721656b4..b28d3868fe03 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestOpenRegionProcedureHang.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestOpenRegionProcedureHang.java @@ -105,7 +105,7 @@ public ReportRegionStateTransitionResponse reportRegionStateTransition( public static final class HMasterForTest extends HMaster { - public HMasterForTest(Configuration conf) throws IOException, KeeperException { + public HMasterForTest(Configuration conf) throws InterruptedException, IOException { super(conf); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionAssignedToMultipleRegionServers.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionAssignedToMultipleRegionServers.java index 0d8202b69f26..9a3f1a56fe69 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionAssignedToMultipleRegionServers.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionAssignedToMultipleRegionServers.java @@ -110,7 +110,7 @@ public ReportRegionStateTransitionResponse reportRegionStateTransition( public static final class HMasterForTest extends HMaster { - public HMasterForTest(Configuration conf) throws IOException, KeeperException { + public HMasterForTest(Configuration conf) throws InterruptedException, IOException { super(conf); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestReportOnlineRegionsRace.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestReportOnlineRegionsRace.java index 371897bdbcdd..ebd584cbd8b6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestReportOnlineRegionsRace.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestReportOnlineRegionsRace.java @@ -106,7 +106,7 @@ public ReportRegionStateTransitionResponse reportRegionStateTransition( public static final class HMasterForTest extends HMaster { - public HMasterForTest(Configuration conf) throws IOException, KeeperException { + public HMasterForTest(Configuration conf) throws InterruptedException, IOException { super(conf); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestReportRegionStateTransitionFromDeadServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestReportRegionStateTransitionFromDeadServer.java index 6c9e5eb34ba6..a2d6f5b291a0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestReportRegionStateTransitionFromDeadServer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestReportRegionStateTransitionFromDeadServer.java @@ -117,7 +117,7 @@ public ReportRegionStateTransitionResponse reportRegionStateTransition( public static final class HMasterForTest extends HMaster { - public HMasterForTest(Configuration conf) throws IOException, KeeperException { + public HMasterForTest(Configuration conf) throws InterruptedException, IOException { super(conf); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestReportRegionStateTransitionRetry.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestReportRegionStateTransitionRetry.java index 6c191c9be8a1..8d30e70b9c02 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestReportRegionStateTransitionRetry.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestReportRegionStateTransitionRetry.java @@ -84,7 +84,7 @@ public ReportRegionStateTransitionResponse reportRegionStateTransition( public static final class HMasterForTest extends HMaster { - public HMasterForTest(Configuration conf) throws IOException, KeeperException { + public HMasterForTest(Configuration conf) throws InterruptedException, IOException { super(conf); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestSCPGetRegionsRace.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestSCPGetRegionsRace.java index cbbdbdc8d157..191b351a1271 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestSCPGetRegionsRace.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestSCPGetRegionsRace.java @@ -130,7 +130,7 @@ public List getRegionsOnServer(ServerName serverName) { public static final class HMasterForTest extends HMaster { - public HMasterForTest(Configuration conf) throws IOException, KeeperException { + public HMasterForTest(Configuration conf) throws InterruptedException, IOException { super(conf); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestWakeUpUnexpectedProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestWakeUpUnexpectedProcedure.java index 47c70a156997..f0876afa6cf4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestWakeUpUnexpectedProcedure.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestWakeUpUnexpectedProcedure.java @@ -198,7 +198,7 @@ public List createDestinationServersList() { public static final class HMasterForTest extends HMaster { - public HMasterForTest(Configuration conf) throws IOException, KeeperException { + public HMasterForTest(Configuration conf) throws InterruptedException, IOException { super(conf); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/TestRegisterPeerWorkerWhenRestarting.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/TestRegisterPeerWorkerWhenRestarting.java index f46bb418ea15..8cc5cb4c7363 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/TestRegisterPeerWorkerWhenRestarting.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/TestRegisterPeerWorkerWhenRestarting.java @@ -53,7 +53,7 @@ public class TestRegisterPeerWorkerWhenRestarting extends SyncReplicationTestBas public static final class HMasterForTest extends HMaster { - public HMasterForTest(Configuration conf) throws IOException, KeeperException { + public HMasterForTest(Configuration conf) throws IOException, InterruptedException { super(conf); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestProtobufUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestProtobufUtil.java index a1de329136bd..69e656fbe52c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestProtobufUtil.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestProtobufUtil.java @@ -356,21 +356,28 @@ public void testToCell() throws Exception { @Test public void testMetaRegionState() throws Exception { ServerName serverName = ServerName.valueOf("localhost", 1234, 5678); + // New region state style. for (RegionState.State state: RegionState.State.values()) { RegionState regionState = - new RegionState(RegionInfoBuilder.FIRST_META_REGIONINFO, state, serverName); + new RegionState(RegionInfoBuilder.FIRST_META_REGIONINFO, state, serverName); MetaRegionServer metars = MetaRegionServer.newBuilder() - .setServer(org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil.toServerName(serverName)) - .setRpcVersion(HConstants.RPC_CURRENT_VERSION) - .setState(state.convert()).build(); + .setServer(org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil.toServerName(serverName)) + .setRpcVersion(HConstants.RPC_CURRENT_VERSION) + .setState(state.convert()).build(); // Serialize byte[] data = ProtobufUtil.prependPBMagic(metars.toByteArray()); ProtobufUtil.prependPBMagic(data); // Deserialize RegionState regionStateNew = - org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil.parseMetaRegionStateFrom(data, 1); + org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil.parseMetaRegionStateFrom(data, 1); assertEquals(regionState.getServerName(), regionStateNew.getServerName()); assertEquals(regionState.getState(), regionStateNew.getState()); } + // old style. + RegionState rs = + org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil.parseMetaRegionStateFrom( + serverName.getVersionedBytes(), 1); + assertEquals(serverName, rs.getServerName()); + assertEquals(rs.getState(), RegionState.State.OPEN); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerReportForDuty.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerReportForDuty.java index aaf2d2eb9a04..ca11ae25d393 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerReportForDuty.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerReportForDuty.java @@ -122,7 +122,7 @@ public void stopCapturing() { * This test HMaster class will always throw ServerNotRunningYetException if checked. */ public static class NeverInitializedMaster extends HMaster { - public NeverInitializedMaster(Configuration conf) throws IOException, KeeperException { + public NeverInitializedMaster(Configuration conf) throws InterruptedException, IOException { super(conf); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationProcedureRetry.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationProcedureRetry.java index a2ae0b434249..8cc7ae8aadd9 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationProcedureRetry.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationProcedureRetry.java @@ -134,7 +134,7 @@ public static final class MockHMaster extends HMaster { private ReplicationPeerManager manager; - public MockHMaster(Configuration conf) throws IOException, KeeperException { + public MockHMaster(Configuration conf) throws InterruptedException, IOException { super(conf); } diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKWatcher.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKWatcher.java index a71c39a9d752..189ee5d31774 100644 --- a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKWatcher.java +++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKWatcher.java @@ -23,10 +23,8 @@ import java.util.ArrayList; import java.util.List; import java.util.concurrent.CopyOnWriteArrayList; -import java.util.concurrent.CountDownLatch; import java.util.regex.Matcher; import java.util.regex.Pattern; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Abortable; import org.apache.hadoop.hbase.AuthUtil; @@ -81,10 +79,6 @@ public class ZKWatcher implements Watcher, Abortable, Closeable { // listeners to be notified private final List listeners = new CopyOnWriteArrayList<>(); - // Used by ZKUtil:waitForZKConnectionIfAuthenticating to wait for SASL - // negotiation to complete - private CountDownLatch saslLatch = new CountDownLatch(1); - private final Configuration conf; /* A pattern that matches a Kerberos name, borrowed from Hadoop's KerberosName */ @@ -391,7 +385,7 @@ public List getMetaReplicaNodes() throws KeeperException { * Same as {@link #getMetaReplicaNodes()} except that this also registers a watcher on base znode * for subsequent CREATE/DELETE operations on child nodes. */ - public List getMetaReplicaNodesAndWatch() throws KeeperException { + public List getMetaReplicaNodesAndWatchChildren() throws KeeperException { List childrenOfBaseNode = ZKUtil.listChildrenAndWatchForNewChildren(this, znodePaths.baseZNode); return filterMetaReplicaNodes(childrenOfBaseNode); From 7b93a4aa81a052f112443efcb5757085cdc2e0d6 Mon Sep 17 00:00:00 2001 From: Bharath Vissapragada Date: Mon, 25 Nov 2019 08:38:32 -0800 Subject: [PATCH 06/10] [checkstyle] Remove unused imports --- .../org/apache/hadoop/hbase/master/TestCloseAnOpeningRegion.java | 1 - .../apache/hadoop/hbase/master/TestClusterRestartFailover.java | 1 - .../hadoop/hbase/master/TestRegionsRecoveryConfigManager.java | 1 - .../org/apache/hadoop/hbase/master/TestShutdownBackupMaster.java | 1 - .../hbase/master/assignment/TestOpenRegionProcedureBackoff.java | 1 - .../assignment/TestRegionAssignedToMultipleRegionServers.java | 1 - .../hbase/master/assignment/TestReportOnlineRegionsRace.java | 1 - .../TestReportRegionStateTransitionFromDeadServer.java | 1 - .../master/assignment/TestReportRegionStateTransitionRetry.java | 1 - .../hadoop/hbase/master/assignment/TestSCPGetRegionsRace.java | 1 - .../hbase/master/assignment/TestWakeUpUnexpectedProcedure.java | 1 - .../master/replication/TestRegisterPeerWorkerWhenRestarting.java | 1 - .../hadoop/hbase/replication/TestReplicationProcedureRetry.java | 1 - 13 files changed, 13 deletions(-) diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCloseAnOpeningRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCloseAnOpeningRegion.java index 9d1d0b25023a..c88c4780f503 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCloseAnOpeningRegion.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCloseAnOpeningRegion.java @@ -35,7 +35,6 @@ import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.zookeeper.KeeperException; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.ClassRule; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClusterRestartFailover.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClusterRestartFailover.java index aa516991b584..6aa52530b297 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClusterRestartFailover.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClusterRestartFailover.java @@ -42,7 +42,6 @@ import org.apache.hadoop.hbase.procedure2.Procedure; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.MasterTests; -import org.apache.zookeeper.KeeperException; import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionsRecoveryConfigManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionsRecoveryConfigManager.java index 017d8392119c..1cb5f0374a29 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionsRecoveryConfigManager.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionsRecoveryConfigManager.java @@ -28,7 +28,6 @@ import org.apache.hadoop.hbase.Stoppable; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; -import org.apache.zookeeper.KeeperException; import org.junit.After; import org.junit.Assert; import org.junit.Before; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestShutdownBackupMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestShutdownBackupMaster.java index 61d2e5ebc9fb..7e2e872cf98f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestShutdownBackupMaster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestShutdownBackupMaster.java @@ -30,7 +30,6 @@ import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread; -import org.apache.zookeeper.KeeperException; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.ClassRule; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestOpenRegionProcedureBackoff.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestOpenRegionProcedureBackoff.java index cfcc6053e327..64a4e077b8b8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestOpenRegionProcedureBackoff.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestOpenRegionProcedureBackoff.java @@ -35,7 +35,6 @@ import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.zookeeper.KeeperException; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.ClassRule; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionAssignedToMultipleRegionServers.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionAssignedToMultipleRegionServers.java index 9a3f1a56fe69..1914ded117be 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionAssignedToMultipleRegionServers.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionAssignedToMultipleRegionServers.java @@ -41,7 +41,6 @@ import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.zookeeper.KeeperException; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.ClassRule; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestReportOnlineRegionsRace.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestReportOnlineRegionsRace.java index ebd584cbd8b6..b43d8e15e037 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestReportOnlineRegionsRace.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestReportOnlineRegionsRace.java @@ -44,7 +44,6 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.IdLock; -import org.apache.zookeeper.KeeperException; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.ClassRule; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestReportRegionStateTransitionFromDeadServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestReportRegionStateTransitionFromDeadServer.java index a2d6f5b291a0..51a35faf4e01 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestReportRegionStateTransitionFromDeadServer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestReportRegionStateTransitionFromDeadServer.java @@ -44,7 +44,6 @@ import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.zookeeper.KeeperException; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.ClassRule; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestReportRegionStateTransitionRetry.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestReportRegionStateTransitionRetry.java index 8d30e70b9c02..7d46cb5b0ed9 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestReportRegionStateTransitionRetry.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestReportRegionStateTransitionRetry.java @@ -40,7 +40,6 @@ import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.zookeeper.KeeperException; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.ClassRule; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestSCPGetRegionsRace.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestSCPGetRegionsRace.java index 191b351a1271..de922f8c83c7 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestSCPGetRegionsRace.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestSCPGetRegionsRace.java @@ -43,7 +43,6 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.IdLock; -import org.apache.zookeeper.KeeperException; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.ClassRule; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestWakeUpUnexpectedProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestWakeUpUnexpectedProcedure.java index f0876afa6cf4..74a3292a904a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestWakeUpUnexpectedProcedure.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestWakeUpUnexpectedProcedure.java @@ -44,7 +44,6 @@ import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread; -import org.apache.zookeeper.KeeperException; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.ClassRule; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/TestRegisterPeerWorkerWhenRestarting.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/TestRegisterPeerWorkerWhenRestarting.java index 8cc5cb4c7363..a7257e26fbb6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/TestRegisterPeerWorkerWhenRestarting.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/TestRegisterPeerWorkerWhenRestarting.java @@ -33,7 +33,6 @@ import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread; -import org.apache.zookeeper.KeeperException; import org.junit.BeforeClass; import org.junit.ClassRule; import org.junit.Test; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationProcedureRetry.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationProcedureRetry.java index 8cc7ae8aadd9..22fa7adc5590 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationProcedureRetry.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationProcedureRetry.java @@ -36,7 +36,6 @@ import org.apache.hadoop.hbase.master.replication.ReplicationPeerManager; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.ReplicationTests; -import org.apache.zookeeper.KeeperException; import org.junit.After; import org.junit.AfterClass; import org.junit.BeforeClass; From dc2912a53a1f04ca72cf36e88001d243648b39fc Mon Sep 17 00:00:00 2001 From: Bharath Vissapragada Date: Tue, 26 Nov 2019 00:15:01 -0800 Subject: [PATCH 07/10] Gracefully handle InterruptedException. Test teardown can potentially trigger nodeChildrenChanged() changed events. It seems reasonable to swallow the interrupted exception in this thread as nothing else is affected by it. --- .../hadoop/hbase/master/MetaRegionLocationCache.java | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java index 72022ccb3b42..7b4f27390eee 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java @@ -106,11 +106,15 @@ private void populateInitialMetaLocations() throws InterruptedException { retryCounter.sleepUntilNextRetry(); } } - if (znodes == null) { + if (znodes == null || znodes.isEmpty()) { // No meta znodes exist at this point but we registered a watcher on the base znode to listen // for updates. They will be handled via nodeChildrenChanged(). return; } + if (znodes.size() == cachedMetaLocations.size()) { + // No new meta znodes got added. + return; + } for (String znode: znodes) { String path = ZNodePaths.joinZNode(watcher.getZNodePaths().baseZNode, znode); updateMetaLocation(path, ZNodeOpType.INIT); @@ -225,13 +229,11 @@ public void nodeChildrenChanged(String path) { if (!path.equals(watcher.getZNodePaths().baseZNode)) { return; } - // Can get triggered for *any* children change, but that is OK. It does not happen once the - // initial set of meta znodes are populated. try { populateInitialMetaLocations(); } catch (InterruptedException ie) { + // log and ignore, we can reload the cache later if needed. LOG.warn("Interrupted while initializing meta region cache", ie); - Thread.currentThread().interrupt(); } } } From 871b74992756f455d01442c14c1dc55f0249dbf3 Mon Sep 17 00:00:00 2001 From: Bharath Vissapragada Date: Mon, 2 Dec 2019 13:11:04 -0800 Subject: [PATCH 08/10] Rename populateInitMetaLocations to something more meaningful. --- .../apache/hadoop/hbase/master/MetaRegionLocationCache.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java index 7b4f27390eee..e39ad0770d84 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java @@ -81,14 +81,14 @@ public MetaRegionLocationCache(ZKWatcher zkWatcher) throws InterruptedException // Populate the initial snapshot of data from meta znodes. // This is needed because stand-by masters can potentially start after the initial znode // creation. - populateInitialMetaLocations(); + populateMetaLocations(); } /** * Populates the current snapshot of meta locations from ZK. If no meta znodes exist, it registers * a watcher on base znode to check for any CREATE/DELETE events on the children. */ - private void populateInitialMetaLocations() throws InterruptedException { + private void populateMetaLocations() throws InterruptedException { RetryCounter retryCounter = retryCounterFactory.create(); List znodes = null; while (retryCounter.shouldRetry()) { @@ -230,7 +230,7 @@ public void nodeChildrenChanged(String path) { return; } try { - populateInitialMetaLocations(); + populateMetaLocations(); } catch (InterruptedException ie) { // log and ignore, we can reload the cache later if needed. LOG.warn("Interrupted while initializing meta region cache", ie); From bec6c478a0cdc47e9428901d1804bd78574f7eeb Mon Sep 17 00:00:00 2001 From: Bharath Vissapragada Date: Wed, 4 Dec 2019 11:25:55 -0800 Subject: [PATCH 09/10] Better handling of InterruptedException and async init of cache. --- .../hadoop/hbase/zookeeper/ZNodePaths.java | 5 ++- .../apache/hadoop/hbase/master/HMaster.java | 6 +-- .../hbase/master/MetaRegionLocationCache.java | 37 ++++++++++++------- .../client/TestMetaRegionLocationCache.java | 8 ++++ .../master/TestCloseAnOpeningRegion.java | 2 +- .../TestRegionsRecoveryConfigManager.java | 2 +- .../master/TestShutdownBackupMaster.java | 2 +- .../TestOpenRegionProcedureBackoff.java | 2 +- .../TestOpenRegionProcedureHang.java | 2 +- ...RegionAssignedToMultipleRegionServers.java | 2 +- .../TestReportOnlineRegionsRace.java | 2 +- ...rtRegionStateTransitionFromDeadServer.java | 2 +- .../TestReportRegionStateTransitionRetry.java | 2 +- .../assignment/TestSCPGetRegionsRace.java | 2 +- .../TestWakeUpUnexpectedProcedure.java | 2 +- .../TestRegisterPeerWorkerWhenRestarting.java | 2 +- .../TestRegionServerReportForDuty.java | 2 +- .../TestReplicationProcedureRetry.java | 2 +- .../hadoop/hbase/zookeeper/ZKWatcher.java | 2 +- 19 files changed, 52 insertions(+), 34 deletions(-) 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 a1cc47fcc2b9..9185de530332 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 @@ -40,7 +40,8 @@ public class ZNodePaths { // TODO: Replace this with ZooKeeper constant when ZOOKEEPER-277 is resolved. public static final char ZNODE_PATH_SEPARATOR = '/'; - public final static String META_ZNODE_PREFIX = "meta-region-server"; + public static final String META_ZNODE_PREFIX_CONF_KEY = "zookeeper.znode.metaserver"; + public static final String META_ZNODE_PREFIX = "meta-region-server"; private static final String DEFAULT_SNAPSHOT_CLEANUP_ZNODE = "snapshot-cleanup"; // base znode for this cluster @@ -96,7 +97,7 @@ public class ZNodePaths { public ZNodePaths(Configuration conf) { baseZNode = conf.get(ZOOKEEPER_ZNODE_PARENT, DEFAULT_ZOOKEEPER_ZNODE_PARENT); ImmutableMap.Builder builder = ImmutableMap.builder(); - metaZNodePrefix = conf.get("zookeeper.znode.metaserver", META_ZNODE_PREFIX); + metaZNodePrefix = conf.get(META_ZNODE_PREFIX_CONF_KEY, META_ZNODE_PREFIX); String defaultMetaReplicaZNode = ZNodePaths.joinZNode(baseZNode, metaZNodePrefix); builder.put(DEFAULT_REPLICA_ID, defaultMetaReplicaZNode); int numMetaReplicas = conf.getInt(META_REPLICAS_NUM, DEFAULT_META_REPLICA_NUM); 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 7674db88520f..55d1a1f34c07 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 @@ -514,8 +514,7 @@ public void doGet(HttpServletRequest request, * #finishActiveMasterInitialization(MonitoredTask) after * the master becomes the active one. */ - public HMaster(final Configuration conf) - throws InterruptedException, IOException { + public HMaster(final Configuration conf) throws IOException { super(conf); TraceUtil.initTracer(conf); try { @@ -528,7 +527,6 @@ public HMaster(final Configuration conf) } else { maintenanceMode = false; } - metaRegionLocationCache = new MetaRegionLocationCache(this.zooKeeper); this.rsFatals = new MemoryBoundedLogMessageBuffer( conf.getLong("hbase.master.buffer.for.rs.fatals", 1 * 1024 * 1024)); LOG.info("hbase.rootdir={}, hbase.cluster.distributed={}", getDataRootDir(), @@ -576,8 +574,10 @@ public HMaster(final Configuration conf) // Some unit tests don't need a cluster, so no zookeeper at all if (!conf.getBoolean("hbase.testing.nocluster", false)) { + this.metaRegionLocationCache = new MetaRegionLocationCache(this.zooKeeper); this.activeMasterManager = new ActiveMasterManager(zooKeeper, this.serverName, this); } else { + this.metaRegionLocationCache = null; this.activeMasterManager = null; } cachedClusterId = new CachedClusterId(conf); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java index e39ad0770d84..e1de973616e2 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java @@ -21,6 +21,7 @@ import java.util.List; import java.util.Optional; import java.util.concurrent.ConcurrentNavigableMap; +import java.util.concurrent.ThreadFactory; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.types.CopyOnWriteArrayMap; @@ -34,6 +35,7 @@ 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.ProtobufUtil; /** @@ -55,6 +57,7 @@ public class MetaRegionLocationCache extends ZKListener { * Sleep interval ms between ZK operation retries. */ private static final int SLEEP_INTERVAL_MS_BETWEEN_RETRIES = 1000; + private static final int SLEEP_INTERVAL_MS_MAX = 10000; private final RetryCounterFactory retryCounterFactory = new RetryCounterFactory(MAX_ZK_META_FETCH_RETRIES, SLEEP_INTERVAL_MS_BETWEEN_RETRIES); @@ -72,24 +75,30 @@ private enum ZNodeOpType { CREATED, CHANGED, DELETED - }; + } - public MetaRegionLocationCache(ZKWatcher zkWatcher) throws InterruptedException { + public MetaRegionLocationCache(ZKWatcher zkWatcher) { super(zkWatcher); cachedMetaLocations = new CopyOnWriteArrayMap<>(); watcher.registerListener(this); // Populate the initial snapshot of data from meta znodes. // This is needed because stand-by masters can potentially start after the initial znode - // creation. - populateMetaLocations(); + // creation. It blocks forever until the initial meta locations are loaded from ZK and watchers + // are established. Subsequent updates are handled by the registered listener. Also, this runs + // in a separate thread in the background to not block master init. + ThreadFactory threadFactory = new ThreadFactoryBuilder().setDaemon(true).build(); + RetryCounterFactory retryFactory = new RetryCounterFactory( + Integer.MAX_VALUE, SLEEP_INTERVAL_MS_BETWEEN_RETRIES, SLEEP_INTERVAL_MS_MAX); + threadFactory.newThread( + ()->loadMetaLocationsFromZk(retryFactory.create(), ZNodeOpType.INIT)).start(); } /** * Populates the current snapshot of meta locations from ZK. If no meta znodes exist, it registers * a watcher on base znode to check for any CREATE/DELETE events on the children. + * @param retryCounter controls the number of retries and sleep between retries. */ - private void populateMetaLocations() throws InterruptedException { - RetryCounter retryCounter = retryCounterFactory.create(); + private void loadMetaLocationsFromZk(RetryCounter retryCounter, ZNodeOpType opType) { List znodes = null; while (retryCounter.shouldRetry()) { try { @@ -103,7 +112,12 @@ private void populateMetaLocations() throws InterruptedException { watcher.abort("Error populating meta locations", ke); return; } - retryCounter.sleepUntilNextRetry(); + try { + retryCounter.sleepUntilNextRetry(); + } catch (InterruptedException ie) { + LOG.error("Interrupted while loading meta locations from ZK", ie); + return; + } } } if (znodes == null || znodes.isEmpty()) { @@ -117,7 +131,7 @@ private void populateMetaLocations() throws InterruptedException { } for (String znode: znodes) { String path = ZNodePaths.joinZNode(watcher.getZNodePaths().baseZNode, znode); - updateMetaLocation(path, ZNodeOpType.INIT); + updateMetaLocation(path, opType); } } @@ -229,11 +243,6 @@ public void nodeChildrenChanged(String path) { if (!path.equals(watcher.getZNodePaths().baseZNode)) { return; } - try { - populateMetaLocations(); - } catch (InterruptedException ie) { - // log and ignore, we can reload the cache later if needed. - LOG.warn("Interrupted while initializing meta region cache", ie); - } + loadMetaLocationsFromZk(retryCounterFactory.create(), ZNodeOpType.CHANGED); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java index 5d6d0f7f77fa..02236a67f44a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java @@ -84,6 +84,14 @@ private List getCurrentMetaLocations(ZKWatcher zk) throws Excep // Verifies that the cached meta locations in the given master are in sync with what is in ZK. private void verifyCachedMetaLocations(HMaster master) throws Exception { + // Wait until initial meta locations are loaded. + int retries = 0; + while (!master.getMetaRegionLocationCache().getMetaRegionLocations().isPresent()) { + Thread.sleep(1000); + if (++retries == 10) { + break; + } + } List metaHRLs = master.getMetaRegionLocationCache().getMetaRegionLocations().get(); assertFalse(metaHRLs.isEmpty()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCloseAnOpeningRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCloseAnOpeningRegion.java index c88c4780f503..492222464f99 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCloseAnOpeningRegion.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCloseAnOpeningRegion.java @@ -64,7 +64,7 @@ public class TestCloseAnOpeningRegion { public static final class MockHMaster extends HMaster { - public MockHMaster(Configuration conf) throws InterruptedException, IOException { + public MockHMaster(Configuration conf) throws IOException { super(conf); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionsRecoveryConfigManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionsRecoveryConfigManager.java index 1cb5f0374a29..d29e061d07fd 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionsRecoveryConfigManager.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionsRecoveryConfigManager.java @@ -119,7 +119,7 @@ public void testChoreSchedule() throws Exception { // Make it public so that JVMClusterUtil can access it. public static class TestHMaster extends HMaster { - public TestHMaster(Configuration conf) throws InterruptedException, IOException { + public TestHMaster(Configuration conf) throws IOException { super(conf); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestShutdownBackupMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestShutdownBackupMaster.java index 7e2e872cf98f..a42a4046d54f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestShutdownBackupMaster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestShutdownBackupMaster.java @@ -55,7 +55,7 @@ public class TestShutdownBackupMaster { public static final class MockHMaster extends HMaster { - public MockHMaster(Configuration conf) throws InterruptedException, IOException { + public MockHMaster(Configuration conf) throws IOException { super(conf); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestOpenRegionProcedureBackoff.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestOpenRegionProcedureBackoff.java index 64a4e077b8b8..4112da713c14 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestOpenRegionProcedureBackoff.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestOpenRegionProcedureBackoff.java @@ -70,7 +70,7 @@ void persistToMeta(RegionStateNode regionNode) throws IOException { public static final class HMasterForTest extends HMaster { - public HMasterForTest(Configuration conf) throws InterruptedException, IOException { + public HMasterForTest(Configuration conf) throws IOException { super(conf); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestOpenRegionProcedureHang.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestOpenRegionProcedureHang.java index b28d3868fe03..a25368f4e420 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestOpenRegionProcedureHang.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestOpenRegionProcedureHang.java @@ -105,7 +105,7 @@ public ReportRegionStateTransitionResponse reportRegionStateTransition( public static final class HMasterForTest extends HMaster { - public HMasterForTest(Configuration conf) throws InterruptedException, IOException { + public HMasterForTest(Configuration conf) throws IOException { super(conf); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionAssignedToMultipleRegionServers.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionAssignedToMultipleRegionServers.java index 1914ded117be..44af256b92af 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionAssignedToMultipleRegionServers.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionAssignedToMultipleRegionServers.java @@ -109,7 +109,7 @@ public ReportRegionStateTransitionResponse reportRegionStateTransition( public static final class HMasterForTest extends HMaster { - public HMasterForTest(Configuration conf) throws InterruptedException, IOException { + public HMasterForTest(Configuration conf) throws IOException { super(conf); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestReportOnlineRegionsRace.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestReportOnlineRegionsRace.java index b43d8e15e037..d07dfef1ae32 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestReportOnlineRegionsRace.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestReportOnlineRegionsRace.java @@ -105,7 +105,7 @@ public ReportRegionStateTransitionResponse reportRegionStateTransition( public static final class HMasterForTest extends HMaster { - public HMasterForTest(Configuration conf) throws InterruptedException, IOException { + public HMasterForTest(Configuration conf) throws IOException { super(conf); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestReportRegionStateTransitionFromDeadServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestReportRegionStateTransitionFromDeadServer.java index 51a35faf4e01..1de806fe0ffb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestReportRegionStateTransitionFromDeadServer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestReportRegionStateTransitionFromDeadServer.java @@ -116,7 +116,7 @@ public ReportRegionStateTransitionResponse reportRegionStateTransition( public static final class HMasterForTest extends HMaster { - public HMasterForTest(Configuration conf) throws InterruptedException, IOException { + public HMasterForTest(Configuration conf) throws IOException { super(conf); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestReportRegionStateTransitionRetry.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestReportRegionStateTransitionRetry.java index 7d46cb5b0ed9..71c4693cfa60 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestReportRegionStateTransitionRetry.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestReportRegionStateTransitionRetry.java @@ -83,7 +83,7 @@ public ReportRegionStateTransitionResponse reportRegionStateTransition( public static final class HMasterForTest extends HMaster { - public HMasterForTest(Configuration conf) throws InterruptedException, IOException { + public HMasterForTest(Configuration conf) throws IOException { super(conf); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestSCPGetRegionsRace.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestSCPGetRegionsRace.java index de922f8c83c7..d676af929392 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestSCPGetRegionsRace.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestSCPGetRegionsRace.java @@ -129,7 +129,7 @@ public List getRegionsOnServer(ServerName serverName) { public static final class HMasterForTest extends HMaster { - public HMasterForTest(Configuration conf) throws InterruptedException, IOException { + public HMasterForTest(Configuration conf) throws IOException { super(conf); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestWakeUpUnexpectedProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestWakeUpUnexpectedProcedure.java index 74a3292a904a..62e31615def4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestWakeUpUnexpectedProcedure.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestWakeUpUnexpectedProcedure.java @@ -197,7 +197,7 @@ public List createDestinationServersList() { public static final class HMasterForTest extends HMaster { - public HMasterForTest(Configuration conf) throws InterruptedException, IOException { + public HMasterForTest(Configuration conf) throws IOException { super(conf); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/TestRegisterPeerWorkerWhenRestarting.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/TestRegisterPeerWorkerWhenRestarting.java index a7257e26fbb6..4dff86d6ce1e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/TestRegisterPeerWorkerWhenRestarting.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/TestRegisterPeerWorkerWhenRestarting.java @@ -52,7 +52,7 @@ public class TestRegisterPeerWorkerWhenRestarting extends SyncReplicationTestBas public static final class HMasterForTest extends HMaster { - public HMasterForTest(Configuration conf) throws IOException, InterruptedException { + public HMasterForTest(Configuration conf) throws IOException { super(conf); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerReportForDuty.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerReportForDuty.java index ca11ae25d393..f61a77ec3937 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerReportForDuty.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerReportForDuty.java @@ -122,7 +122,7 @@ public void stopCapturing() { * This test HMaster class will always throw ServerNotRunningYetException if checked. */ public static class NeverInitializedMaster extends HMaster { - public NeverInitializedMaster(Configuration conf) throws InterruptedException, IOException { + public NeverInitializedMaster(Configuration conf) throws IOException { super(conf); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationProcedureRetry.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationProcedureRetry.java index 22fa7adc5590..e9fcc6634664 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationProcedureRetry.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationProcedureRetry.java @@ -133,7 +133,7 @@ public static final class MockHMaster extends HMaster { private ReplicationPeerManager manager; - public MockHMaster(Configuration conf) throws InterruptedException, IOException { + public MockHMaster(Configuration conf) throws IOException { super(conf); } diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKWatcher.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKWatcher.java index 189ee5d31774..71918a7d216c 100644 --- a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKWatcher.java +++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKWatcher.java @@ -400,7 +400,7 @@ private List filterMetaReplicaNodes(List nodes) { return new ArrayList<>(); } List metaReplicaNodes = new ArrayList<>(2); - String pattern = conf.get("zookeeper.znode.metaserver","meta-region-server"); + String pattern = conf.get(ZNodePaths.META_ZNODE_PREFIX_CONF_KEY, ZNodePaths.META_ZNODE_PREFIX); for (String child : nodes) { if (child.startsWith(pattern)) { metaReplicaNodes.add(child); From ef7c0f0bfd01fa13796f7f78c3f4f18d251713e5 Mon Sep 17 00:00:00 2001 From: Bharath Vissapragada Date: Wed, 4 Dec 2019 14:57:48 -0800 Subject: [PATCH 10/10] Address comments from Nick --- .../org/apache/hadoop/hbase/master/MetaRegionLocationCache.java | 1 + .../apache/hadoop/hbase/master/TestClusterRestartFailover.java | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java index e1de973616e2..f4e91b56051d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java @@ -116,6 +116,7 @@ private void loadMetaLocationsFromZk(RetryCounter retryCounter, ZNodeOpType opTy retryCounter.sleepUntilNextRetry(); } catch (InterruptedException ie) { LOG.error("Interrupted while loading meta locations from ZK", ie); + Thread.currentThread().interrupt(); return; } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClusterRestartFailover.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClusterRestartFailover.java index 6aa52530b297..84722df676dc 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClusterRestartFailover.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClusterRestartFailover.java @@ -146,7 +146,7 @@ private void setupTable() throws Exception { public static final class HMasterForTest extends HMaster { - public HMasterForTest(Configuration conf) throws InterruptedException, IOException { + public HMasterForTest(Configuration conf) throws IOException { super(conf); }