[GitHub] [hbase] ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta region locations in masters

2019-12-04 Thread GitBox
ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta 
region locations in masters
URL: https://github.com/apache/hbase/pull/830#discussion_r354021378
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java
 ##
 @@ -0,0 +1,248 @@
+/*
+ * 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 java.util.concurrent.ThreadFactory;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+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;
+import 
org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
+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
+ * meta table znodes. Clients are expected to retry if the meta information is 
stale. This class
+ * 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.
+   */
+  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 static final int SLEEP_INTERVAL_MS_MAX = 1;
+  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.
+   */
+  private final CopyOnWriteArrayMap 
cachedMetaLocations;
+
+  private enum ZNodeOpType {
+INIT,
+CREATED,
+CHANGED,
+DELETED
+  }
+
+  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. 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 

[GitHub] [hbase] ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta region locations in masters

2019-12-03 Thread GitBox
ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta 
region locations in masters
URL: https://github.com/apache/hbase/pull/830#discussion_r353491403
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
 ##
 @@ -522,7 +528,7 @@ public HMaster(final Configuration conf)
   } else {
 maintenanceMode = false;
   }
-
+  metaRegionLocationCache = new MetaRegionLocationCache(this.zooKeeper);
 
 Review comment:
   Looks like initialization should move down to line ~574 inside the `if 
(!conf.getBoolean("hbase.testing.nocluster", false))` block -- seems that some 
tests depend on a master but don't spin up a ZK.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [hbase] ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta region locations in masters

2019-12-03 Thread GitBox
ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta 
region locations in masters
URL: https://github.com/apache/hbase/pull/830#discussion_r353381948
 
 

 ##
 File path: 
hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java
 ##
 @@ -0,0 +1,178 @@
+/*
+ * 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 org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import java.util.ArrayList;
+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;
+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(TestMetaRegionLocationCache.class);
+
+  private static final HBaseTestingUtility TEST_UTIL = new 
HBaseTestingUtility();
+  private static AsyncRegistry REGISTRY;
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+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().getMetaRegionLocations().get();
+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 

[GitHub] [hbase] ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta region locations in masters

2019-12-03 Thread GitBox
ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta 
region locations in masters
URL: https://github.com/apache/hbase/pull/830#discussion_r353384464
 
 

 ##
 File path: 
hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKWatcher.java
 ##
 @@ -384,13 +378,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 getMetaReplicaNodesAndWatchChildren() 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");
 
 Review comment:
   nit: this class, or 
hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZNodePaths.java 
should own this configuration point and its default value.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [hbase] ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta region locations in masters

2019-11-22 Thread GitBox
ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta 
region locations in masters
URL: https://github.com/apache/hbase/pull/830#discussion_r349774266
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java
 ##
 @@ -0,0 +1,237 @@
+/*
+ * 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.exceptions.DeserializationException;
+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;
+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
+ * meta table znodes. Clients are expected to retry if the meta information is 
stale. This class
+ * 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.
+   */
+  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);
+
+  /**
+   * 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);
+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();
+  }
+
+  /**
+   * 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;
+while (retryCounter.shouldRetry()) {
+  try {
+znodes = watcher.getMetaReplicaNodesAndWatch();
+break;
+  } catch (KeeperException 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) {
+  

[GitHub] [hbase] ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta region locations in masters

2019-11-22 Thread GitBox
ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta 
region locations in masters
URL: https://github.com/apache/hbase/pull/830#discussion_r349775033
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java
 ##
 @@ -0,0 +1,237 @@
+/*
+ * 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.exceptions.DeserializationException;
+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;
+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
+ * meta table znodes. Clients are expected to retry if the meta information is 
stale. This class
+ * 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.
+   */
+  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);
+
+  /**
+   * 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);
+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();
+  }
+
+  /**
+   * 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;
+while (retryCounter.shouldRetry()) {
+  try {
+znodes = watcher.getMetaReplicaNodesAndWatch();
+break;
+  } catch (KeeperException 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) {
+  

[GitHub] [hbase] ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta region locations in masters

2019-11-21 Thread GitBox
ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta 
region locations in masters
URL: https://github.com/apache/hbase/pull/830#discussion_r349383351
 
 

 ##
 File path: 
hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java
 ##
 @@ -0,0 +1,120 @@
+/*
+ * 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 org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import java.util.ArrayList;
+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.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 {
 
 Review comment:
   Mind adding tests for creation and deletion of unrelated znodes under 
`/hbase` ? That'll give us confidence that re-using the initializer is safe.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [hbase] ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta region locations in masters

2019-11-21 Thread GitBox
ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta 
region locations in masters
URL: https://github.com/apache/hbase/pull/830#discussion_r349385225
 
 

 ##
 File path: 
hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestProtobufUtil.java
 ##
 @@ -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 {
 
 Review comment:
   This covers the "new" state info, but what about the old serialized version?


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [hbase] ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta region locations in masters

2019-11-21 Thread GitBox
ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta 
region locations in masters
URL: https://github.com/apache/hbase/pull/830#discussion_r349382703
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java
 ##
 @@ -0,0 +1,237 @@
+/*
+ * 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.exceptions.DeserializationException;
+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;
+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
+ * meta table znodes. Clients are expected to retry if the meta information is 
stale. This class
+ * 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.
+   */
+  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);
+
+  /**
+   * 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);
+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();
+  }
+
+  /**
+   * 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;
+while (retryCounter.shouldRetry()) {
+  try {
+znodes = watcher.getMetaReplicaNodesAndWatch();
+break;
+  } catch (KeeperException 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) {
+  

[GitHub] [hbase] ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta region locations in masters

2019-11-21 Thread GitBox
ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta 
region locations in masters
URL: https://github.com/apache/hbase/pull/830#discussion_r349376520
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java
 ##
 @@ -0,0 +1,211 @@
+/*
+ * 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.exceptions.DeserializationException;
+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;
+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
+ * 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);
+
+  // 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);
+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
 
 Review comment:
   Yeah I think it's more clear. One more improvement would be to distinguish 
in your method names whether a method is handling the parent znode or one of 
the replicaId children. I was misunderstanding how watch notification works -- 
specifically I was expecting that a watch notification event for 
`nodeDataChanged` includes the new node data and it's updated version 
information; not the case. I now understand why you have all this 
back-and-forth with zk.
   
   Writing a good test for these situations is difficult without inserting 
barriers into the masters. Let me think about how to do that in our test 
harness.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [hbase] ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta region locations in masters

2019-11-21 Thread GitBox
ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta 
region locations in masters
URL: https://github.com/apache/hbase/pull/830#discussion_r349383132
 
 

 ##
 File path: 
hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java
 ##
 @@ -0,0 +1,124 @@
+/*
+ * 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.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(TestMetaRegionLocationCache.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().getMetaRegionLocations().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 

[GitHub] [hbase] ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta region locations in masters

2019-11-21 Thread GitBox
ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta 
region locations in masters
URL: https://github.com/apache/hbase/pull/830#discussion_r349385405
 
 

 ##
 File path: 
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 {
 
 Review comment:
   `getMetaReplicaNodesAndWatchChildren`


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [hbase] ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta region locations in masters

2019-11-21 Thread GitBox
ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta 
region locations in masters
URL: https://github.com/apache/hbase/pull/830#discussion_r349379911
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java
 ##
 @@ -0,0 +1,237 @@
+/*
+ * 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.exceptions.DeserializationException;
+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;
+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
+ * meta table znodes. Clients are expected to retry if the meta information is 
stale. This class
+ * 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.
+   */
+  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);
+
+  /**
+   * 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);
+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();
+  }
+
+  /**
+   * 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;
+while (retryCounter.shouldRetry()) {
+  try {
+znodes = watcher.getMetaReplicaNodesAndWatch();
+break;
+  } catch (KeeperException 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) {
+  

[GitHub] [hbase] ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta region locations in masters

2019-11-21 Thread GitBox
ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta 
region locations in masters
URL: https://github.com/apache/hbase/pull/830#discussion_r349325137
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java
 ##
 @@ -0,0 +1,237 @@
+/*
+ * 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.exceptions.DeserializationException;
+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;
+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
+ * meta table znodes. Clients are expected to retry if the meta information is 
stale. This class
+ * 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.
+   */
+  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);
+
+  /**
+   * 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);
+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();
+  }
+
+  /**
+   * 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;
+while (retryCounter.shouldRetry()) {
+  try {
+znodes = watcher.getMetaReplicaNodesAndWatch();
+break;
+  } catch (KeeperException 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) {
+  

[GitHub] [hbase] ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta region locations in masters

2019-11-18 Thread GitBox
ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta 
region locations in masters
URL: https://github.com/apache/hbase/pull/830#discussion_r347666779
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java
 ##
 @@ -0,0 +1,211 @@
+/*
+ * 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.exceptions.DeserializationException;
+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;
+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
+ * 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);
+
+  // 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);
+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 

[GitHub] [hbase] ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta region locations in masters

2019-11-18 Thread GitBox
ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta 
region locations in masters
URL: https://github.com/apache/hbase/pull/830#discussion_r347673418
 
 

 ##
 File path: 
hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java
 ##
 @@ -0,0 +1,124 @@
+/*
+ * 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.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(TestMetaRegionLocationCache.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");
 
 Review comment:
   `"none"`? this configuration point is read as a boolean... sometimes. Or a 
string other times  I asked over on HBASE-18511. Over in the book, 
https://hbase.apache.org/book.html#_changes_of_note > "Master hosting regions" 
feature broken and unsupported , I guess it's a boolean.
   
   Either way, I think you want the default behavior, which is to not carry 
tables on master, so you can just leave it unspecified?


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [hbase] ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta region locations in masters

2019-11-18 Thread GitBox
ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta 
region locations in masters
URL: https://github.com/apache/hbase/pull/830#discussion_r347658931
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java
 ##
 @@ -0,0 +1,211 @@
+/*
+ * 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.exceptions.DeserializationException;
+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;
+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
+ * 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);
+
+  // 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);
+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);
 
 Review comment:
   I think these are okay at `WARN`. Thoughts?


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [hbase] ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta region locations in masters

2019-11-18 Thread GitBox
ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta 
region locations in masters
URL: https://github.com/apache/hbase/pull/830#discussion_r347663604
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java
 ##
 @@ -0,0 +1,211 @@
+/*
+ * 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.exceptions.DeserializationException;
+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;
+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
+ * 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);
+
+  // 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);
+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 

[GitHub] [hbase] ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta region locations in masters

2019-11-18 Thread GitBox
ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta 
region locations in masters
URL: https://github.com/apache/hbase/pull/830#discussion_r347674584
 
 

 ##
 File path: 
hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java
 ##
 @@ -0,0 +1,124 @@
+/*
+ * 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.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(TestMetaRegionLocationCache.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().getMetaRegionLocations().get();
+assertTrue(metaHRLs != null);
 
 Review comment:
   Because you used `Optional.get()` above, this can never be null. Also, 
`assertNotNull` exists.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [hbase] ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta region locations in masters

2019-11-18 Thread GitBox
ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta 
region locations in masters
URL: https://github.com/apache/hbase/pull/830#discussion_r347659731
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java
 ##
 @@ -0,0 +1,211 @@
+/*
+ * 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.exceptions.DeserializationException;
+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;
+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
+ * 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);
+
+  // 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);
+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;
 
 Review comment:
   Instead of logging and swallowing the exception, I think an 
`InterruptedException` should be rethrown. I cannot think of a case where we 
actually want to swallow the interruption? In that case, no need for the log 
line either.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [hbase] ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta region locations in masters

2019-11-18 Thread GitBox
ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta 
region locations in masters
URL: https://github.com/apache/hbase/pull/830#discussion_r347659868
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java
 ##
 @@ -0,0 +1,211 @@
+/*
+ * 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.exceptions.DeserializationException;
+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;
+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
+ * 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);
+
+  // 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);
+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);
 
 Review comment:
   Also, I think `DEBUG` is fine because presumably cache population can be 
accomplished at a later time.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache 

[GitHub] [hbase] ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta region locations in masters

2019-11-18 Thread GitBox
ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta 
region locations in masters
URL: https://github.com/apache/hbase/pull/830#discussion_r347675910
 
 

 ##
 File path: 
hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java
 ##
 @@ -0,0 +1,124 @@
+/*
+ * 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.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(TestMetaRegionLocationCache.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().getMetaRegionLocations().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 

[GitHub] [hbase] ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta region locations in masters

2019-11-18 Thread GitBox
ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta 
region locations in masters
URL: https://github.com/apache/hbase/pull/830#discussion_r347656749
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java
 ##
 @@ -0,0 +1,211 @@
+/*
+ * 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.exceptions.DeserializationException;
+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;
+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
+ * 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);
+
+  // 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);
+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);
 
 Review comment:
   "initial"


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [hbase] ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta region locations in masters

2019-11-18 Thread GitBox
ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta 
region locations in masters
URL: https://github.com/apache/hbase/pull/830#discussion_r347675740
 
 

 ##
 File path: 
hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java
 ##
 @@ -0,0 +1,124 @@
+/*
+ * 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.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(TestMetaRegionLocationCache.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().getMetaRegionLocations().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 

[GitHub] [hbase] ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta region locations in masters

2019-11-18 Thread GitBox
ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta 
region locations in masters
URL: https://github.com/apache/hbase/pull/830#discussion_r347656880
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java
 ##
 @@ -0,0 +1,211 @@
+/*
+ * 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.exceptions.DeserializationException;
+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;
+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
+ * 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);
+
+  // 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);
+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);
 
 Review comment:
   "initial"


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [hbase] ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta region locations in masters

2019-11-18 Thread GitBox
ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta 
region locations in masters
URL: https://github.com/apache/hbase/pull/830#discussion_r347667370
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java
 ##
 @@ -0,0 +1,211 @@
+/*
+ * 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.exceptions.DeserializationException;
+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;
+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
+ * 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);
+
+  // 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);
+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 

[GitHub] [hbase] ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta region locations in masters

2019-11-18 Thread GitBox
ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta 
region locations in masters
URL: https://github.com/apache/hbase/pull/830#discussion_r347650440
 
 

 ##
 File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
 ##
 @@ -3051,6 +3053,44 @@ public static ProcedureDescription 
buildProcedureDescription(String signature, S
 return builder.build();
   }
 
+  /**
+   * 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.
+   * @param replicaId replica ID for this region
+   * @return RegionState instance corresponding to the serialized data.
+   * @throws DeserializationException if the data is invalid.
+   */
+  public static RegionState parseMetaRegionStateFrom(final byte[] data, int 
replicaId)
+  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);
+}
+if (serverName == null) {
+  state = RegionState.State.OFFLINE;
 
 Review comment:
   nit: the default state value be `OFFLINE`, then this null-check and 
assignment is not necessary.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [hbase] ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta region locations in masters

2019-11-18 Thread GitBox
ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta 
region locations in masters
URL: https://github.com/apache/hbase/pull/830#discussion_r347655680
 
 

 ##
 File path: 
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() {
 
 Review comment:
   Thank you!


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [hbase] ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta region locations in masters

2019-11-18 Thread GitBox
ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta 
region locations in masters
URL: https://github.com/apache/hbase/pull/830#discussion_r347668152
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java
 ##
 @@ -0,0 +1,211 @@
+/*
+ * 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.exceptions.DeserializationException;
+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;
+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
+ * 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.
 
 Review comment:
   Mind promoting all these juicy member comments to javadoc ?


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [hbase] ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta region locations in masters

2019-11-18 Thread GitBox
ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta 
region locations in masters
URL: https://github.com/apache/hbase/pull/830#discussion_r347655476
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java
 ##
 @@ -0,0 +1,211 @@
+/*
+ * 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.exceptions.DeserializationException;
+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;
+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
+ * meta table znodes. Clients are expected to retry if the meta information is 
stale. This class
 
 Review comment:
   nit: "This class is thread-safe" is ambiguous to me. Maybe you mean "A 
single instance of this class can safely be shared across threads"?


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [hbase] ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta region locations in masters

2019-11-18 Thread GitBox
ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta 
region locations in masters
URL: https://github.com/apache/hbase/pull/830#discussion_r347666141
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java
 ##
 @@ -0,0 +1,211 @@
+/*
+ * 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.exceptions.DeserializationException;
+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;
+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
+ * 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);
+
+  // 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);
+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 

[GitHub] [hbase] ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta region locations in masters

2019-11-18 Thread GitBox
ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta 
region locations in masters
URL: https://github.com/apache/hbase/pull/830#discussion_r347669433
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java
 ##
 @@ -0,0 +1,211 @@
+/*
+ * 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.exceptions.DeserializationException;
+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;
+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
+ * 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);
+
+  // 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);
+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;
 
 Review comment:
   Yeah I think this class becomes a zombie if it's unable to locate the 
replica locations. It would be more resilient as a state machine that can 
transition back to repeat the work done in your `populateInitialMetaLocations` 
method.
   
   Alternatively, it looks like 

[GitHub] [hbase] ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta region locations in masters

2019-11-18 Thread GitBox
ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta 
region locations in masters
URL: https://github.com/apache/hbase/pull/830#discussion_r347658636
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java
 ##
 @@ -0,0 +1,211 @@
+/*
+ * 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.exceptions.DeserializationException;
+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;
+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
+ * 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);
+
+  // 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);
+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
 
 Review comment:
   "... because stand-by masters can potentially start **before** the initial 
znode creation," right? If this code was executed after initial znode creation, 
there'd already be a znode to watch.
   
   Am I misunderstanding something?


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [hbase] ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta region locations in masters

2019-11-18 Thread GitBox
ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta 
region locations in masters
URL: https://github.com/apache/hbase/pull/830#discussion_r347635366
 
 

 ##
 File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
 ##
 @@ -3051,6 +3053,44 @@ public static ProcedureDescription 
buildProcedureDescription(String signature, S
 return builder.build();
   }
 
+  /**
+   * 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.
+   * @param replicaId replica ID for this region
+   * @return RegionState instance corresponding to the serialized data.
+   * @throws DeserializationException if the data is invalid.
+   */
+  public static RegionState parseMetaRegionStateFrom(final byte[] data, int 
replicaId)
 
 Review comment:
   Is this a method that should exist in the non-shaded equivalent?


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [hbase] ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta region locations in masters

2019-11-18 Thread GitBox
ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta 
region locations in masters
URL: https://github.com/apache/hbase/pull/830#discussion_r347656602
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java
 ##
 @@ -0,0 +1,211 @@
+/*
+ * 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.exceptions.DeserializationException;
+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;
+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
+ * 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);
+
+  // 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);
+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;
 
 Review comment:
   What happens now? The znodes don't exist, we have no watchers established. 
Is this instance now a zombie Should it abort the Master instance? Should it 
throw an exception to notify the caller that things didn't go as planned?
   
   If it's not a zombie and this is a perfectly fine state, why did we try to 
establish 

[GitHub] [hbase] ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta region locations in masters

2019-11-18 Thread GitBox
ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta 
region locations in masters
URL: https://github.com/apache/hbase/pull/830#discussion_r347657346
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java
 ##
 @@ -0,0 +1,211 @@
+/*
+ * 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.exceptions.DeserializationException;
+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;
+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
+ * 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);
+
+  // 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);
+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) {
 
 Review comment:
   What thread are we blocking on this synchronous ZK lookup? I think it's the 
main thread for the master.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [hbase] ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta region locations in masters

2019-11-18 Thread GitBox
ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta 
region locations in masters
URL: https://github.com/apache/hbase/pull/830#discussion_r347676342
 
 

 ##
 File path: 
hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java
 ##
 @@ -274,42 +269,17 @@ 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 {
-RegionState.State state = RegionState.State.OPEN;
-ServerName serverName = null;
+  throws KeeperException {
+RegionState regionState = 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);
-  }
+  regionState = ProtobufUtil.parseMetaRegionStateFrom(data, replicaId);
 
 Review comment:
   Ah, I see; the earlier code was moved.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [hbase] ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta region locations in masters

2019-11-18 Thread GitBox
ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta 
region locations in masters
URL: https://github.com/apache/hbase/pull/830#discussion_r347656778
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java
 ##
 @@ -0,0 +1,211 @@
+/*
+ * 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.exceptions.DeserializationException;
+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;
+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
+ * 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);
+
+  // 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);
+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);
 
 Review comment:
   "initial"


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [hbase] ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta region locations in masters

2019-11-18 Thread GitBox
ndimiduk commented on a change in pull request #830: HBASE-23281: Track meta 
region locations in masters
URL: https://github.com/apache/hbase/pull/830#discussion_r347651220
 
 

 ##
 File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
 ##
 @@ -3051,6 +3053,44 @@ public static ProcedureDescription 
buildProcedureDescription(String signature, S
 return builder.build();
   }
 
+  /**
+   * 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.
+   * @param replicaId replica ID for this region
+   * @return RegionState instance corresponding to the serialized data.
+   * @throws DeserializationException if the data is invalid.
+   */
+  public static RegionState parseMetaRegionStateFrom(final byte[] data, int 
replicaId)
 
 Review comment:
   Looks like there's no unit tests covering this new method, nor the 
`parseServerNameFrom` that you wrap. This parsing business seems pretty 
critical to correctness, so it would be nice to see direct unit test coverage.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services