From 423ba4f585a6b1c31abef5b05dda2e86a288412c Mon Sep 17 00:00:00 2001 From: zhangduo Date: Thu, 22 Sep 2016 16:46:28 +0800 Subject: [PATCH] HBASE-15921 Add first AsyncTable impl and create TableImpl based on it --- hbase-client/pom.xml | 4 + .../hbase/client/ConnectionImplementation.java | 6 +- .../org/apache/hadoop/hbase/client/HBaseAdmin.java | 2 +- .../hadoop/hbase/client/ZooKeeperRegistry.java | 2 +- .../hadoop/hbase/client/async/AsyncConnection.java | 42 ++++ .../hbase/client/async/AsyncConnectionImpl.java | 171 +++++++++++++ .../hbase/client/async/AsyncRegionLocator.java | 53 ++++ .../hbase/client/async/AsyncRegionLocatorImpl.java | 73 ++++++ .../hadoop/hbase/client/async/AsyncTable.java | 43 ++++ .../hadoop/hbase/client/async/AsyncTableImpl.java | 118 +++++++++ .../hadoop/hbase/client/async/ClusterRegistry.java | 49 ++++ .../hbase/client/async/ClusterRegistryFactory.java | 43 ++++ .../hbase/client/async/RegionLocatorImpl.java | 60 +++++ .../hbase/client/async/ZKClusterRegistry.java | 272 +++++++++++++++++++++ .../hbase/replication/ReplicationStateZKBase.java | 3 +- .../replication/ReplicationTrackerZKImpl.java | 4 +- .../hbase/zookeeper/MasterAddressTracker.java | 3 +- .../hadoop/hbase/zookeeper/MetaTableLocator.java | 2 +- .../hbase/zookeeper/RecoverableZooKeeper.java | 14 +- .../apache/hadoop/hbase/zookeeper/ZKClusterId.java | 6 +- .../org/apache/hadoop/hbase/zookeeper/ZKUtil.java | 15 +- .../apache/hadoop/hbase/zookeeper/ZNodePaths.java | 108 ++++++++ .../hbase/zookeeper/ZooKeeperNodeTracker.java | 10 +- .../hadoop/hbase/zookeeper/ZooKeeperWatcher.java | 140 ++--------- .../hbase/zookeeper/TestZooKeeperWatcher.java | 24 +- .../apache/hadoop/hbase/util/CollectionUtils.java | 32 ++- .../apache/hadoop/hbase/util/ReflectionUtils.java | 4 +- .../test/IntegrationTestZKAndFSPermissions.java | 2 +- .../hbase/rsgroup/RSGroupInfoManagerImpl.java | 6 +- .../hbase/rsgroup/VerifyingRSGroupAdminClient.java | 2 +- .../apache/hadoop/hbase/ZKNamespaceManager.java | 2 +- .../hbase/backup/example/ZKTableArchiveClient.java | 2 +- .../ZKSplitLogManagerCoordination.java | 40 +-- .../coordination/ZkSplitLogWorkerCoordination.java | 22 +- .../hadoop/hbase/master/ActiveMasterManager.java | 4 +- .../org/apache/hadoop/hbase/master/HMaster.java | 8 +- .../hadoop/hbase/master/MasterMetaBootstrap.java | 2 +- .../apache/hadoop/hbase/master/ServerManager.java | 2 +- .../hadoop/hbase/master/TableLockManager.java | 14 +- .../hadoop/hbase/mob/mapreduce/SweepJob.java | 3 +- .../hadoop/hbase/procedure/ZKProcedureUtil.java | 2 +- .../hadoop/hbase/regionserver/HRegionServer.java | 6 +- .../replication/HBaseReplicationEndpoint.java | 4 +- .../hbase/security/access/ZKPermissionWatcher.java | 8 +- .../hbase/security/token/ZKSecretWatcher.java | 2 +- .../visibility/ZKVisibilityLabelWatcher.java | 4 +- .../org/apache/hadoop/hbase/util/HBaseFsck.java | 3 +- .../apache/hadoop/hbase/util/ZKDataMigrator.java | 4 +- .../hadoop/hbase/util/hbck/ReplicationChecker.java | 2 +- .../hbase/zookeeper/ClusterStatusTracker.java | 10 +- .../hbase/zookeeper/DrainingServerTracker.java | 8 +- .../hbase/zookeeper/LoadBalancerTracker.java | 8 +- .../zookeeper/MasterMaintenanceModeTracker.java | 4 +- .../hbase/zookeeper/RecoveringRegionWatcher.java | 4 +- .../hbase/zookeeper/RegionServerTracker.java | 10 +- .../apache/hadoop/hbase/zookeeper/ZKSplitLog.java | 14 +- .../hadoop/hbase/client/async/TestAsyncGet.java | 77 ++++++ .../hbase/client/async/TestZKClusterRegistry.java | 78 ++++++ .../hbase/master/TestActiveMasterManager.java | 4 +- .../hbase/master/TestDistributedLogSplitting.java | 16 +- .../hadoop/hbase/master/TestMasterNoCluster.java | 2 +- .../hadoop/hbase/master/TestMasterWalManager.java | 9 +- .../hbase/master/TestMetaShutdownHandler.java | 2 +- .../hadoop/hbase/master/TestSplitLogManager.java | 18 +- .../hadoop/hbase/master/TestTableLockManager.java | 5 +- .../hadoop/hbase/master/TestTableStateManager.java | 2 +- .../hbase/mob/mapreduce/TestMobSweepMapper.java | 2 +- .../hbase/mob/mapreduce/TestMobSweepReducer.java | 2 +- .../regionserver/TestMasterAddressTracker.java | 2 +- .../regionserver/TestRegionServerHostname.java | 2 +- .../hbase/regionserver/TestSplitLogWorker.java | 26 +- .../replication/TestReplicationStateHBaseImpl.java | 2 +- .../replication/TestReplicationStateZKImpl.java | 4 +- .../replication/TestReplicationTrackerZKImpl.java | 16 +- .../apache/hadoop/hbase/zookeeper/TestZKMulti.java | 40 +-- .../hbase/zookeeper/TestZooKeeperNodeTracker.java | 4 +- .../lock/TestZKInterProcessReadWriteLock.java | 6 +- pom.xml | 48 ++-- 78 files changed, 1506 insertions(+), 366 deletions(-) create mode 100644 hbase-client/src/main/java/org/apache/hadoop/hbase/client/async/AsyncConnection.java create mode 100644 hbase-client/src/main/java/org/apache/hadoop/hbase/client/async/AsyncConnectionImpl.java create mode 100644 hbase-client/src/main/java/org/apache/hadoop/hbase/client/async/AsyncRegionLocator.java create mode 100644 hbase-client/src/main/java/org/apache/hadoop/hbase/client/async/AsyncRegionLocatorImpl.java create mode 100644 hbase-client/src/main/java/org/apache/hadoop/hbase/client/async/AsyncTable.java create mode 100644 hbase-client/src/main/java/org/apache/hadoop/hbase/client/async/AsyncTableImpl.java create mode 100644 hbase-client/src/main/java/org/apache/hadoop/hbase/client/async/ClusterRegistry.java create mode 100644 hbase-client/src/main/java/org/apache/hadoop/hbase/client/async/ClusterRegistryFactory.java create mode 100644 hbase-client/src/main/java/org/apache/hadoop/hbase/client/async/RegionLocatorImpl.java create mode 100644 hbase-client/src/main/java/org/apache/hadoop/hbase/client/async/ZKClusterRegistry.java create mode 100644 hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZNodePaths.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/client/async/TestAsyncGet.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/client/async/TestZKClusterRegistry.java diff --git a/hbase-client/pom.xml b/hbase-client/pom.xml index b78f198..f2af987 100644 --- a/hbase-client/pom.xml +++ b/hbase-client/pom.xml @@ -199,6 +199,10 @@ mockito-all test + + org.apache.curator + curator-framework + diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java index 38178b4..6920393 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java @@ -104,7 +104,7 @@ import com.google.protobuf.ServiceException; value="AT_OPERATION_SEQUENCE_ON_CONCURRENT_ABSTRACTION", justification="Access to the conncurrent hash map is under a lock so should be fine.") @InterfaceAudience.Private -class ConnectionImplementation implements ClusterConnection, Closeable { +public class ConnectionImplementation implements ClusterConnection, Closeable { public static final String RETRIES_BY_SERVER_KEY = "hbase.client.retries.by.server"; private static final Log LOG = LogFactory.getLog(ConnectionImplementation.class); private static final String CLIENT_NONCES_ENABLED_KEY = "hbase.client.nonces.enabled"; @@ -457,8 +457,8 @@ class ConnectionImplementation implements ClusterConnection, Closeable { throws MasterNotRunningException { String errorMsg; try { - if (ZKUtil.checkExists(zkw, zkw.baseZNode) == -1) { - errorMsg = "The node " + zkw.baseZNode+" is not in ZooKeeper. " + if (ZKUtil.checkExists(zkw, zkw.znodePaths.baseZNode) == -1) { + errorMsg = "The node " + zkw.znodePaths.baseZNode+" is not in ZooKeeper. " + "It should have been written by the master. " + "Check the value configured in 'zookeeper.znode.parent'. " + "There could be a mismatch with the one configured in the master."; diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java index 3b41755..01960ab 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java @@ -2107,7 +2107,7 @@ public class HBaseAdmin implements Admin { ZooKeeperKeepAliveConnection zkw = ((ConnectionImplementation) connection). getKeepAliveZooKeeperWatcher();) { // This is NASTY. FIX!!!! Dependent on internal implementation! TODO - zkw.getRecoverableZooKeeper().getZooKeeper().exists(zkw.baseZNode, false); + zkw.getRecoverableZooKeeper().getZooKeeper().exists(zkw.znodePaths.baseZNode, false); connection.isMasterRunning(); } catch (IOException e) { throw new ZooKeeperConnectionException("Can't connect to ZooKeeper", e); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZooKeeperRegistry.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZooKeeperRegistry.java index 83f828c..9fca027 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZooKeeperRegistry.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZooKeeperRegistry.java @@ -121,7 +121,7 @@ class ZooKeeperRegistry implements Registry { try { // We go to zk rather than to master to get count of regions to avoid // HTable having a Master dependency. See HBase-2828 - return ZKUtil.getNumberOfChildren(zkw, zkw.rsZNode); + return ZKUtil.getNumberOfChildren(zkw, zkw.znodePaths.rsZNode); } catch (KeeperException ke) { throw new IOException("Unexpected ZooKeeper exception", ke); } finally { diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/async/AsyncConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/async/AsyncConnection.java new file mode 100644 index 0000000..fd54053 --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/async/AsyncConnection.java @@ -0,0 +1,42 @@ +/** + * 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.async; + +import java.io.Closeable; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceStability; + +/** + * + */ +@InterfaceAudience.Private +@InterfaceStability.Evolving +public interface AsyncConnection extends Closeable { + + Configuration getConfiguration(); + + AsyncRegionLocator getRegionLocator(TableName tableName); + + AsyncTable getTable(TableName tableName); + + @Override + void close(); +} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/async/AsyncConnectionImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/async/AsyncConnectionImpl.java new file mode 100644 index 0000000..e52dd6f --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/async/AsyncConnectionImpl.java @@ -0,0 +1,171 @@ +/** + * 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.async; + +import io.netty.util.HashedWheelTimer; +import io.netty.util.Timeout; +import io.netty.util.TimerTask; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.net.InetAddress; +import java.net.UnknownHostException; +import java.util.Optional; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.TimeUnit; + +import org.apache.commons.io.IOUtils; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.ipc.RpcClient; +import org.apache.hadoop.hbase.ipc.RpcClientFactory; +import org.apache.hadoop.hbase.ipc.RpcControllerFactory; +import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService; +import org.apache.hadoop.hbase.security.User; +import org.apache.hadoop.hbase.util.CollectionUtils; +import org.apache.hadoop.hbase.util.Threads; + +/** + * + */ +@InterfaceAudience.Private +class AsyncConnectionImpl implements AsyncConnection { + + private static final Log LOG = LogFactory.getLog(AsyncConnectionImpl.class); + + private static final HashedWheelTimer RETRY_TIMER = new HashedWheelTimer( + Threads.newDaemonThreadFactory("Async-Client-Retry-Timer"), 10, TimeUnit.MILLISECONDS); + + private static final String RESOLVE_HOSTNAME_ON_FAIL_KEY = "hbase.resolve.hostnames.on.failure"; + + private final Configuration conf; + + private final User user; + + private final ClusterRegistry registry; + + private final String clusterId; + + private final int rpcTimeout; + + private final RpcClient rpcClient; + + final RpcControllerFactory rpcControllerFactory; + + private final boolean hostnameCanChange; + + final RegionLocatorImpl locator; + + public AsyncConnectionImpl(Configuration conf, User user) throws IOException { + this.conf = conf; + this.user = user; + + this.locator = new RegionLocatorImpl(conf); + + // action below will not throw exception so no need to catch and close. + this.registry = ClusterRegistryFactory.getRegistry(conf); + this.clusterId = Optional.ofNullable(registry.getClusterId()).orElseGet(() -> { + if (LOG.isDebugEnabled()) { + LOG.debug("cluster id came back null, using default " + HConstants.CLUSTER_ID_DEFAULT); + } + return HConstants.CLUSTER_ID_DEFAULT; + }); + this.rpcClient = RpcClientFactory.createClient(conf, clusterId); + this.rpcControllerFactory = RpcControllerFactory.instantiate(conf); + this.hostnameCanChange = conf.getBoolean(RESOLVE_HOSTNAME_ON_FAIL_KEY, true); + this.rpcTimeout = conf.getInt(HConstants.HBASE_RPC_TIMEOUT_KEY, + HConstants.DEFAULT_HBASE_RPC_TIMEOUT); + } + + @Override + public Configuration getConfiguration() { + return conf; + } + + @Override + public void close() { + IOUtils.closeQuietly(locator); + IOUtils.closeQuietly(rpcClient); + IOUtils.closeQuietly(registry); + } + + @Override + public AsyncRegionLocator getRegionLocator(TableName tableName) { + return new AsyncRegionLocatorImpl(tableName, locator); + } + + void retryAfter(long delay, TimeUnit unit, Runnable action) { + RETRY_TIMER.newTimeout(new TimerTask() { + + @Override + public void run(Timeout timeout) throws Exception { + action.run(); + } + }, delay, unit); + } + + private final ConcurrentMap rsStubs = new ConcurrentHashMap<>(); + + private String getStubKey(String serviceName, ServerName serverName) { + // Sometimes, servers go down and they come back up with the same hostname but a different + // IP address. Force a resolution of the rsHostname by trying to instantiate an + // InetSocketAddress, and this way we will rightfully get a new stubKey. + // Also, include the hostname in the key so as to take care of those cases where the + // DNS name is different but IP address remains the same. + String hostname = serverName.getHostname(); + int port = serverName.getPort(); + if (hostnameCanChange) { + try { + InetAddress ip = InetAddress.getByName(hostname); + return serviceName + "@" + hostname + "-" + ip.getHostAddress() + ":" + port; + } catch (UnknownHostException e) { + LOG.warn("Can not resolve " + hostname + ", please check your network", e); + } + } + return serviceName + "@" + hostname + ":" + port; + } + + private ClientService.Interface createRegionServerStub(ServerName serverName) { + try { + return ClientService.newStub(rpcClient.createRpcChannel(serverName, user, rpcTimeout)); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + ClientService.Interface getRegionServerStub(ServerName serverName) throws IOException { + try { + return CollectionUtils.computeIfAbsent(rsStubs, + getStubKey(ClientService.Interface.class.getSimpleName(), serverName), + () -> createRegionServerStub(serverName)); + } catch (UncheckedIOException e) { + throw e.getCause(); + } + } + + @Override + public AsyncTable getTable(TableName tableName) { + return new AsyncTableImpl(this, tableName); + } +} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/async/AsyncRegionLocator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/async/AsyncRegionLocator.java new file mode 100644 index 0000000..6a94828 --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/async/AsyncRegionLocator.java @@ -0,0 +1,53 @@ +/** + * 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.async; + +import java.util.List; +import java.util.concurrent.CompletableFuture; + +import org.apache.hadoop.hbase.HRegionLocation; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceStability; +import org.apache.hadoop.hbase.util.Pair; + +/** + * The asynchronous version of RegionLocator. + *

+ * Declared as InterfaceAudience.Private because it is only an experimental API yet. + */ +@InterfaceAudience.Private +@InterfaceStability.Evolving +public interface AsyncRegionLocator { + + TableName getName(); + + default CompletableFuture getRegionLocation(byte[] row) { + return getRegionLocation(row, false); + } + + CompletableFuture getRegionLocation(byte[] row, boolean reload); + + CompletableFuture> getAllRegionLocations(); + + CompletableFuture getStartKeys(); + + CompletableFuture getEndKeys(); + + CompletableFuture> getStartEndKeys(); +} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/async/AsyncRegionLocatorImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/async/AsyncRegionLocatorImpl.java new file mode 100644 index 0000000..3eb7e80 --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/async/AsyncRegionLocatorImpl.java @@ -0,0 +1,73 @@ +/** + * 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.async; + +import java.util.List; +import java.util.concurrent.CompletableFuture; + +import org.apache.hadoop.hbase.HRegionLocation; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.util.Pair; + +/** + * + */ +@InterfaceAudience.Private +public class AsyncRegionLocatorImpl implements AsyncRegionLocator { + + private final TableName tableName; + + private final RegionLocatorImpl locator; + + public AsyncRegionLocatorImpl(TableName tableName, RegionLocatorImpl locator) { + this.tableName = tableName; + this.locator = locator; + } + + @Override + public TableName getName() { + return tableName; + } + + @Override + public CompletableFuture getRegionLocation(byte[] row, boolean reload) { + return locator.getRegionLocation(tableName, row, reload); + } + + @Override + public CompletableFuture> getAllRegionLocations() { + throw new UnsupportedOperationException(); + } + + @Override + public CompletableFuture getStartKeys() { + throw new UnsupportedOperationException(); + } + + @Override + public CompletableFuture getEndKeys() { + throw new UnsupportedOperationException(); + } + + @Override + public CompletableFuture> getStartEndKeys() { + throw new UnsupportedOperationException(); + } + +} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/async/AsyncTable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/async/AsyncTable.java new file mode 100644 index 0000000..debb36a --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/async/AsyncTable.java @@ -0,0 +1,43 @@ +/** + * 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.async; + +import java.util.concurrent.CompletableFuture; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceStability; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.Result; + +/** + * The asynchronous version of Table. + *

+ * Declared as InterfaceAudience.Private because it is only an experimental API yet. + */ +@InterfaceAudience.Private +@InterfaceStability.Evolving +public interface AsyncTable { + + TableName getName(); + + Configuration getConfiguration(); + + CompletableFuture get(Get get); +} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/async/AsyncTableImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/async/AsyncTableImpl.java new file mode 100644 index 0000000..c23da73 --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/async/AsyncTableImpl.java @@ -0,0 +1,118 @@ +/** + * 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.async; + +import com.google.protobuf.RpcCallback; + +import java.io.IOException; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.ipc.HBaseRpcController; +import org.apache.hadoop.hbase.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.protobuf.RequestConverter; +import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService; +import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetRequest; +import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetResponse; + +/** + * + */ +@InterfaceAudience.Private +public class AsyncTableImpl implements AsyncTable { + + private static final Log LOG = LogFactory.getLog(AsyncTableImpl.class); + + private final AsyncConnectionImpl conn; + + private final TableName tableName; + + public AsyncTableImpl(AsyncConnectionImpl conn, TableName tableName) { + this.conn = conn; + this.tableName = tableName; + } + + @Override + public TableName getName() { + return tableName; + } + + @Override + public Configuration getConfiguration() { + return conn.getConfiguration(); + } + + private void get(CompletableFuture future, Get get, int retryCount) { + conn.locator.getRegionLocation(tableName, get.getRow(), false).whenComplete((loc, error) -> { + if (error != null) { + LOG.warn("Get region location failed", error); + // inject backoff calculation here, and quit if we are timed out. + conn.retryAfter(100 * (retryCount + 1), TimeUnit.MILLISECONDS, + () -> get(future, get, retryCount + 1)); + return; + } + + try { + ClientService.Interface stub = conn.getRegionServerStub(loc.getServerName()); + GetRequest req = RequestConverter.buildGetRequest(loc.getRegionInfo().getRegionName(), get); + HBaseRpcController controller = conn.rpcControllerFactory.newController(); + stub.get(controller, req, new RpcCallback() { + + @Override + public void run(GetResponse resp) { + if (controller.failed()) { + LOG.warn("request failed", controller.getFailed()); + // inject backoff calculation here, and quit if we are timed out. + conn.retryAfter(100 * (retryCount + 1), TimeUnit.MILLISECONDS, + () -> get(future, get, retryCount + 1)); + return; + } + try { + future.complete(ProtobufUtil.toResult(resp.getResult(), controller.cellScanner())); + } catch (IOException e) { + LOG.warn("convert result failed", e); + // inject backoff calculation here, and quit if we are timed out. + conn.retryAfter(100 * (retryCount + 1), TimeUnit.MILLISECONDS, + () -> get(future, get, retryCount + 1)); + } + } + }); + } catch (IOException e) { + LOG.warn("send request failed", e); + // inject backoff calculation here, and quit if we are timed out. + conn.retryAfter(100 * (retryCount + 1), TimeUnit.MILLISECONDS, + () -> get(future, get, retryCount + 1)); + return; + } + }); + } + + @Override + public CompletableFuture get(Get get) { + CompletableFuture future = new CompletableFuture<>(); + get(future, get, 0); + return future; + } +} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/async/ClusterRegistry.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/async/ClusterRegistry.java new file mode 100644 index 0000000..1058c34 --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/async/ClusterRegistry.java @@ -0,0 +1,49 @@ +/** + * 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.async; + +import java.io.Closeable; + +import org.apache.hadoop.hbase.RegionLocations; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.classification.InterfaceAudience; + +/** + * Implementations hold cluster information such as this cluster's id, location of hbase:meta, etc. + * Internal use only. + */ +@InterfaceAudience.Private +interface ClusterRegistry extends Closeable { + + RegionLocations getMetaRegionLocation(); + + /** + * Should only be called once, the upper layer should store this value somewhere as it will not + * change any more. + */ + String getClusterId(); + + int getCurrentNrHRS(); + + ServerName getMasterAddress(); + + int getMasterInfoPort(); + + @Override + void close(); +} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/async/ClusterRegistryFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/async/ClusterRegistryFactory.java new file mode 100644 index 0000000..33765b3 --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/async/ClusterRegistryFactory.java @@ -0,0 +1,43 @@ +/** + * 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.async; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.util.ReflectionUtils; + +/** + * Get instance of configured Registry. + */ +@InterfaceAudience.Private +final class ClusterRegistryFactory { + + static final String REGISTRY_IMPL_CONF_KEY = "hbase.client.registry.impl"; + + private ClusterRegistryFactory() { + } + + /** + * @return The cluster registry implementation to use. + */ + static ClusterRegistry getRegistry(Configuration conf) { + Class clazz = conf.getClass(REGISTRY_IMPL_CONF_KEY, + ZKClusterRegistry.class, ClusterRegistry.class); + return ReflectionUtils.newInstance(clazz, conf); + } +} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/async/RegionLocatorImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/async/RegionLocatorImpl.java new file mode 100644 index 0000000..adbf5cc --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/async/RegionLocatorImpl.java @@ -0,0 +1,60 @@ +/** + * 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.async; + +import java.io.Closeable; +import java.io.IOException; +import java.util.concurrent.CompletableFuture; + +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HRegionLocation; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.ConnectionImplementation; + +/** + * TODO: reimplement using aync connection when the scan logic is ready. The current implementation + * is based on the blocking client. + */ +@InterfaceAudience.Private +class RegionLocatorImpl implements Closeable { + + private final ConnectionImplementation conn; + + RegionLocatorImpl(Configuration conf) throws IOException { + conn = (ConnectionImplementation) ConnectionFactory.createConnection(conf); + } + + CompletableFuture getRegionLocation(TableName tableName, byte[] row, + boolean reload) { + CompletableFuture future = new CompletableFuture<>(); + try { + future.complete(conn.getRegionLocation(tableName, row, reload)); + } catch (IOException e) { + future.completeExceptionally(e); + } + return future; + } + + @Override + public void close() { + IOUtils.closeQuietly(conn); + } +} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/async/ZKClusterRegistry.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/async/ZKClusterRegistry.java new file mode 100644 index 0000000..92859e0 --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/async/ZKClusterRegistry.java @@ -0,0 +1,272 @@ +/** + * 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.async; + +import static java.util.stream.Collectors.collectingAndThen; +import static java.util.stream.Collectors.toList; +import static org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.removeMetaData; + +import com.google.protobuf.InvalidProtocolBufferException; + +import java.util.ArrayDeque; +import java.util.Arrays; +import java.util.Collections; +import java.util.Deque; +import java.util.List; + +import org.apache.commons.io.IOUtils; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.curator.framework.CuratorFramework; +import org.apache.curator.framework.CuratorFrameworkFactory; +import org.apache.curator.framework.recipes.cache.ChildData; +import org.apache.curator.framework.recipes.cache.NodeCache; +import org.apache.curator.framework.recipes.cache.NodeCacheListener; +import org.apache.curator.framework.recipes.cache.PathChildrenCache; +import org.apache.curator.framework.recipes.cache.PathChildrenCacheEvent; +import org.apache.curator.framework.recipes.cache.PathChildrenCacheListener; +import org.apache.curator.retry.RetryNTimes; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.ClusterId; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.HRegionLocation; +import org.apache.hadoop.hbase.RegionLocations; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.client.RegionReplicaUtil; +import org.apache.hadoop.hbase.master.RegionState; +import org.apache.hadoop.hbase.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos; +import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos; +import org.apache.hadoop.hbase.util.Threads; +import org.apache.hadoop.hbase.zookeeper.ZKConfig; +import org.apache.hadoop.hbase.zookeeper.ZNodePaths; + +/** + * Cache the cluster registry data in memory and use zk watcher to update. + *

+ * The constructor is a blocking operation as we want to fetch cluster id immediately from zk. All + * other methods will return the cached data in memory immediately. + */ +@InterfaceAudience.Private +class ZKClusterRegistry implements ClusterRegistry { + + private static final Log LOG = LogFactory.getLog(ZKClusterRegistry.class); + + private final CuratorFramework zk; + + private final ZNodePaths znodePaths; + + private final List metaRegionLocationsWatcher; + + private volatile RegionLocations metaRegionLocations; + + private final PathChildrenCache currentNrHRSWatcher; + + private volatile int currentNrHRS; + + private final NodeCache masterAddressWatcher; + + private volatile ServerName masterAddress; + + private volatile int masterInfoPort; + + private RegionState createRegionState(int replicaId, byte[] data) + throws InvalidProtocolBufferException { + if (data == null || data.length == 0) { + return new RegionState( + RegionReplicaUtil.getRegionInfoForReplica(HRegionInfo.FIRST_META_REGIONINFO, replicaId), + RegionState.State.OFFLINE); + } + data = removeMetaData(data); + int prefixLen = ProtobufUtil.lengthOfPBMagic(); + ZooKeeperProtos.MetaRegionServer mrs = ZooKeeperProtos.MetaRegionServer.PARSER.parseFrom(data, + prefixLen, data.length - prefixLen); + HBaseProtos.ServerName sn = mrs.getServer(); + ServerName serverName = ServerName.valueOf(sn.getHostName(), sn.getPort(), sn.getStartCode()); + RegionState.State state; + if (mrs.hasState()) { + state = RegionState.State.convert(mrs.getState()); + } else { + state = RegionState.State.OPEN; + } + return new RegionState( + RegionReplicaUtil.getRegionInfoForReplica(HRegionInfo.FIRST_META_REGIONINFO, replicaId), + state, serverName); + } + + // All listeners of NodeCaches will be called inside the same thread so no need to lock + private void updateMetaRegionLocation(int replicaId, ChildData childData) { + if (childData == null) { + return; + } + RegionState state; + try { + state = createRegionState(replicaId, childData.getData()); + } catch (InvalidProtocolBufferException e) { + LOG.warn("Failed to parse meta region state", e); + return; + } + if (state.getState() != RegionState.State.OPEN || state.getServerName() == null) { + return; + } + RegionLocations oldLocs = metaRegionLocations; + Deque newLocs = new ArrayDeque<>(); + if (oldLocs != null) { + Arrays.stream(oldLocs.getRegionLocations()) + .filter(loc -> loc.getRegionInfo().getReplicaId() != replicaId) + .forEach(loc -> newLocs.add(loc)); + } + HRegionLocation newLoc = new HRegionLocation( + RegionReplicaUtil.getRegionInfoForReplica(HRegionInfo.FIRST_META_REGIONINFO, replicaId), + state.getServerName()); + if (replicaId == HRegionInfo.DEFAULT_REPLICA_ID) { + newLocs.addFirst(newLoc); + } else { + newLocs.addLast(newLoc); + } + metaRegionLocations = new RegionLocations(newLocs); + } + + private void updateCurrentNrHRS() { + currentNrHRS = currentNrHRSWatcher.getCurrentData().size(); + } + + private void updateMasterAddressAndInfoPort() { + ChildData childData = masterAddressWatcher.getCurrentData(); + if (childData == null) { + return; + } + byte[] data = childData.getData(); + if (data == null || data.length == 0) { + return; + } + data = removeMetaData(data); + int prefixLen = ProtobufUtil.lengthOfPBMagic(); + try { + ZooKeeperProtos.Master masterProto = ZooKeeperProtos.Master.PARSER.parseFrom(data, prefixLen, + data.length - prefixLen); + HBaseProtos.ServerName snProto = masterProto.getMaster(); + masterAddress = ServerName.valueOf(snProto.getHostName(), snProto.getPort(), + snProto.getStartCode()); + masterInfoPort = masterProto.getInfoPort(); + } catch (InvalidProtocolBufferException e) { + LOG.warn("Failed to parse master address", e); + } + } + + ZKClusterRegistry(Configuration conf) { + this.znodePaths = new ZNodePaths(conf); + int zkSessionTimeout = conf.getInt(HConstants.ZK_SESSION_TIMEOUT, + HConstants.DEFAULT_ZK_SESSION_TIMEOUT); + int zkRetry = conf.getInt("zookeeper.recovery.retry", 3); + int zkRetryIntervalMs = conf.getInt("zookeeper.recovery.retry.intervalmill", 1000); + this.zk = CuratorFrameworkFactory.builder() + .connectString(ZKConfig.getZKQuorumServersString(conf)).sessionTimeoutMs(zkSessionTimeout) + .retryPolicy(new RetryNTimes(zkRetry, zkRetryIntervalMs)) + .threadFactory( + Threads.newDaemonThreadFactory(String.format("ZKClusterRegistry-0x%08x", hashCode()))) + .build(); + this.zk.start(); + + metaRegionLocationsWatcher = znodePaths.metaReplicaZNodes.entrySet().stream().map(entry -> { + NodeCache nc = new NodeCache(zk, entry.getValue()); + nc.getListenable().addListener(new NodeCacheListener() { + + @Override + public void nodeChanged() { + updateMetaRegionLocation(entry.getKey().intValue(), nc.getCurrentData()); + } + }); + return nc; + }).collect(collectingAndThen(toList(), Collections::unmodifiableList)); + + currentNrHRSWatcher = new PathChildrenCache(zk, znodePaths.rsZNode, false, + Threads.newDaemonThreadFactory(String.format("ZKClusterRegistry-0x%08x-RS", hashCode()))); + currentNrHRSWatcher.getListenable().addListener(new PathChildrenCacheListener() { + + @Override + public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) { + updateCurrentNrHRS(); + } + }); + + masterAddressWatcher = new NodeCache(zk, znodePaths.masterAddressZNode); + masterAddressWatcher.getListenable().addListener(new NodeCacheListener() { + + @Override + public void nodeChanged() { + updateMasterAddressAndInfoPort(); + } + }); + + try { + for (NodeCache watcher : metaRegionLocationsWatcher) { + watcher.start(); + } + currentNrHRSWatcher.start(); + masterAddressWatcher.start(); + } catch (Exception e) { + // normal start will not throw any exception. + throw new AssertionError(e); + } + } + + @Override + public RegionLocations getMetaRegionLocation() { + return metaRegionLocations; + } + + @Override + public String getClusterId() { + try { + byte[] data = zk.getData().forPath(znodePaths.clusterIdZNode); + if (data == null || data.length == 0) { + return null; + } + return ClusterId.parseFrom(removeMetaData(data)).toString(); + } catch (Exception e) { + LOG.warn("failed to get cluster id", e); + return null; + } + } + + @Override + public int getCurrentNrHRS() { + return currentNrHRS; + } + + @Override + public ServerName getMasterAddress() { + return masterAddress; + } + + @Override + public int getMasterInfoPort() { + return masterInfoPort; + } + + @Override + public void close() { + metaRegionLocationsWatcher.forEach(IOUtils::closeQuietly); + IOUtils.closeQuietly(currentNrHRSWatcher); + IOUtils.closeQuietly(masterAddressWatcher); + IOUtils.closeQuietly(zk); + } +} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java index 79853a8..f727ce8 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java @@ -83,7 +83,8 @@ public abstract class ReplicationStateZKBase { this.peerStateNodeName = conf.get("zookeeper.znode.replication.peers.state", "peer-state"); this.tableCFsNodeName = conf.get("zookeeper.znode.replication.peers.tableCFs", "tableCFs"); this.ourClusterKey = ZKConfig.getZooKeeperClusterKey(this.conf); - this.replicationZNode = ZKUtil.joinZNode(this.zookeeper.baseZNode, replicationZNodeName); + this.replicationZNode = ZKUtil.joinZNode(this.zookeeper.znodePaths.baseZNode, + replicationZNodeName); this.peersZNode = ZKUtil.joinZNode(replicationZNode, peersZNodeName); this.queuesZNode = ZKUtil.joinZNode(replicationZNode, queuesZNodeName); this.hfileRefsZNode = ZKUtil.joinZNode(replicationZNode, hfileRefsZNodeName); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationTrackerZKImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationTrackerZKImpl.java index 6e4983e..9d182dc 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationTrackerZKImpl.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationTrackerZKImpl.java @@ -136,7 +136,7 @@ public class ReplicationTrackerZKImpl extends ReplicationStateZKBase implements } private boolean refreshListIfRightPath(String path) { - if (!path.startsWith(this.watcher.rsZNode)) { + if (!path.startsWith(this.watcher.znodePaths.rsZNode)) { return false; } return refreshOtherRegionServersList(); @@ -242,7 +242,7 @@ public class ReplicationTrackerZKImpl extends ReplicationStateZKBase implements private List getRegisteredRegionServers() { List result = null; try { - result = ZKUtil.listChildrenAndWatchThem(this.zookeeper, this.zookeeper.rsZNode); + result = ZKUtil.listChildrenAndWatchThem(this.zookeeper, this.zookeeper.znodePaths.rsZNode); } catch (KeeperException e) { this.abortable.abort("Get list of registered region servers", e); } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MasterAddressTracker.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MasterAddressTracker.java index 6f4859a..1d97a65 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MasterAddressTracker.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MasterAddressTracker.java @@ -101,7 +101,8 @@ public class MasterAddressTracker extends ZooKeeperNodeTracker { * @return info port or 0 if timed out or exceptions */ public int getBackupMasterInfoPort(final ServerName sn) { - String backupZNode = ZKUtil.joinZNode(watcher.backupMasterAddressesZNode, sn.toString()); + String backupZNode = ZKUtil.joinZNode(watcher.znodePaths.backupMasterAddressesZNode, + sn.toString()); try { byte[] data = ZKUtil.getData(watcher, backupZNode); final ZooKeeperProtos.Master backup = parse(data); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java index 359617a..31fc5e3 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java @@ -204,7 +204,7 @@ public class MetaTableLocator { public ServerName waitMetaRegionLocation(ZooKeeperWatcher zkw, int replicaId, long timeout) throws InterruptedException, NotAllMetaRegionsOnlineException { try { - if (ZKUtil.checkExists(zkw, zkw.baseZNode) == -1) { + if (ZKUtil.checkExists(zkw, zkw.znodePaths.baseZNode) == -1) { String errorMsg = "Check the value configured in 'zookeeper.znode.parent'. " + "There could be a mismatch with the one configured in the master."; LOG.error(errorMsg); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java index 371279e..8fe78b5 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java @@ -352,7 +352,7 @@ public class RecoverableZooKeeper { while (true) { try { byte[] revData = checkZk().getData(path, watcher, stat); - return this.removeMetaData(revData); + return removeMetaData(revData); } catch (KeeperException e) { switch (e.code()) { case CONNECTIONLOSS: @@ -384,7 +384,7 @@ public class RecoverableZooKeeper { while (true) { try { byte[] revData = checkZk().getData(path, watch, stat); - return this.removeMetaData(revData); + return removeMetaData(revData); } catch (KeeperException e) { switch (e.code()) { case CONNECTIONLOSS: @@ -707,19 +707,19 @@ public class RecoverableZooKeeper { return null; } - public byte[] removeMetaData(byte[] data) { - if(data == null || data.length == 0) { + public static byte[] removeMetaData(byte[] data) { + if (data == null || data.length == 0) { return data; } // check the magic data; to be backward compatible byte magic = data[0]; - if(magic != MAGIC) { + if (magic != MAGIC) { return data; } int idLength = Bytes.toInt(data, ID_LENGTH_OFFSET); - int dataLength = data.length-MAGIC_SIZE-ID_LENGTH_SIZE-idLength; - int dataOffset = MAGIC_SIZE+ID_LENGTH_SIZE+idLength; + int dataLength = data.length - MAGIC_SIZE - ID_LENGTH_SIZE - idLength; + int dataOffset = MAGIC_SIZE + ID_LENGTH_SIZE + idLength; byte[] newData = new byte[dataLength]; System.arraycopy(data, dataOffset, newData, 0, dataLength); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKClusterId.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKClusterId.java index b603ab2..3679602 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKClusterId.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKClusterId.java @@ -62,10 +62,10 @@ public class ZKClusterId { public static String readClusterIdZNode(ZooKeeperWatcher watcher) throws KeeperException { - if (ZKUtil.checkExists(watcher, watcher.clusterIdZNode) != -1) { + if (ZKUtil.checkExists(watcher, watcher.znodePaths.clusterIdZNode) != -1) { byte [] data; try { - data = ZKUtil.getData(watcher, watcher.clusterIdZNode); + data = ZKUtil.getData(watcher, watcher.znodePaths.clusterIdZNode); } catch (InterruptedException e) { Thread.currentThread().interrupt(); return null; @@ -83,7 +83,7 @@ public class ZKClusterId { public static void setClusterId(ZooKeeperWatcher watcher, ClusterId id) throws KeeperException { - ZKUtil.createSetData(watcher, watcher.clusterIdZNode, id.toByteArray()); + ZKUtil.createSetData(watcher, watcher.znodePaths.clusterIdZNode, id.toByteArray()); } /** diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java index 0896725..703db14 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java @@ -903,7 +903,7 @@ public class ZKUtil { public static ArrayList createACL(ZooKeeperWatcher zkw, String node, boolean isSecureZooKeeper) { - if (!node.startsWith(zkw.baseZNode)) { + if (!node.startsWith(zkw.znodePaths.baseZNode)) { return Ids.OPEN_ACL_UNSAFE; } if (isSecureZooKeeper) { @@ -1703,7 +1703,7 @@ public class ZKUtil { public static String dump(ZooKeeperWatcher zkw) { StringBuilder sb = new StringBuilder(); try { - sb.append("HBase is rooted at ").append(zkw.baseZNode); + sb.append("HBase is rooted at ").append(zkw.znodePaths.baseZNode); sb.append("\nActive master address: "); try { sb.append(MasterAddressTracker.getMasterAddress(zkw)); @@ -1711,8 +1711,7 @@ public class ZKUtil { sb.append("<>"); } sb.append("\nBackup master addresses:"); - for (String child : listChildrenNoWatch(zkw, - zkw.backupMasterAddressesZNode)) { + for (String child : listChildrenNoWatch(zkw, zkw.znodePaths.backupMasterAddressesZNode)) { sb.append("\n ").append(child); } sb.append("\nRegion server holding hbase:meta: " @@ -1725,7 +1724,7 @@ public class ZKUtil { + new MetaTableLocator().getMetaRegionLocation(zkw, i)); } sb.append("\nRegion servers:"); - for (String child : listChildrenNoWatch(zkw, zkw.rsZNode)) { + for (String child : listChildrenNoWatch(zkw, zkw.znodePaths.rsZNode)) { sb.append("\n ").append(child); } try { @@ -1769,7 +1768,7 @@ public class ZKUtil { throws KeeperException { String replicationZNodeName = zkw.getConfiguration().get("zookeeper.znode.replication", "replication"); - String replicationZnode = joinZNode(zkw.baseZNode, replicationZNodeName); + String replicationZnode = joinZNode(zkw.znodePaths.baseZNode, replicationZNodeName); if (ZKUtil.checkExists(zkw, replicationZnode) == -1) return; // do a ls -r on this znode sb.append("\n").append(replicationZnode).append(": "); @@ -1950,9 +1949,9 @@ public class ZKUtil { " byte(s) of data from znode " + znode + (watcherSet? " and set watcher; ": "; data=") + (data == null? "null": data.length == 0? "empty": ( - znode.startsWith(ZooKeeperWatcher.META_ZNODE_PREFIX)? + znode.startsWith(ZNodePaths.META_ZNODE_PREFIX)? getServerNameOrEmptyString(data): - znode.startsWith(zkw.backupMasterAddressesZNode)? + znode.startsWith(zkw.znodePaths.backupMasterAddressesZNode)? getServerNameOrEmptyString(data): StringUtils.abbreviate(Bytes.toStringBinary(data), 32))))); } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZNodePaths.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZNodePaths.java new file mode 100644 index 0000000..aae753a --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZNodePaths.java @@ -0,0 +1,108 @@ +/** + * 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.zookeeper; + +import com.google.common.collect.ImmutableMap; + +import java.util.stream.IntStream; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.classification.InterfaceAudience; + +/** + * Holder the znode paths on zookeeper. + */ +@InterfaceAudience.Private +public class ZNodePaths { + + public final static String META_ZNODE_PREFIX = "meta-region-server"; + // base znode for this cluster + public final String baseZNode; + // znodes containing the locations of the servers hosting the meta replicas + public final ImmutableMap metaReplicaZNodes; + // znode containing ephemeral nodes of the regionservers + public final String rsZNode; + // znode containing ephemeral nodes of the draining regionservers + public final String drainingZNode; + // znode of currently active master + public final String masterAddressZNode; + // znode of this master in backup master directory, if not the active master + public final String backupMasterAddressesZNode; + // znode containing the current cluster state + public final String clusterStateZNode; + // znode used for table disabling/enabling + @Deprecated + public final String tableZNode; + // znode containing the unique cluster ID + public final String clusterIdZNode; + // znode used for log splitting work assignment + public final String splitLogZNode; + // znode containing the state of the load balancer + public final String balancerZNode; + // znode containing the state of region normalizer + public final String regionNormalizerZNode; + // znode containing the state of all switches, currently there are split and merge child node. + public final String switchZNode; + // znode containing the lock for the tables + public final String tableLockZNode; + // znode containing the state of recovering regions + public final String recoveringRegionsZNode; + // znode containing namespace descriptors + public final String namespaceZNode; + // znode of indicating master maintenance mode + // XXX: bad design, the value could be modified by both static and non-static methods + public static String masterMaintZNode = "masterMaintenance"; + + public ZNodePaths(Configuration conf) { + baseZNode = conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT, + HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT); + String metaServer = conf.get("zookeeper.znode.metaserver", META_ZNODE_PREFIX); + ImmutableMap.Builder metaReplicaZNodesBuilder = ImmutableMap.builder(); + metaReplicaZNodesBuilder.put(0, ZKUtil.joinZNode(baseZNode, metaServer)); + int numMetaReplicas = conf.getInt(HConstants.META_REPLICAS_NUM, + HConstants.DEFAULT_META_REPLICA_NUM); + IntStream.range(1, numMetaReplicas).forEach( + i -> metaReplicaZNodesBuilder.put(i, ZKUtil.joinZNode(baseZNode, metaServer + "-" + i))); + this.metaReplicaZNodes = metaReplicaZNodesBuilder.build(); + rsZNode = ZKUtil.joinZNode(baseZNode, conf.get("zookeeper.znode.rs", "rs")); + drainingZNode = ZKUtil.joinZNode(baseZNode, + conf.get("zookeeper.znode.draining.rs", "draining")); + masterAddressZNode = ZKUtil.joinZNode(baseZNode, conf.get("zookeeper.znode.master", "master")); + backupMasterAddressesZNode = ZKUtil.joinZNode(baseZNode, + conf.get("zookeeper.znode.backup.masters", "backup-masters")); + clusterStateZNode = ZKUtil.joinZNode(baseZNode, conf.get("zookeeper.znode.state", "running")); + tableZNode = ZKUtil.joinZNode(baseZNode, + conf.get("zookeeper.znode.tableEnableDisable", "table")); + clusterIdZNode = ZKUtil.joinZNode(baseZNode, conf.get("zookeeper.znode.clusterId", "hbaseid")); + splitLogZNode = ZKUtil.joinZNode(baseZNode, + conf.get("zookeeper.znode.splitlog", HConstants.SPLIT_LOGDIR_NAME)); + balancerZNode = ZKUtil.joinZNode(baseZNode, conf.get("zookeeper.znode.balancer", "balancer")); + regionNormalizerZNode = ZKUtil.joinZNode(baseZNode, + conf.get("zookeeper.znode.regionNormalizer", "normalizer")); + switchZNode = ZKUtil.joinZNode(baseZNode, conf.get("zookeeper.znode.switch", "switch")); + tableLockZNode = ZKUtil.joinZNode(baseZNode, + conf.get("zookeeper.znode.tableLock", "table-lock")); + recoveringRegionsZNode = ZKUtil.joinZNode(baseZNode, + conf.get("zookeeper.znode.recovering.regions", "recovering-regions")); + namespaceZNode = ZKUtil.joinZNode(baseZNode, + conf.get("zookeeper.znode.namespace", "namespace")); + masterMaintZNode = ZKUtil.joinZNode(baseZNode, + conf.get("zookeeper.znode.masterMaintenance", "master-maintenance")); + } +} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperNodeTracker.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperNodeTracker.java index 262432e..a0b58cb 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperNodeTracker.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperNodeTracker.java @@ -232,15 +232,13 @@ public abstract class ZooKeeperNodeTracker extends ZooKeeperListener { */ public boolean checkIfBaseNodeAvailable() { try { - if (ZKUtil.checkExists(watcher, watcher.baseZNode) == -1) { + if (ZKUtil.checkExists(watcher, watcher.znodePaths.baseZNode) == -1) { return false; } } catch (KeeperException e) { - abortable - .abort( - "Exception while checking if basenode ("+watcher.baseZNode+ - ") exists in ZooKeeper.", - e); + abortable.abort("Exception while checking if basenode (" + watcher.znodePaths.baseZNode + + ") exists in ZooKeeper.", + e); } return true; } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java index f7d7e26..bb041c6 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java @@ -21,9 +21,7 @@ package org.apache.hadoop.hbase.zookeeper; import java.io.Closeable; import java.io.IOException; import java.util.ArrayList; -import java.util.HashMap; import java.util.List; -import java.util.Map; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.CountDownLatch; import java.util.regex.Matcher; @@ -34,7 +32,6 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Abortable; import org.apache.hadoop.hbase.AuthUtil; -import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.ZooKeeperConnectionException; import org.apache.hadoop.hbase.classification.InterfaceAudience; @@ -89,44 +86,7 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable { public CountDownLatch saslLatch = new CountDownLatch(1); // node names - - // base znode for this cluster - public String baseZNode; - //znodes containing the locations of the servers hosting the meta replicas - private Map metaReplicaZnodes = new HashMap(); - // znode containing ephemeral nodes of the regionservers - public String rsZNode; - // znode containing ephemeral nodes of the draining regionservers - public String drainingZNode; - // znode of currently active master - private String masterAddressZNode; - // znode of this master in backup master directory, if not the active master - public String backupMasterAddressesZNode; - // znode containing the current cluster state - public String clusterStateZNode; - // znode used for table disabling/enabling - @Deprecated - public String tableZNode; - // znode containing the unique cluster ID - public String clusterIdZNode; - // znode used for log splitting work assignment - public String splitLogZNode; - // znode containing the state of the load balancer - public String balancerZNode; - // znode containing the state of region normalizer - private String regionNormalizerZNode; - // znode containing the state of all switches, currently there are split and merge child node. - private String switchZNode; - // znode containing the lock for the tables - public String tableLockZNode; - // znode containing the state of recovering regions - public String recoveringRegionsZNode; - // znode containing namespace descriptors - public String namespaceZNode = "namespace"; - // znode of indicating master maintenance mode - public static String masterMaintZNode = "masterMaintenance"; - - public final static String META_ZNODE_PREFIX = "meta-region-server"; + public final ZNodePaths znodePaths; private final Configuration conf; @@ -166,7 +126,7 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable { // handle the syncconnect event. this.identifier = identifier + "0x0"; this.abortable = abortable; - setNodeNames(conf); + this.znodePaths = new ZNodePaths(conf); PendingWatcher pendingWatcher = new PendingWatcher(); this.recoverableZooKeeper = ZKUtil.connect(conf, quorum, pendingWatcher, identifier); pendingWatcher.prepare(this); @@ -188,15 +148,15 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable { private void createBaseZNodes() throws ZooKeeperConnectionException { try { // Create all the necessary "directories" of znodes - ZKUtil.createWithParents(this, baseZNode); - ZKUtil.createAndFailSilent(this, rsZNode); - ZKUtil.createAndFailSilent(this, drainingZNode); - ZKUtil.createAndFailSilent(this, tableZNode); - ZKUtil.createAndFailSilent(this, splitLogZNode); - ZKUtil.createAndFailSilent(this, backupMasterAddressesZNode); - ZKUtil.createAndFailSilent(this, tableLockZNode); - ZKUtil.createAndFailSilent(this, recoveringRegionsZNode); - ZKUtil.createAndFailSilent(this, masterMaintZNode); + ZKUtil.createWithParents(this, znodePaths.baseZNode); + ZKUtil.createAndFailSilent(this, znodePaths.rsZNode); + ZKUtil.createAndFailSilent(this, znodePaths.drainingZNode); + ZKUtil.createAndFailSilent(this, znodePaths.tableZNode); + ZKUtil.createAndFailSilent(this, znodePaths.splitLogZNode); + ZKUtil.createAndFailSilent(this, znodePaths.backupMasterAddressesZNode); + ZKUtil.createAndFailSilent(this, znodePaths.tableLockZNode); + ZKUtil.createAndFailSilent(this, znodePaths.recoveringRegionsZNode); + ZKUtil.createAndFailSilent(this, ZNodePaths.masterMaintZNode); } catch (KeeperException e) { throw new ZooKeeperConnectionException( prefix("Unexpected KeeperException creating base node"), e); @@ -210,14 +170,14 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable { // all clients need to access this data to work. Using zk for sharing data to clients (other // than service lookup case is not a recommended design pattern. return - node.equals(baseZNode) || + node.equals(znodePaths.baseZNode) || isAnyMetaReplicaZnode(node) || node.equals(getMasterAddressZNode()) || - node.equals(clusterIdZNode)|| - node.equals(rsZNode) || + node.equals(znodePaths.clusterIdZNode)|| + node.equals(znodePaths.rsZNode) || // /hbase/table and /hbase/table/foo is allowed, /hbase/table-lock is not - node.equals(tableZNode) || - node.startsWith(tableZNode + "/"); + node.equals(znodePaths.tableZNode) || + node.startsWith(znodePaths.tableZNode + "/"); } /** @@ -235,11 +195,11 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable { // Check the base znodes permission first. Only do the recursion if base znode's perms are not // correct. try { - List actualAcls = recoverableZooKeeper.getAcl(baseZNode, new Stat()); + List actualAcls = recoverableZooKeeper.getAcl(znodePaths.baseZNode, new Stat()); if (!isBaseZnodeAclSetup(actualAcls)) { LOG.info("setting znode ACLs"); - setZnodeAclsRecursive(baseZNode); + setZnodeAclsRecursive(znodePaths.baseZNode); } } catch(KeeperException.NoNodeException nne) { return; @@ -390,7 +350,7 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable { @Override public String toString() { - return this.identifier + ", quorum=" + quorum + ", baseZNode=" + baseZNode; + return this.identifier + ", quorum=" + quorum + ", baseZNode=" + znodePaths.baseZNode; } /** @@ -404,58 +364,12 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable { } /** - * Set the local variable node names using the specified configuration. - */ - private void setNodeNames(Configuration conf) { - baseZNode = conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT, - HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT); - metaReplicaZnodes.put(0, ZKUtil.joinZNode(baseZNode, - conf.get("zookeeper.znode.metaserver", "meta-region-server"))); - int numMetaReplicas = conf.getInt(HConstants.META_REPLICAS_NUM, - HConstants.DEFAULT_META_REPLICA_NUM); - for (int i = 1; i < numMetaReplicas; i++) { - String str = ZKUtil.joinZNode(baseZNode, - conf.get("zookeeper.znode.metaserver", "meta-region-server") + "-" + i); - metaReplicaZnodes.put(i, str); - } - rsZNode = ZKUtil.joinZNode(baseZNode, - conf.get("zookeeper.znode.rs", "rs")); - drainingZNode = ZKUtil.joinZNode(baseZNode, - conf.get("zookeeper.znode.draining.rs", "draining")); - masterAddressZNode = ZKUtil.joinZNode(baseZNode, - conf.get("zookeeper.znode.master", "master")); - backupMasterAddressesZNode = ZKUtil.joinZNode(baseZNode, - conf.get("zookeeper.znode.backup.masters", "backup-masters")); - clusterStateZNode = ZKUtil.joinZNode(baseZNode, - conf.get("zookeeper.znode.state", "running")); - tableZNode = ZKUtil.joinZNode(baseZNode, - conf.get("zookeeper.znode.tableEnableDisable", "table")); - clusterIdZNode = ZKUtil.joinZNode(baseZNode, - conf.get("zookeeper.znode.clusterId", "hbaseid")); - splitLogZNode = ZKUtil.joinZNode(baseZNode, - conf.get("zookeeper.znode.splitlog", HConstants.SPLIT_LOGDIR_NAME)); - balancerZNode = ZKUtil.joinZNode(baseZNode, - conf.get("zookeeper.znode.balancer", "balancer")); - regionNormalizerZNode = ZKUtil.joinZNode(baseZNode, - conf.get("zookeeper.znode.regionNormalizer", "normalizer")); - switchZNode = ZKUtil.joinZNode(baseZNode, conf.get("zookeeper.znode.switch", "switch")); - tableLockZNode = ZKUtil.joinZNode(baseZNode, - conf.get("zookeeper.znode.tableLock", "table-lock")); - recoveringRegionsZNode = ZKUtil.joinZNode(baseZNode, - conf.get("zookeeper.znode.recovering.regions", "recovering-regions")); - namespaceZNode = ZKUtil.joinZNode(baseZNode, - conf.get("zookeeper.znode.namespace", "namespace")); - masterMaintZNode = ZKUtil.joinZNode(baseZNode, - conf.get("zookeeper.znode.masterMaintenance", "master-maintenance")); - } - - /** * Is the znode of any meta replica * @param node * @return true or false */ public boolean isAnyMetaReplicaZnode(String node) { - if (metaReplicaZnodes.values().contains(node)) { + if (znodePaths.metaReplicaZNodes.values().contains(node)) { return true; } return false; @@ -479,7 +393,7 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable { * @throws KeeperException */ public List getMetaReplicaNodes() throws KeeperException { - List childrenOfBaseNode = ZKUtil.listChildrenNoWatch(this, baseZNode); + List childrenOfBaseNode = ZKUtil.listChildrenNoWatch(this, znodePaths.baseZNode); List metaReplicaNodes = new ArrayList(2); String pattern = conf.get("zookeeper.znode.metaserver","meta-region-server"); for (String child : childrenOfBaseNode) { @@ -494,12 +408,12 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable { * @return znode */ public String getZNodeForReplica(int replicaId) { - String str = metaReplicaZnodes.get(replicaId); + String str = znodePaths.metaReplicaZNodes.get(replicaId); // return a newly created path but don't update the cache of paths // This is mostly needed for tests that attempt to create meta replicas // from outside the master if (str == null) { - str = ZKUtil.joinZNode(baseZNode, + str = ZKUtil.joinZNode(znodePaths.baseZNode, conf.get("zookeeper.znode.metaserver", "meta-region-server") + "-" + replicaId); } return str; @@ -584,7 +498,7 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable { * @return the base znode of this zookeeper connection instance. */ public String getBaseZNode() { - return baseZNode; + return znodePaths.baseZNode; } /** @@ -768,20 +682,20 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable { * @return Path to the currently active master. */ public String getMasterAddressZNode() { - return this.masterAddressZNode; + return this.znodePaths.masterAddressZNode; } /** * @return ZooKeeper znode for region normalizer state */ public String getRegionNormalizerZNode() { - return regionNormalizerZNode; + return znodePaths.regionNormalizerZNode; } /** * @return ZK node for switch * */ public String getSwitchZNode() { - return switchZNode; + return znodePaths.switchZNode; } } diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperWatcher.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperWatcher.java index 10a3816..beb92d0 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperWatcher.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperWatcher.java @@ -36,23 +36,23 @@ public class TestZooKeeperWatcher { ZooKeeperWatcher watcher = new ZooKeeperWatcher(HBaseConfiguration.create(), "testIsClientReadable", null, false); - assertTrue(watcher.isClientReadable(watcher.baseZNode)); + assertTrue(watcher.isClientReadable(watcher.znodePaths.baseZNode)); assertTrue(watcher.isClientReadable(watcher.getZNodeForReplica(0))); assertTrue(watcher.isClientReadable(watcher.getMasterAddressZNode())); - assertTrue(watcher.isClientReadable(watcher.clusterIdZNode)); - assertTrue(watcher.isClientReadable(watcher.tableZNode)); - assertTrue(watcher.isClientReadable(ZKUtil.joinZNode(watcher.tableZNode, "foo"))); - assertTrue(watcher.isClientReadable(watcher.rsZNode)); + assertTrue(watcher.isClientReadable(watcher.znodePaths.clusterIdZNode)); + assertTrue(watcher.isClientReadable(watcher.znodePaths.tableZNode)); + assertTrue(watcher.isClientReadable(ZKUtil.joinZNode(watcher.znodePaths.tableZNode, "foo"))); + assertTrue(watcher.isClientReadable(watcher.znodePaths.rsZNode)); - assertFalse(watcher.isClientReadable(watcher.tableLockZNode)); - assertFalse(watcher.isClientReadable(watcher.balancerZNode)); + assertFalse(watcher.isClientReadable(watcher.znodePaths.tableLockZNode)); + assertFalse(watcher.isClientReadable(watcher.znodePaths.balancerZNode)); assertFalse(watcher.isClientReadable(watcher.getRegionNormalizerZNode())); - assertFalse(watcher.isClientReadable(watcher.clusterStateZNode)); - assertFalse(watcher.isClientReadable(watcher.drainingZNode)); - assertFalse(watcher.isClientReadable(watcher.recoveringRegionsZNode)); - assertFalse(watcher.isClientReadable(watcher.splitLogZNode)); - assertFalse(watcher.isClientReadable(watcher.backupMasterAddressesZNode)); + assertFalse(watcher.isClientReadable(watcher.znodePaths.clusterStateZNode)); + assertFalse(watcher.isClientReadable(watcher.znodePaths.drainingZNode)); + assertFalse(watcher.isClientReadable(watcher.znodePaths.recoveringRegionsZNode)); + assertFalse(watcher.isClientReadable(watcher.znodePaths.splitLogZNode)); + assertFalse(watcher.isClientReadable(watcher.znodePaths.backupMasterAddressesZNode)); watcher.close(); } diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/CollectionUtils.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/CollectionUtils.java index b7b9beb..1a1ce15 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/CollectionUtils.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/CollectionUtils.java @@ -22,6 +22,8 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.List; +import java.util.concurrent.ConcurrentMap; +import java.util.function.Supplier; import org.apache.hadoop.hbase.classification.InterfaceAudience; @@ -31,14 +33,13 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience; @InterfaceAudience.Private public class CollectionUtils { - private static final List EMPTY_LIST = Collections.unmodifiableList( - new ArrayList(0)); + private static final List EMPTY_LIST = Collections + .unmodifiableList(new ArrayList(0)); - @SuppressWarnings("unchecked") public static Collection nullSafe(Collection in) { if (in == null) { - return (Collection)EMPTY_LIST; + return (Collection) EMPTY_LIST; } return in; } @@ -77,24 +78,24 @@ public class CollectionUtils { } return null; } - + /** * @param list any list * @return -1 if list is empty, otherwise the max index */ - public static int getLastIndex(List list){ - if(isEmpty(list)){ + public static int getLastIndex(List list) { + if (isEmpty(list)) { return -1; } return list.size() - 1; } - + /** * @param list * @param index the index in question * @return true if it is the last index or if list is empty and -1 is passed for the index param */ - public static boolean isLastIndex(List list, int index){ + public static boolean isLastIndex(List list, int index) { return index == getLastIndex(list); } @@ -104,4 +105,17 @@ public class CollectionUtils { } return list.get(list.size() - 1); } + + /** + * In HBASE-16648 we found that ConcurrentHashMap.get is much faster than computeIfAbsent if the + * value already exists. So here we copy the implementation of + * {@link ConcurrentMap#computeIfAbsent(Object, java.util.function.Function)} here. It uses get + * and putIfAbsent to implement computeIfAbsent. And notice that the implementation does not + * guarantee that the supplier will only be executed once. + */ + public static V computeIfAbsent(ConcurrentMap map, K key, Supplier supplier) { + V v, newValue; + return ((v = map.get(key)) == null && (newValue = supplier.get()) != null + && (v = map.putIfAbsent(key, newValue)) == null) ? newValue : v; + } } diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ReflectionUtils.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ReflectionUtils.java index 15b3930..0fdbe61 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ReflectionUtils.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ReflectionUtils.java @@ -52,6 +52,7 @@ public class ReflectionUtils { private static T instantiate(final String className, Constructor ctor, Object[] ctorArgs) { try { + ctor.setAccessible(true); return ctor.newInstance(ctorArgs); } catch (IllegalAccessException e) { throw new UnsupportedOperationException( @@ -65,14 +66,13 @@ public class ReflectionUtils { } } - @SuppressWarnings("unchecked") public static T newInstance(Class type, Object... params) { return instantiate(type.getName(), findConstructor(type, params), params); } @SuppressWarnings("unchecked") public static Constructor findConstructor(Class type, Object... paramTypes) { - Constructor[] constructors = (Constructor[])type.getConstructors(); + Constructor[] constructors = (Constructor[])type.getDeclaredConstructors(); for (Constructor ctor : constructors) { Class[] ctorParamTypes = ctor.getParameterTypes(); if (ctorParamTypes.length != paramTypes.length) { diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestZKAndFSPermissions.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestZKAndFSPermissions.java index 018b406..9d30b7c 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestZKAndFSPermissions.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestZKAndFSPermissions.java @@ -140,7 +140,7 @@ public class IntegrationTestZKAndFSPermissions extends AbstractHBaseTool { ZooKeeperWatcher watcher = new ZooKeeperWatcher(conf, "IntegrationTestZnodeACLs", null); RecoverableZooKeeper zk = ZKUtil.connect(this.conf, watcher); - String baseZNode = watcher.baseZNode; + String baseZNode = watcher.znodePaths.baseZNode; LOG.info(""); LOG.info("***********************************************************************************"); diff --git a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java index e9f322e..49528de 100644 --- a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java +++ b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java @@ -297,7 +297,7 @@ public class RSGroupInfoManagerImpl implements RSGroupInfoManager, ServerListene groupList.addAll(rsGroupSerDe.retrieveGroupList(rsGroupTable)); } else { LOG.debug("Refershing in Offline mode."); - String groupBasePath = ZKUtil.joinZNode(watcher.baseZNode, rsGroupZNode); + String groupBasePath = ZKUtil.joinZNode(watcher.znodePaths.baseZNode, rsGroupZNode); groupList.addAll(rsGroupSerDe.retrieveGroupList(watcher, groupBasePath)); } @@ -411,7 +411,7 @@ public class RSGroupInfoManagerImpl implements RSGroupInfoManager, ServerListene try { - String groupBasePath = ZKUtil.joinZNode(watcher.baseZNode, rsGroupZNode); + String groupBasePath = ZKUtil.joinZNode(watcher.znodePaths.baseZNode, rsGroupZNode); ZKUtil.createAndFailSilent(watcher, groupBasePath, ProtobufMagic.PB_MAGIC); List zkOps = new ArrayList(newGroupMap.size()); @@ -452,7 +452,7 @@ public class RSGroupInfoManagerImpl implements RSGroupInfoManager, ServerListene try { LOG.debug("Reading online RS from zookeeper"); List servers = new LinkedList(); - for (String el: ZKUtil.listChildrenNoWatch(watcher, watcher.rsZNode)) { + for (String el: ZKUtil.listChildrenNoWatch(watcher, watcher.znodePaths.rsZNode)) { servers.add(ServerName.parseServerName(el)); } return servers; diff --git a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/VerifyingRSGroupAdminClient.java b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/VerifyingRSGroupAdminClient.java index 26853b7..950c25a 100644 --- a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/VerifyingRSGroupAdminClient.java +++ b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/VerifyingRSGroupAdminClient.java @@ -122,7 +122,7 @@ public class VerifyingRSGroupAdminClient extends RSGroupAdmin { Assert.assertEquals(Sets.newHashSet(groupMap.values()), Sets.newHashSet(wrapped.listRSGroups())); try { - String groupBasePath = ZKUtil.joinZNode(zkw.baseZNode, "rsgroup"); + String groupBasePath = ZKUtil.joinZNode(zkw.znodePaths.baseZNode, "rsgroup"); for(String znode: ZKUtil.listChildrenNoWatch(zkw, groupBasePath)) { byte[] data = ZKUtil.getData(zkw, ZKUtil.joinZNode(groupBasePath, znode)); if(data.length > 0) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ZKNamespaceManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ZKNamespaceManager.java index 7b53333..5232fd5 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ZKNamespaceManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ZKNamespaceManager.java @@ -54,7 +54,7 @@ public class ZKNamespaceManager extends ZooKeeperListener { public ZKNamespaceManager(ZooKeeperWatcher zkw) throws IOException { super(zkw); - nsZNode = zkw.namespaceZNode; + nsZNode = zkw.znodePaths.namespaceZNode; cache = new ConcurrentSkipListMap(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/example/ZKTableArchiveClient.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/example/ZKTableArchiveClient.java index 6f06476..ab31f9c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/example/ZKTableArchiveClient.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/example/ZKTableArchiveClient.java @@ -149,7 +149,7 @@ public class ZKTableArchiveClient extends Configured { * @return get the znode for long-term archival of a table for */ public static String getArchiveZNode(Configuration conf, ZooKeeperWatcher zooKeeper) { - return ZKUtil.joinZNode(zooKeeper.baseZNode, conf.get(ZOOKEEPER_ZNODE_HFILE_ARCHIVE_KEY, + return ZKUtil.joinZNode(zooKeeper.znodePaths.baseZNode, conf.get(ZOOKEEPER_ZNODE_HFILE_ARCHIVE_KEY, TableHFileArchiveTracker.HFILE_ARCHIVE_ZNODE_PARENT)); } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZKSplitLogManagerCoordination.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZKSplitLogManagerCoordination.java index cc39e9f..b52334e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZKSplitLogManagerCoordination.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZKSplitLogManagerCoordination.java @@ -136,7 +136,7 @@ public class ZKSplitLogManagerCoordination extends ZooKeeperListener implements public int remainingTasksInCoordination() { int count = 0; try { - List tasks = ZKUtil.listChildrenNoWatch(watcher, watcher.splitLogZNode); + List tasks = ZKUtil.listChildrenNoWatch(watcher, watcher.znodePaths.splitLogZNode); if (tasks != null) { int listSize = tasks.size(); for (int i = 0; i < listSize; i++) { @@ -288,7 +288,7 @@ public class ZKSplitLogManagerCoordination extends ZooKeeperListener implements final String metaEncodeRegionName = HRegionInfo.FIRST_META_REGIONINFO.getEncodedName(); int count = 0; try { - List tasks = ZKUtil.listChildrenNoWatch(watcher, watcher.splitLogZNode); + List tasks = ZKUtil.listChildrenNoWatch(watcher, watcher.znodePaths.splitLogZNode); if (tasks != null) { int listSize = tasks.size(); for (int i = 0; i < listSize; i++) { @@ -306,7 +306,8 @@ public class ZKSplitLogManagerCoordination extends ZooKeeperListener implements lastRecoveringNodeCreationTime = Long.MAX_VALUE; } else if (!recoveredServerNameSet.isEmpty()) { // Remove recovering regions which don't have any RS associated with it - List regions = ZKUtil.listChildrenNoWatch(watcher, watcher.recoveringRegionsZNode); + List regions = ZKUtil.listChildrenNoWatch(watcher, + watcher.znodePaths.recoveringRegionsZNode); if (regions != null) { int listSize = regions.size(); if (LOG.isDebugEnabled()) { @@ -323,7 +324,7 @@ public class ZKSplitLogManagerCoordination extends ZooKeeperListener implements continue; } } - String nodePath = ZKUtil.joinZNode(watcher.recoveringRegionsZNode, region); + String nodePath = ZKUtil.joinZNode(watcher.znodePaths.recoveringRegionsZNode, region); List failedServers = ZKUtil.listChildrenNoWatch(watcher, nodePath); if (failedServers == null || failedServers.isEmpty()) { ZKUtil.deleteNode(watcher, nodePath); @@ -562,13 +563,13 @@ public class ZKSplitLogManagerCoordination extends ZooKeeperListener implements private void lookForOrphans() { List orphans; try { - orphans = ZKUtil.listChildrenNoWatch(this.watcher, this.watcher.splitLogZNode); + orphans = ZKUtil.listChildrenNoWatch(this.watcher, this.watcher.znodePaths.splitLogZNode); if (orphans == null) { - LOG.warn("could not get children of " + this.watcher.splitLogZNode); + LOG.warn("could not get children of " + this.watcher.znodePaths.splitLogZNode); return; } } catch (KeeperException e) { - LOG.warn("could not get children of " + this.watcher.splitLogZNode + " " + LOG.warn("could not get children of " + this.watcher.znodePaths.splitLogZNode + " " + StringUtils.stringifyException(e)); return; } @@ -576,7 +577,7 @@ public class ZKSplitLogManagerCoordination extends ZooKeeperListener implements int listSize = orphans.size(); for (int i = 0; i < listSize; i++) { String path = orphans.get(i); - String nodepath = ZKUtil.joinZNode(watcher.splitLogZNode, path); + String nodepath = ZKUtil.joinZNode(watcher.znodePaths.splitLogZNode, path); if (ZKSplitLog.isRescanNode(watcher, nodepath)) { rescan_nodes++; LOG.debug("found orphan rescan node " + path); @@ -604,7 +605,7 @@ public class ZKSplitLogManagerCoordination extends ZooKeeperListener implements long retries = this.zkretries; do { - String nodePath = ZKUtil.joinZNode(watcher.recoveringRegionsZNode, regionEncodeName); + String nodePath = ZKUtil.joinZNode(watcher.znodePaths.recoveringRegionsZNode, regionEncodeName); long lastRecordedFlushedSequenceId = -1; try { long lastSequenceId = @@ -680,14 +681,15 @@ public class ZKSplitLogManagerCoordination extends ZooKeeperListener implements throws IOException, InterruptedIOException { try { - List tasks = ZKUtil.listChildrenNoWatch(watcher, watcher.splitLogZNode); + List tasks = ZKUtil.listChildrenNoWatch(watcher, watcher.znodePaths.splitLogZNode); if (tasks != null) { int listSize = tasks.size(); for (int i = 0; i < listSize; i++) { String t = tasks.get(i); byte[] data; try { - data = ZKUtil.getData(this.watcher, ZKUtil.joinZNode(watcher.splitLogZNode, t)); + data = ZKUtil.getData(this.watcher, + ZKUtil.joinZNode(watcher.znodePaths.splitLogZNode, t)); } catch (InterruptedException e) { throw new InterruptedIOException(); } @@ -715,11 +717,13 @@ public class ZKSplitLogManagerCoordination extends ZooKeeperListener implements } // remove recovering regions which doesn't have any RS associated with it - List regions = ZKUtil.listChildrenNoWatch(watcher, watcher.recoveringRegionsZNode); + List regions = ZKUtil.listChildrenNoWatch(watcher, + watcher.znodePaths.recoveringRegionsZNode); if (regions != null) { int listSize = regions.size(); for (int i = 0; i < listSize; i++) { - String nodePath = ZKUtil.joinZNode(watcher.recoveringRegionsZNode, regions.get(i)); + String nodePath = ZKUtil.joinZNode(watcher.znodePaths.recoveringRegionsZNode, + regions.get(i)); List regionFailedServers = ZKUtil.listChildrenNoWatch(watcher, nodePath); if (regionFailedServers == null || regionFailedServers.isEmpty()) { ZKUtil.deleteNode(watcher, nodePath); @@ -754,7 +758,8 @@ public class ZKSplitLogManagerCoordination extends ZooKeeperListener implements } private List listSplitLogTasks() throws KeeperException { - List taskOrRescanList = ZKUtil.listChildrenNoWatch(watcher, watcher.splitLogZNode); + List taskOrRescanList = ZKUtil.listChildrenNoWatch(watcher, + watcher.znodePaths.splitLogZNode); if (taskOrRescanList == null || taskOrRescanList.isEmpty()) { return Collections. emptyList(); } @@ -799,7 +804,8 @@ public class ZKSplitLogManagerCoordination extends ZooKeeperListener implements // Firstly check if there are outstanding recovering regions try { - List regions = ZKUtil.listChildrenNoWatch(watcher, watcher.recoveringRegionsZNode); + List regions = ZKUtil.listChildrenNoWatch(watcher, + watcher.znodePaths.recoveringRegionsZNode); if (regions != null && !regions.isEmpty()) { hasRecoveringRegions = true; previousRecoveryMode = RecoveryMode.LOG_REPLAY; @@ -815,8 +821,8 @@ public class ZKSplitLogManagerCoordination extends ZooKeeperListener implements for (int i = 0; i < listSize; i++) { String task = tasks.get(i); try { - byte[] data = - ZKUtil.getData(this.watcher, ZKUtil.joinZNode(watcher.splitLogZNode, task)); + byte[] data = ZKUtil.getData(this.watcher, + ZKUtil.joinZNode(watcher.znodePaths.splitLogZNode, task)); if (data == null) continue; SplitLogTask slt = SplitLogTask.parseFrom(data); previousRecoveryMode = slt.getMode(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkSplitLogWorkerCoordination.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkSplitLogWorkerCoordination.java index e788860..14a9896 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkSplitLogWorkerCoordination.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkSplitLogWorkerCoordination.java @@ -103,7 +103,7 @@ public class ZkSplitLogWorkerCoordination extends ZooKeeperListener implements */ @Override public void nodeChildrenChanged(String path) { - if (path.equals(watcher.splitLogZNode)) { + if (path.equals(watcher.znodePaths.splitLogZNode)) { if (LOG.isTraceEnabled()) LOG.trace("tasks arrived or departed on " + path); synchronized (taskReadyLock) { this.taskReadySeq.incrementAndGet(); @@ -335,7 +335,7 @@ public class ZkSplitLogWorkerCoordination extends ZooKeeperListener implements int availableRSs = 1; try { List regionServers = - ZKUtil.listChildrenNoWatch(watcher, watcher.rsZNode); + ZKUtil.listChildrenNoWatch(watcher, watcher.znodePaths.rsZNode); availableRSs = Math.max(availableRSs, (regionServers == null) ? 0 : regionServers.size()); } catch (KeeperException e) { // do nothing @@ -411,7 +411,7 @@ public class ZkSplitLogWorkerCoordination extends ZooKeeperListener implements List paths = null; paths = getTaskList(); if (paths == null) { - LOG.warn("Could not get tasks, did someone remove " + watcher.splitLogZNode + LOG.warn("Could not get tasks, did someone remove " + watcher.znodePaths.splitLogZNode + " ... worker thread exiting."); return; } @@ -429,7 +429,7 @@ public class ZkSplitLogWorkerCoordination extends ZooKeeperListener implements // don't call ZKSplitLog.getNodeName() because that will lead to // double encoding of the path name if (this.calculateAvailableSplitters(numTasks) > 0) { - grabTask(ZKUtil.joinZNode(watcher.splitLogZNode, paths.get(idx))); + grabTask(ZKUtil.joinZNode(watcher.znodePaths.splitLogZNode, paths.get(idx))); } else { LOG.debug("Current region server " + server.getServerName() + " has " + this.tasksInProgress.get() + " tasks in progress and can't take more."); @@ -453,7 +453,7 @@ public class ZkSplitLogWorkerCoordination extends ZooKeeperListener implements int listSize = tmpCopy.size(); for (int i = 0; i < listSize; i++) { String region = tmpCopy.get(i); - String nodePath = ZKUtil.joinZNode(watcher.recoveringRegionsZNode, region); + String nodePath = ZKUtil.joinZNode(watcher.znodePaths.recoveringRegionsZNode, region); try { if (ZKUtil.checkExists(watcher, nodePath) == -1) { server.getExecutorService().submit( @@ -488,14 +488,14 @@ public class ZkSplitLogWorkerCoordination extends ZooKeeperListener implements try { childrenPaths = ZKUtil.listChildrenAndWatchForNewChildren(watcher, - watcher.splitLogZNode); + watcher.znodePaths.splitLogZNode); if (childrenPaths != null) { return childrenPaths; } } catch (KeeperException e) { - LOG.warn("Could not get children of znode " + watcher.splitLogZNode, e); + LOG.warn("Could not get children of znode " + watcher.znodePaths.splitLogZNode, e); } - LOG.debug("Retry listChildren of znode " + watcher.splitLogZNode + LOG.debug("Retry listChildren of znode " + watcher.znodePaths.splitLogZNode + " after sleep for " + sleepTime + "ms!"); Thread.sleep(sleepTime); } @@ -511,14 +511,14 @@ public class ZkSplitLogWorkerCoordination extends ZooKeeperListener implements public boolean isReady() throws InterruptedException { int result = -1; try { - result = ZKUtil.checkExists(watcher, watcher.splitLogZNode); + result = ZKUtil.checkExists(watcher, watcher.znodePaths.splitLogZNode); } catch (KeeperException e) { // ignore - LOG.warn("Exception when checking for " + watcher.splitLogZNode + LOG.warn("Exception when checking for " + watcher.znodePaths.splitLogZNode + " ... retrying", e); } if (result == -1) { - LOG.info(watcher.splitLogZNode + LOG.info(watcher.znodePaths.splitLogZNode + " znode does not exist, waiting for master to create"); Thread.sleep(1000); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ActiveMasterManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ActiveMasterManager.java index efb1cfd..b8dfdf4 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ActiveMasterManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ActiveMasterManager.java @@ -91,7 +91,7 @@ public class ActiveMasterManager extends ZooKeeperListener { // shut down, so that state is now irrelevant. This means that the shutdown // state must be set while we wait on the active master in order // to shutdown this master. See HBASE-8519. - if(path.equals(watcher.clusterStateZNode) && !master.isStopped()) { + if (path.equals(watcher.znodePaths.clusterStateZNode) && !master.isStopped()) { clusterShutDown.set(true); } @@ -155,7 +155,7 @@ public class ActiveMasterManager extends ZooKeeperListener { boolean blockUntilBecomingActiveMaster( int checkInterval, MonitoredTask startupStatus) { String backupZNode = ZKUtil.joinZNode( - this.watcher.backupMasterAddressesZNode, this.sn.toString()); + this.watcher.znodePaths.backupMasterAddressesZNode, this.sn.toString()); while (!(master.isAborted() || master.isStopped())) { startupStatus.setStatus("Trying to register in ZK as active master"); // Try to become the active master, watch if there is another master. diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java index 810f95b..8688ed7 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java @@ -768,7 +768,7 @@ public class HMaster extends HRegionServer implements MasterServices { LOG.info("Converting state from zk to new states:" + entry); tableStateManager.setTableState(entry.getKey(), entry.getValue()); } - ZKUtil.deleteChildrenRecursively(getZooKeeper(), getZooKeeper().tableZNode); + ZKUtil.deleteChildrenRecursively(getZooKeeper(), getZooKeeper().znodePaths.tableZNode); status.setStatus("Submitting log splitting work for previously failed region servers"); metaBootstrap.processDeadServers(); @@ -1693,7 +1693,7 @@ public class HMaster extends HRegionServer implements MasterServices { private void startActiveMasterManager(int infoPort) throws KeeperException { String backupZNode = ZKUtil.joinZNode( - zooKeeper.backupMasterAddressesZNode, serverName.toString()); + zooKeeper.znodePaths.backupMasterAddressesZNode, serverName.toString()); /* * Add a ZNode for ourselves in the backup master directory since we * may not become the active master. If so, we want the actual active @@ -2093,7 +2093,7 @@ public class HMaster extends HRegionServer implements MasterServices { List backupMasterStrings; try { backupMasterStrings = ZKUtil.listChildrenNoWatch(this.zooKeeper, - this.zooKeeper.backupMasterAddressesZNode); + this.zooKeeper.znodePaths.backupMasterAddressesZNode); } catch (KeeperException e) { LOG.warn(this.zooKeeper.prefix("Unable to list backup servers"), e); backupMasterStrings = null; @@ -2107,7 +2107,7 @@ public class HMaster extends HRegionServer implements MasterServices { byte [] bytes; try { bytes = ZKUtil.getData(this.zooKeeper, ZKUtil.joinZNode( - this.zooKeeper.backupMasterAddressesZNode, s)); + this.zooKeeper.znodePaths.backupMasterAddressesZNode, s)); } catch (InterruptedException e) { throw new InterruptedIOException(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMetaBootstrap.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMetaBootstrap.java index 0504555..3ebf0eb 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMetaBootstrap.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMetaBootstrap.java @@ -243,7 +243,7 @@ public class MasterMetaBootstrap { private Set getPreviouselyFailedMetaServersFromZK() throws KeeperException { final ZooKeeperWatcher zooKeeper = master.getZooKeeper(); Set result = new HashSet(); - String metaRecoveringZNode = ZKUtil.joinZNode(zooKeeper.recoveringRegionsZNode, + String metaRecoveringZNode = ZKUtil.joinZNode(zooKeeper.znodePaths.recoveringRegionsZNode, HRegionInfo.FIRST_META_REGIONINFO.getEncodedName()); List regionFailedServers = ZKUtil.listChildrenNoWatch(zooKeeper, metaRecoveringZNode); if (regionFailedServers == null) return result; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java index f97dfb4..a65bab4 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java @@ -563,7 +563,7 @@ public class ServerManager { } try { - List servers = ZKUtil.listChildrenNoWatch(zkw, zkw.rsZNode); + List servers = ZKUtil.listChildrenNoWatch(zkw, zkw.znodePaths.rsZNode); if (servers == null || servers.size() == 0 || (servers.size() == 1 && servers.contains(sn.toString()))) { LOG.info("ZK shows there is only the master self online, exiting now"); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableLockManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableLockManager.java index 86e9093..6a69059 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableLockManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableLockManager.java @@ -316,8 +316,8 @@ public abstract class TableLockManager { } private InterProcessLock createTableLock() { - String tableLockZNode = ZKUtil.joinZNode(zkWatcher.tableLockZNode, - tableName.getNameAsString()); + String tableLockZNode = ZKUtil.joinZNode(zkWatcher.znodePaths.tableLockZNode, + tableName.getNameAsString()); ZooKeeperProtos.TableLock data = ZooKeeperProtos.TableLock.newBuilder() .setTableName(ProtobufUtil.toProtoTableName(tableName)) @@ -376,7 +376,7 @@ public abstract class TableLockManager { public void visitAllLocks(MetadataHandler handler) throws IOException { for (String tableName : getTableNames()) { - String tableLockZNode = ZKUtil.joinZNode(zkWatcher.tableLockZNode, tableName); + String tableLockZNode = ZKUtil.joinZNode(zkWatcher.znodePaths.tableLockZNode, tableName); ZKInterProcessReadWriteLock lock = new ZKInterProcessReadWriteLock( zkWatcher, tableLockZNode, null); lock.readLock(null).visitLocks(handler); @@ -388,7 +388,7 @@ public abstract class TableLockManager { List tableNames; try { - tableNames = ZKUtil.listChildrenNoWatch(zkWatcher, zkWatcher.tableLockZNode); + tableNames = ZKUtil.listChildrenNoWatch(zkWatcher, zkWatcher.znodePaths.tableLockZNode); } catch (KeeperException e) { LOG.error("Unexpected ZooKeeper error when listing children", e); throw new IOException("Unexpected ZooKeeper exception", e); @@ -401,7 +401,7 @@ public abstract class TableLockManager { //get the table names try { for (String tableName : getTableNames()) { - String tableLockZNode = ZKUtil.joinZNode(zkWatcher.tableLockZNode, tableName); + String tableLockZNode = ZKUtil.joinZNode(zkWatcher.znodePaths.tableLockZNode, tableName); ZKInterProcessReadWriteLock lock = new ZKInterProcessReadWriteLock( zkWatcher, tableLockZNode, null); lock.writeLock(null).reapAllLocks(); @@ -418,7 +418,7 @@ public abstract class TableLockManager { //get the table names try { for (String tableName : getTableNames()) { - String tableLockZNode = ZKUtil.joinZNode(zkWatcher.tableLockZNode, tableName); + String tableLockZNode = ZKUtil.joinZNode(zkWatcher.znodePaths.tableLockZNode, tableName); ZKInterProcessReadWriteLock lock = new ZKInterProcessReadWriteLock( zkWatcher, tableLockZNode, null); lock.readLock(null).reapExpiredLocks(lockExpireTimeoutMs); @@ -435,7 +435,7 @@ public abstract class TableLockManager { public void tableDeleted(TableName tableName) throws IOException { //table write lock from DeleteHandler is already released, just delete the parent znode String tableNameStr = tableName.getNameAsString(); - String tableLockZNode = ZKUtil.joinZNode(zkWatcher.tableLockZNode, tableNameStr); + String tableLockZNode = ZKUtil.joinZNode(zkWatcher.znodePaths.tableLockZNode, tableNameStr); try { ZKUtil.deleteNode(zkWatcher, tableLockZNode); } catch (KeeperException ex) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/mapreduce/SweepJob.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/mapreduce/SweepJob.java index 79cd39c..0b3ca29 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/mapreduce/SweepJob.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/mapreduce/SweepJob.java @@ -200,7 +200,8 @@ public class SweepJob { JavaSerialization.class.getName() + "," + WritableSerialization.class.getName()); conf.set(SWEEP_JOB_ID, id); conf.set(SWEEP_JOB_SERVERNAME, serverName.toString()); - String tableLockNode = ZKUtil.joinZNode(zkw.tableLockZNode, lockName.getNameAsString()); + String tableLockNode = ZKUtil.joinZNode(zkw.znodePaths.tableLockZNode, + lockName.getNameAsString()); conf.set(SWEEP_JOB_TABLE_NODE, tableLockNode); job = prepareJob(tn, familyName, scan, conf); job.getConfiguration().set(TableInputFormat.SCAN_COLUMN_FAMILY, familyName); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureUtil.java index 56983ff..4ebb411 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureUtil.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureUtil.java @@ -78,7 +78,7 @@ public abstract class ZKProcedureUtil // make sure we are listening for events watcher.registerListener(this); // setup paths for the zknodes used in procedures - this.baseZNode = ZKUtil.joinZNode(watcher.baseZNode, procDescription); + this.baseZNode = ZKUtil.joinZNode(watcher.znodePaths.baseZNode, procDescription); acquiredZnode = ZKUtil.joinZNode(baseZNode, ACQUIRED_BARRIER_ZNODE_DEFAULT); reachedZnode = ZKUtil.joinZNode(baseZNode, REACHED_BARRIER_ZNODE_DEFAULT); abortZnode = ZKUtil.joinZNode(baseZNode, ABORT_ZNODE_DEFAULT); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java index a11d367..8fd5df9 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java @@ -3230,7 +3230,7 @@ public class HRegionServer extends HasThread implements } private String getMyEphemeralNodePath() { - return ZKUtil.joinZNode(this.zooKeeper.rsZNode, getServerName().toString()); + return ZKUtil.joinZNode(this.zooKeeper.znodePaths.rsZNode, getServerName().toString()); } private boolean isHealthCheckerConfigured() { @@ -3272,7 +3272,7 @@ public class HRegionServer extends HasThread implements try { long lastRecordedFlushedSequenceId = -1; - String nodePath = ZKUtil.joinZNode(this.zooKeeper.recoveringRegionsZNode, + String nodePath = ZKUtil.joinZNode(this.zooKeeper.znodePaths.recoveringRegionsZNode, regionInfo.getEncodedName()); // recovering-region level byte[] data; @@ -3313,7 +3313,7 @@ public class HRegionServer extends HasThread implements String result = null; long maxZxid = 0; ZooKeeperWatcher zkw = this.getZooKeeper(); - String nodePath = ZKUtil.joinZNode(zkw.recoveringRegionsZNode, encodedRegionName); + String nodePath = ZKUtil.joinZNode(zkw.znodePaths.recoveringRegionsZNode, encodedRegionName); List failedServers = ZKUtil.listChildrenNoWatch(zkw, nodePath); if (failedServers == null || failedServers.isEmpty()) { return result; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/HBaseReplicationEndpoint.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/HBaseReplicationEndpoint.java index 6485e4a..1a603e0 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/HBaseReplicationEndpoint.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/HBaseReplicationEndpoint.java @@ -147,7 +147,7 @@ public abstract class HBaseReplicationEndpoint extends BaseReplicationEndpoint */ protected static List fetchSlavesAddresses(ZooKeeperWatcher zkw) throws KeeperException { - List children = ZKUtil.listChildrenAndWatchForNewChildren(zkw, zkw.rsZNode); + List children = ZKUtil.listChildrenAndWatchForNewChildren(zkw, zkw.znodePaths.rsZNode); if (children == null) { return Collections.emptyList(); } @@ -208,7 +208,7 @@ public abstract class HBaseReplicationEndpoint extends BaseReplicationEndpoint public PeerRegionServerListener(HBaseReplicationEndpoint replicationPeer) { super(replicationPeer.getZkw()); this.replicationEndpoint = replicationPeer; - this.regionServerListNode = replicationEndpoint.getZkw().rsZNode; + this.regionServerListNode = replicationEndpoint.getZkw().znodePaths.rsZNode; } @Override diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/ZKPermissionWatcher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/ZKPermissionWatcher.java index c4b3219..308ef41 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/ZKPermissionWatcher.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/ZKPermissionWatcher.java @@ -66,7 +66,7 @@ public class ZKPermissionWatcher extends ZooKeeperListener implements Closeable super(watcher); this.authManager = authManager; String aclZnodeParent = conf.get("zookeeper.znode.acl.parent", ACL_NODE); - this.aclZNode = ZKUtil.joinZNode(watcher.baseZNode, aclZnodeParent); + this.aclZNode = ZKUtil.joinZNode(watcher.znodePaths.baseZNode, aclZnodeParent); executor = Executors.newSingleThreadExecutor( new DaemonThreadFactory("zk-permission-watcher")); } @@ -249,7 +249,7 @@ public class ZKPermissionWatcher extends ZooKeeperListener implements Closeable */ public void writeToZookeeper(byte[] entry, byte[] permsData) { String entryName = Bytes.toString(entry); - String zkNode = ZKUtil.joinZNode(watcher.baseZNode, ACL_NODE); + String zkNode = ZKUtil.joinZNode(watcher.znodePaths.baseZNode, ACL_NODE); zkNode = ZKUtil.joinZNode(zkNode, entryName); try { @@ -267,7 +267,7 @@ public class ZKPermissionWatcher extends ZooKeeperListener implements Closeable * @param tableName */ public void deleteTableACLNode(final TableName tableName) { - String zkNode = ZKUtil.joinZNode(watcher.baseZNode, ACL_NODE); + String zkNode = ZKUtil.joinZNode(watcher.znodePaths.baseZNode, ACL_NODE); zkNode = ZKUtil.joinZNode(zkNode, tableName.getNameAsString()); try { @@ -284,7 +284,7 @@ public class ZKPermissionWatcher extends ZooKeeperListener implements Closeable * Delete the acl notify node of namespace */ public void deleteNamespaceACLNode(final String namespace) { - String zkNode = ZKUtil.joinZNode(watcher.baseZNode, ACL_NODE); + String zkNode = ZKUtil.joinZNode(watcher.znodePaths.baseZNode, ACL_NODE); zkNode = ZKUtil.joinZNode(zkNode, AccessControlLists.NAMESPACE_PREFIX + namespace); try { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/ZKSecretWatcher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/ZKSecretWatcher.java index a1c3b66..f9d716b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/ZKSecretWatcher.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/ZKSecretWatcher.java @@ -52,7 +52,7 @@ public class ZKSecretWatcher extends ZooKeeperListener { super(watcher); this.secretManager = secretManager; String keyZNodeParent = conf.get("zookeeper.znode.tokenauth.parent", DEFAULT_ROOT_NODE); - this.baseKeyZNode = ZKUtil.joinZNode(watcher.baseZNode, keyZNodeParent); + this.baseKeyZNode = ZKUtil.joinZNode(watcher.znodePaths.baseZNode, keyZNodeParent); this.keysParentZNode = ZKUtil.joinZNode(baseKeyZNode, DEFAULT_KEYS_PARENT); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/ZKVisibilityLabelWatcher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/ZKVisibilityLabelWatcher.java index f30136f..a60a3e9 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/ZKVisibilityLabelWatcher.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/ZKVisibilityLabelWatcher.java @@ -53,8 +53,8 @@ public class ZKVisibilityLabelWatcher extends ZooKeeperListener { String labelZnodeParent = conf.get(VISIBILITY_LABEL_ZK_PATH, DEFAULT_VISIBILITY_LABEL_NODE); String userAuthsZnodeParent = conf.get(VISIBILITY_USER_AUTHS_ZK_PATH, DEFAULT_VISIBILITY_USER_AUTHS_NODE); - this.labelZnode = ZKUtil.joinZNode(watcher.baseZNode, labelZnodeParent); - this.userAuthsZnode = ZKUtil.joinZNode(watcher.baseZNode, userAuthsZnodeParent); + this.labelZnode = ZKUtil.joinZNode(watcher.znodePaths.baseZNode, labelZnodeParent); + this.userAuthsZnode = ZKUtil.joinZNode(watcher.znodePaths.baseZNode, userAuthsZnodeParent); } public void start() throws KeeperException { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java index a1a6099..de89d64 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java @@ -135,6 +135,7 @@ import org.apache.hadoop.hbase.wal.WALFactory; import org.apache.hadoop.hbase.wal.WALSplitter; import org.apache.hadoop.hbase.zookeeper.MetaTableLocator; import org.apache.hadoop.hbase.zookeeper.ZKUtil; +import org.apache.hadoop.hbase.zookeeper.ZNodePaths; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException; import org.apache.hadoop.ipc.RemoteException; @@ -697,7 +698,7 @@ public class HBaseFsck extends Configured implements Closeable { private boolean setMasterInMaintenanceMode() throws IOException { RetryCounter retryCounter = createZNodeRetryCounterFactory.create(); hbckEphemeralNodePath = ZKUtil.joinZNode( - ZooKeeperWatcher.masterMaintZNode, + ZNodePaths.masterMaintZNode, "hbck-" + Long.toString(EnvironmentEdgeManager.currentTime())); do { try { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ZKDataMigrator.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ZKDataMigrator.java index 8f1b13b..b61fa19 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ZKDataMigrator.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ZKDataMigrator.java @@ -54,7 +54,7 @@ public class ZKDataMigrator { public static Map queryForTableStates(ZooKeeperWatcher zkw) throws KeeperException, InterruptedException { Map rv = new HashMap<>(); - List children = ZKUtil.listChildrenNoWatch(zkw, zkw.tableZNode); + List children = ZKUtil.listChildrenNoWatch(zkw, zkw.znodePaths.tableZNode); if (children == null) return rv; for (String child: children) { @@ -94,7 +94,7 @@ public class ZKDataMigrator { private static ZooKeeperProtos.DeprecatedTableState.State getTableState( final ZooKeeperWatcher zkw, final TableName tableName) throws KeeperException, InterruptedException { - String znode = ZKUtil.joinZNode(zkw.tableZNode, tableName.getNameAsString()); + String znode = ZKUtil.joinZNode(zkw.znodePaths.tableZNode, tableName.getNameAsString()); byte [] data = ZKUtil.getData(zkw, znode); if (data == null || data.length <= 0) return null; try { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/hbck/ReplicationChecker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/hbck/ReplicationChecker.java index 8d66c8f..4a430ec 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/hbck/ReplicationChecker.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/hbck/ReplicationChecker.java @@ -80,7 +80,7 @@ public class ReplicationChecker { } String replicationZNodeName = conf.get("zookeeper.znode.replication", "replication"); - String replicationZNode = ZKUtil.joinZNode(this.zkw.baseZNode, replicationZNodeName); + String replicationZNode = ZKUtil.joinZNode(this.zkw.znodePaths.baseZNode, replicationZNodeName); String hfileRefsZNodeName = conf.get(ReplicationStateZKBase.ZOOKEEPER_ZNODE_REPLICATION_HFILE_REFS_KEY, ReplicationStateZKBase.ZOOKEEPER_ZNODE_REPLICATION_HFILE_REFS_DEFAULT); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ClusterStatusTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ClusterStatusTracker.java index ccfdf1d..4e97ab0 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ClusterStatusTracker.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ClusterStatusTracker.java @@ -46,7 +46,7 @@ public class ClusterStatusTracker extends ZooKeeperNodeTracker { * @param abortable */ public ClusterStatusTracker(ZooKeeperWatcher watcher, Abortable abortable) { - super(watcher, watcher.clusterStateZNode, abortable); + super(watcher, watcher.znodePaths.clusterStateZNode, abortable); } /** @@ -66,9 +66,9 @@ public class ClusterStatusTracker extends ZooKeeperNodeTracker { throws KeeperException { byte [] upData = toByteArray(); try { - ZKUtil.createAndWatch(watcher, watcher.clusterStateZNode, upData); + ZKUtil.createAndWatch(watcher, watcher.znodePaths.clusterStateZNode, upData); } catch(KeeperException.NodeExistsException nee) { - ZKUtil.setData(watcher, watcher.clusterStateZNode, upData); + ZKUtil.setData(watcher, watcher.znodePaths.clusterStateZNode, upData); } } @@ -79,10 +79,10 @@ public class ClusterStatusTracker extends ZooKeeperNodeTracker { public void setClusterDown() throws KeeperException { try { - ZKUtil.deleteNode(watcher, watcher.clusterStateZNode); + ZKUtil.deleteNode(watcher, watcher.znodePaths.clusterStateZNode); } catch(KeeperException.NoNodeException nne) { LOG.warn("Attempted to set cluster as down but already down, cluster " + - "state node (" + watcher.clusterStateZNode + ") not found"); + "state node (" + watcher.znodePaths.clusterStateZNode + ") not found"); } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/DrainingServerTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/DrainingServerTracker.java index 5969143..684fdec 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/DrainingServerTracker.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/DrainingServerTracker.java @@ -69,7 +69,7 @@ public class DrainingServerTracker extends ZooKeeperListener { public void start() throws KeeperException, IOException { watcher.registerListener(this); List servers = - ZKUtil.listChildrenAndWatchThem(watcher, watcher.drainingZNode); + ZKUtil.listChildrenAndWatchThem(watcher, watcher.znodePaths.drainingZNode); add(servers); } @@ -96,7 +96,7 @@ public class DrainingServerTracker extends ZooKeeperListener { @Override public void nodeDeleted(final String path) { - if(path.startsWith(watcher.drainingZNode)) { + if(path.startsWith(watcher.znodePaths.drainingZNode)) { final ServerName sn = ServerName.valueOf(ZKUtil.getNodeName(path)); LOG.info("Draining RS node deleted, removing from list [" + sn + "]"); @@ -106,10 +106,10 @@ public class DrainingServerTracker extends ZooKeeperListener { @Override public void nodeChildrenChanged(final String path) { - if(path.equals(watcher.drainingZNode)) { + if(path.equals(watcher.znodePaths.drainingZNode)) { try { final List newNodes = - ZKUtil.listChildrenAndWatchThem(watcher, watcher.drainingZNode); + ZKUtil.listChildrenAndWatchThem(watcher, watcher.znodePaths.drainingZNode); add(newNodes); } catch (KeeperException e) { abortable.abort("Unexpected zk exception getting RS nodes", e); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/LoadBalancerTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/LoadBalancerTracker.java index 0a0d1ab..67eb664 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/LoadBalancerTracker.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/LoadBalancerTracker.java @@ -38,7 +38,7 @@ public class LoadBalancerTracker extends ZooKeeperNodeTracker { public LoadBalancerTracker(ZooKeeperWatcher watcher, Abortable abortable) { - super(watcher, watcher.balancerZNode, abortable); + super(watcher, watcher.znodePaths.balancerZNode, abortable); } /** @@ -64,11 +64,11 @@ public class LoadBalancerTracker extends ZooKeeperNodeTracker { public void setBalancerOn(boolean balancerOn) throws KeeperException { byte [] upData = toByteArray(balancerOn); try { - ZKUtil.setData(watcher, watcher.balancerZNode, upData); + ZKUtil.setData(watcher, watcher.znodePaths.balancerZNode, upData); } catch(KeeperException.NoNodeException nne) { - ZKUtil.createAndWatch(watcher, watcher.balancerZNode, upData); + ZKUtil.createAndWatch(watcher, watcher.znodePaths.balancerZNode, upData); } - super.nodeDataChanged(watcher.balancerZNode); + super.nodeDataChanged(watcher.znodePaths.balancerZNode); } private byte [] toByteArray(boolean isBalancerOn) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/MasterMaintenanceModeTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/MasterMaintenanceModeTracker.java index fc0e05f..733cae7 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/MasterMaintenanceModeTracker.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/MasterMaintenanceModeTracker.java @@ -40,7 +40,7 @@ public class MasterMaintenanceModeTracker extends ZooKeeperListener { } private void update(String path) { - if (path.startsWith(ZooKeeperWatcher.masterMaintZNode)) { + if (path.startsWith(ZNodePaths.masterMaintZNode)) { update(); } } @@ -48,7 +48,7 @@ public class MasterMaintenanceModeTracker extends ZooKeeperListener { private void update() { try { List children = - ZKUtil.listChildrenAndWatchForNewChildren(watcher, ZooKeeperWatcher.masterMaintZNode); + ZKUtil.listChildrenAndWatchForNewChildren(watcher, ZNodePaths.masterMaintZNode); hasChildren = (children != null && children.size() > 0); } catch (KeeperException e) { // Ignore the ZK keeper exception diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoveringRegionWatcher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoveringRegionWatcher.java index 5fff9d2..abe9d22 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoveringRegionWatcher.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoveringRegionWatcher.java @@ -54,7 +54,7 @@ public class RecoveringRegionWatcher extends ZooKeeperListener { } String parentPath = path.substring(0, path.lastIndexOf('/')); - if (!this.watcher.recoveringRegionsZNode.equalsIgnoreCase(parentPath)) { + if (!this.watcher.znodePaths.recoveringRegionsZNode.equalsIgnoreCase(parentPath)) { return; } @@ -79,7 +79,7 @@ public class RecoveringRegionWatcher extends ZooKeeperListener { */ private void registerWatcher(String path) { String parentPath = path.substring(0, path.lastIndexOf('/')); - if (!this.watcher.recoveringRegionsZNode.equalsIgnoreCase(parentPath)) { + if (!this.watcher.znodePaths.recoveringRegionsZNode.equalsIgnoreCase(parentPath)) { return; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/RegionServerTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/RegionServerTracker.java index 365010f..234fe26 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/RegionServerTracker.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/RegionServerTracker.java @@ -71,7 +71,7 @@ public class RegionServerTracker extends ZooKeeperListener { public void start() throws KeeperException, IOException { watcher.registerListener(this); List servers = - ZKUtil.listChildrenAndWatchThem(watcher, watcher.rsZNode); + ZKUtil.listChildrenAndWatchThem(watcher, watcher.znodePaths.rsZNode); add(servers); } @@ -83,7 +83,7 @@ public class RegionServerTracker extends ZooKeeperListener { if (regionServers.get(sn) == null) { RegionServerInfo.Builder rsInfoBuilder = RegionServerInfo.newBuilder(); try { - String nodePath = ZKUtil.joinZNode(watcher.rsZNode, n); + String nodePath = ZKUtil.joinZNode(watcher.znodePaths.rsZNode, n); byte[] data = ZKUtil.getData(watcher, nodePath); if (data != null && data.length > 0 && ProtobufUtil.isPBMagicPrefix(data)) { int magicLen = ProtobufUtil.lengthOfPBMagic(); @@ -113,7 +113,7 @@ public class RegionServerTracker extends ZooKeeperListener { @Override public void nodeDeleted(String path) { - if (path.startsWith(watcher.rsZNode)) { + if (path.startsWith(watcher.znodePaths.rsZNode)) { String serverName = ZKUtil.getNodeName(path); LOG.info("RegionServer ephemeral node deleted, processing expiration [" + serverName + "]"); @@ -130,11 +130,11 @@ public class RegionServerTracker extends ZooKeeperListener { @Override public void nodeChildrenChanged(String path) { - if (path.equals(watcher.rsZNode) + if (path.equals(watcher.znodePaths.rsZNode) && !server.isAborted() && !server.isStopped()) { try { List servers = - ZKUtil.listChildrenAndWatchThem(watcher, watcher.rsZNode); + ZKUtil.listChildrenAndWatchThem(watcher, watcher.znodePaths.rsZNode); add(servers); } catch (IOException e) { server.abort("Unexpected zk exception getting RS nodes", e); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKSplitLog.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKSplitLog.java index c6bc690..559931d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKSplitLog.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKSplitLog.java @@ -50,7 +50,7 @@ public class ZKSplitLog { * @param filename log file name (only the basename) */ public static String getEncodedNodeName(ZooKeeperWatcher zkw, String filename) { - return ZKUtil.joinZNode(zkw.splitLogZNode, encode(filename)); + return ZKUtil.joinZNode(zkw.znodePaths.splitLogZNode, encode(filename)); } public static String getFileName(String node) { @@ -75,7 +75,7 @@ public class ZKSplitLog { } public static String getRescanNode(ZooKeeperWatcher zkw) { - return ZKUtil.joinZNode(zkw.splitLogZNode, "RESCAN"); + return ZKUtil.joinZNode(zkw.znodePaths.splitLogZNode, "RESCAN"); } /** @@ -106,7 +106,7 @@ public class ZKSplitLog { public static boolean isTaskPath(ZooKeeperWatcher zkw, String path) { String dirname = path.substring(0, path.lastIndexOf('/')); - return dirname.equals(zkw.splitLogZNode); + return dirname.equals(zkw.znodePaths.splitLogZNode); } public static Path getSplitLogDir(Path rootdir, String tmpname) { @@ -153,7 +153,7 @@ public class ZKSplitLog { isRegionMarkedRecoveringInZK(ZooKeeperWatcher zkw, String regionEncodedName) throws KeeperException { boolean result = false; - String nodePath = ZKUtil.joinZNode(zkw.recoveringRegionsZNode, regionEncodedName); + String nodePath = ZKUtil.joinZNode(zkw.znodePaths.recoveringRegionsZNode, regionEncodedName); byte[] node = ZKUtil.getDataAndWatch(zkw, nodePath); if (node != null) { @@ -182,10 +182,10 @@ public class ZKSplitLog { if (regions == null) { // remove all children under /home/recovering-regions LOG.debug("Garbage collecting all recovering region znodes"); - ZKUtil.deleteChildrenRecursively(watcher, watcher.recoveringRegionsZNode); + ZKUtil.deleteChildrenRecursively(watcher, watcher.znodePaths.recoveringRegionsZNode); } else { for (String curRegion : regions) { - String nodePath = ZKUtil.joinZNode(watcher.recoveringRegionsZNode, curRegion); + String nodePath = ZKUtil.joinZNode(watcher.znodePaths.recoveringRegionsZNode, curRegion); ZKUtil.deleteNodeRecursively(watcher, nodePath); } } @@ -213,7 +213,7 @@ public class ZKSplitLog { // Therefore, in this mode we need to fetch last sequence Ids from ZK where we keep history of // last flushed sequence Id for each failed RS instance. RegionStoreSequenceIds result = null; - String nodePath = ZKUtil.joinZNode(zkw.recoveringRegionsZNode, encodedRegionName); + String nodePath = ZKUtil.joinZNode(zkw.znodePaths.recoveringRegionsZNode, encodedRegionName); nodePath = ZKUtil.joinZNode(nodePath, serverName); try { byte[] data; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/async/TestAsyncGet.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/async/TestAsyncGet.java new file mode 100644 index 0000000..9ef60d3 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/async/TestAsyncGet.java @@ -0,0 +1,77 @@ +/** + * 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.async; + +import static org.junit.Assert.assertArrayEquals; + +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.security.User; +import org.apache.hadoop.hbase.testclassification.ClientTests; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +/** + * Test simple async get + */ +@Category({ MediumTests.class, ClientTests.class }) +public class TestAsyncGet { + + private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); + + private static TableName TABLE_NAME = TableName.valueOf("async"); + + private static byte[] FAMILY = Bytes.toBytes("cf"); + + private static byte[] QUALIFIER = Bytes.toBytes("cq"); + + private static byte[] ROW = Bytes.toBytes("row"); + + private static byte[] VALUE = Bytes.toBytes("value"); + + private static AsyncConnection ASYNC_CONN; + + @BeforeClass + public static void setUp() throws Exception { + TEST_UTIL.startMiniCluster(1); + Table table = TEST_UTIL.createTable(TABLE_NAME, FAMILY); + TEST_UTIL.waitTableAvailable(TABLE_NAME); + table.put(new Put(ROW).addColumn(FAMILY, QUALIFIER, VALUE)); + ASYNC_CONN = new AsyncConnectionImpl(TEST_UTIL.getConfiguration(), User.getCurrent()); + } + + @AfterClass + public static void tearDown() throws Exception { + TEST_UTIL.shutdownMiniCluster(); + } + + @Test + public void test() throws Exception { + AsyncTable table = ASYNC_CONN.getTable(TABLE_NAME); + Result result = table.get(new Get(ROW).addColumn(FAMILY, QUALIFIER)).get(); + assertArrayEquals(VALUE, result.getValue(FAMILY, QUALIFIER)); + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/async/TestZKClusterRegistry.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/async/TestZKClusterRegistry.java new file mode 100644 index 0000000..ae8e70b --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/async/TestZKClusterRegistry.java @@ -0,0 +1,78 @@ +/** + * 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.async; + +import static org.hamcrest.CoreMatchers.hasItem; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThat; + +import java.io.IOException; +import java.util.List; + +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HRegionLocation; +import org.apache.hadoop.hbase.RegionLocations; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.testclassification.ClientTests; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker; +import org.apache.hadoop.hbase.zookeeper.MetaTableLocator; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; +import org.apache.zookeeper.KeeperException; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category({ MediumTests.class, ClientTests.class }) +public class TestZKClusterRegistry { + + private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); + + @BeforeClass + public static void setUp() throws Exception { + TEST_UTIL.startMiniCluster(1); + } + + @AfterClass + public static void tearDown() throws Exception { + TEST_UTIL.shutdownMiniCluster(); + } + + @Test + public void test() throws InterruptedException, IOException, KeeperException { + try (ZKClusterRegistry registry = new ZKClusterRegistry(TEST_UTIL.getConfiguration()); + ZooKeeperWatcher zkw = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(), "test", null)) { + Thread.sleep(2000); + Connection conn = TEST_UTIL.getConnection(); + assertEquals(conn.getAdmin().getClusterStatus().getClusterId(), registry.getClusterId()); + assertEquals(MasterAddressTracker.getMasterAddress(zkw), registry.getMasterAddress()); + assertEquals(MasterAddressTracker.getMasterInfoPort(zkw), registry.getMasterInfoPort()); + assertEquals(2, registry.getCurrentNrHRS()); + List expected = new MetaTableLocator().blockUntilAvailable(zkw, 5000, + TEST_UTIL.getConfiguration()); + RegionLocations locs = registry.getMetaRegionLocation(); + assertEquals(expected.size(), locs.getRegionLocations().length); + + for (HRegionLocation loc : locs.getRegionLocations()) { + assertThat(expected, hasItem(loc.getServerName())); + } + } + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestActiveMasterManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestActiveMasterManager.java index e10ab2a..7117c6f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestActiveMasterManager.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestActiveMasterManager.java @@ -73,7 +73,7 @@ public class TestActiveMasterManager { "testActiveMasterManagerFromZK", null, true); try { ZKUtil.deleteNode(zk, zk.getMasterAddressZNode()); - ZKUtil.deleteNode(zk, zk.clusterStateZNode); + ZKUtil.deleteNode(zk, zk.znodePaths.clusterStateZNode); } catch(KeeperException.NoNodeException nne) {} // Create the master node with a dummy address @@ -115,7 +115,7 @@ public class TestActiveMasterManager { "testActiveMasterManagerFromZK", null, true); try { ZKUtil.deleteNode(zk, zk.getMasterAddressZNode()); - ZKUtil.deleteNode(zk, zk.clusterStateZNode); + ZKUtil.deleteNode(zk, zk.znodePaths.clusterStateZNode); } catch(KeeperException.NoNodeException nne) {} // Create the master node with a dummy address diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java index 002438a..fd1bcef 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java @@ -459,7 +459,7 @@ public class TestDistributedLogSplitting { @Override public boolean evaluate() throws Exception { List recoveringRegions = zkw.getRecoverableZooKeeper().getChildren( - zkw.recoveringRegionsZNode, false); + zkw.znodePaths.recoveringRegionsZNode, false); return (recoveringRegions != null && recoveringRegions.size() == 0); } }); @@ -566,7 +566,7 @@ public class TestDistributedLogSplitting { @Override public boolean evaluate() throws Exception { List recoveringRegions = zkw.getRecoverableZooKeeper().getChildren( - zkw.recoveringRegionsZNode, false); + zkw.znodePaths.recoveringRegionsZNode, false); boolean done = recoveringRegions != null && recoveringRegions.size() == 0; if (!done) { LOG.info("Recovering regions: " + recoveringRegions); @@ -658,7 +658,7 @@ public class TestDistributedLogSplitting { @Override public boolean evaluate() throws Exception { List recoveringRegions = zkw.getRecoverableZooKeeper().getChildren( - zkw.recoveringRegionsZNode, false); + zkw.znodePaths.recoveringRegionsZNode, false); return (recoveringRegions != null && recoveringRegions.size() == 0); } }); @@ -702,7 +702,7 @@ public class TestDistributedLogSplitting { slm.markRegionsRecovering(secondFailedServer, regionSet); List recoveringRegions = ZKUtil.listChildrenNoWatch(zkw, - ZKUtil.joinZNode(zkw.recoveringRegionsZNode, region.getEncodedName())); + ZKUtil.joinZNode(zkw.znodePaths.recoveringRegionsZNode, region.getEncodedName())); assertEquals(recoveringRegions.size(), 2); @@ -859,7 +859,7 @@ public class TestDistributedLogSplitting { @Override public boolean evaluate() throws Exception { List recoveringRegions = zkw.getRecoverableZooKeeper().getChildren( - zkw.recoveringRegionsZNode, false); + zkw.znodePaths.recoveringRegionsZNode, false); ServerManager serverManager = master.getServerManager(); return (!serverManager.areDeadServersInProgress() && recoveringRegions != null && recoveringRegions.size() == 0); @@ -1104,7 +1104,7 @@ public class TestDistributedLogSplitting { @Override public boolean evaluate() throws Exception { List recoveringRegions = zkw.getRecoverableZooKeeper().getChildren( - zkw.recoveringRegionsZNode, false); + zkw.znodePaths.recoveringRegionsZNode, false); return (recoveringRegions != null && recoveringRegions.size() == 0); } }); @@ -1210,7 +1210,7 @@ public class TestDistributedLogSplitting { master.getMasterWalManager().prepareLogReplay(hrs.getServerName(), userRegionSet); boolean isMetaRegionInRecovery = false; List recoveringRegions = - zkw.getRecoverableZooKeeper().getChildren(zkw.recoveringRegionsZNode, false); + zkw.getRecoverableZooKeeper().getChildren(zkw.znodePaths.recoveringRegionsZNode, false); for (String curEncodedRegionName : recoveringRegions) { if (curEncodedRegionName.equals(HRegionInfo.FIRST_META_REGIONINFO.getEncodedName())) { isMetaRegionInRecovery = true; @@ -1223,7 +1223,7 @@ public class TestDistributedLogSplitting { isMetaRegionInRecovery = false; recoveringRegions = - zkw.getRecoverableZooKeeper().getChildren(zkw.recoveringRegionsZNode, false); + zkw.getRecoverableZooKeeper().getChildren(zkw.znodePaths.recoveringRegionsZNode, false); for (String curEncodedRegionName : recoveringRegions) { if (curEncodedRegionName.equals(HRegionInfo.FIRST_META_REGIONINFO.getEncodedName())) { isMetaRegionInRecovery = true; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java index 6da7a38..5be278f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java @@ -117,7 +117,7 @@ public class TestMasterNoCluster { return false; } }); - ZKUtil.deleteNodeRecursively(zkw, zkw.baseZNode); + ZKUtil.deleteNodeRecursively(zkw, zkw.znodePaths.baseZNode); zkw.close(); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterWalManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterWalManager.java index feb97d9..7c7531f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterWalManager.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterWalManager.java @@ -83,9 +83,10 @@ public class TestMasterWalManager { zkw.getRecoverableZooKeeper().create(ZKSplitLog.getEncodedNodeName(zkw, walPath), new SplitLogTask.Owned(inRecoveryServerName, mwm.getLogRecoveryMode()).toByteArray(), Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); - String staleRegionPath = ZKUtil.joinZNode(zkw.recoveringRegionsZNode, staleRegion); + String staleRegionPath = ZKUtil.joinZNode(zkw.znodePaths.recoveringRegionsZNode, staleRegion); ZKUtil.createWithParents(zkw, staleRegionPath); - String inRecoveringRegionPath = ZKUtil.joinZNode(zkw.recoveringRegionsZNode, failedRegion); + String inRecoveringRegionPath = ZKUtil.joinZNode(zkw.znodePaths.recoveringRegionsZNode, + failedRegion); inRecoveringRegionPath = ZKUtil.joinZNode(inRecoveringRegionPath, inRecoveryServerName.getServerName()); ZKUtil.createWithParents(zkw, inRecoveringRegionPath); @@ -97,8 +98,8 @@ public class TestMasterWalManager { assertFalse(ZKUtil.checkExists(zkw, staleRegionPath) != -1); assertTrue(ZKUtil.checkExists(zkw, inRecoveringRegionPath) != -1); - ZKUtil.deleteChildrenRecursively(zkw, zkw.recoveringRegionsZNode); - ZKUtil.deleteChildrenRecursively(zkw, zkw.splitLogZNode); + ZKUtil.deleteChildrenRecursively(zkw, zkw.znodePaths.recoveringRegionsZNode); + ZKUtil.deleteChildrenRecursively(zkw, zkw.znodePaths.splitLogZNode); zkw.close(); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMetaShutdownHandler.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMetaShutdownHandler.java index 619d367..a845a73 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMetaShutdownHandler.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMetaShutdownHandler.java @@ -94,7 +94,7 @@ public class TestMetaShutdownHandler { // Delete the ephemeral node of the meta-carrying region server. // This is trigger the expire of this region server on the master. String rsEphemeralNodePath = - ZKUtil.joinZNode(master.getZooKeeper().rsZNode, metaServerName.toString()); + ZKUtil.joinZNode(master.getZooKeeper().znodePaths.rsZNode, metaServerName.toString()); ZKUtil.deleteNode(master.getZooKeeper(), rsEphemeralNodePath); // Wait for SSH to finish final ServerManager serverManager = master.getServerManager(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java index d928d1c..93537f0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java @@ -134,13 +134,13 @@ public class TestSplitLogManager { new ZooKeeperWatcher(conf, "split-log-manager-tests" + UUID.randomUUID().toString(), null); master = new DummyMasterServices(zkw, conf); - ZKUtil.deleteChildrenRecursively(zkw, zkw.baseZNode); - ZKUtil.createAndFailSilent(zkw, zkw.baseZNode); - assertTrue(ZKUtil.checkExists(zkw, zkw.baseZNode) != -1); - LOG.debug(zkw.baseZNode + " created"); - ZKUtil.createAndFailSilent(zkw, zkw.splitLogZNode); - assertTrue(ZKUtil.checkExists(zkw, zkw.splitLogZNode) != -1); - LOG.debug(zkw.splitLogZNode + " created"); + ZKUtil.deleteChildrenRecursively(zkw, zkw.znodePaths.baseZNode); + ZKUtil.createAndFailSilent(zkw, zkw.znodePaths.baseZNode); + assertTrue(ZKUtil.checkExists(zkw, zkw.znodePaths.baseZNode) != -1); + LOG.debug(zkw.znodePaths.baseZNode + " created"); + ZKUtil.createAndFailSilent(zkw, zkw.znodePaths.splitLogZNode); + assertTrue(ZKUtil.checkExists(zkw, zkw.znodePaths.splitLogZNode) != -1); + LOG.debug(zkw.znodePaths.splitLogZNode + " created"); resetCounters(); @@ -567,7 +567,7 @@ public class TestSplitLogManager { LOG.info("testRecoveryRegionRemovedFromZK"); conf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, false); String nodePath = - ZKUtil.joinZNode(zkw.recoveringRegionsZNode, + ZKUtil.joinZNode(zkw.znodePaths.recoveringRegionsZNode, HRegionInfo.FIRST_META_REGIONINFO.getEncodedName()); ZKUtil.createSetData(zkw, nodePath, ZKUtil.positionToByteArray(0L)); @@ -575,7 +575,7 @@ public class TestSplitLogManager { slm.removeStaleRecoveringRegions(null); List recoveringRegions = - zkw.getRecoverableZooKeeper().getChildren(zkw.recoveringRegionsZNode, false); + zkw.getRecoverableZooKeeper().getChildren(zkw.znodePaths.recoveringRegionsZNode, false); assertTrue("Recovery regions isn't cleaned", recoveringRegions.isEmpty()); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestTableLockManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestTableLockManager.java index 36f505b..7426437 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestTableLockManager.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestTableLockManager.java @@ -220,7 +220,8 @@ public class TestTableLockManager { //ensure that znode for the table node has been deleted final ZooKeeperWatcher zkWatcher = TEST_UTIL.getZooKeeperWatcher(); - final String znode = ZKUtil.joinZNode(zkWatcher.tableLockZNode, TABLE_NAME.getNameAsString()); + final String znode = ZKUtil.joinZNode(zkWatcher.znodePaths.tableLockZNode, + TABLE_NAME.getNameAsString()); TEST_UTIL.waitFor(5000, new Waiter.Predicate() { @Override @@ -230,7 +231,7 @@ public class TestTableLockManager { } }); int ver = ZKUtil.checkExists(zkWatcher, - ZKUtil.joinZNode(zkWatcher.tableLockZNode, TABLE_NAME.getNameAsString())); + ZKUtil.joinZNode(zkWatcher.znodePaths.tableLockZNode, TABLE_NAME.getNameAsString())); assertTrue("Unexpected znode version " + ver, ver < 0); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestTableStateManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestTableStateManager.java index e0e969e..3b12a1b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestTableStateManager.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestTableStateManager.java @@ -68,7 +68,7 @@ public class TestTableStateManager { private void setTableStateInZK(ZooKeeperWatcher watcher, final TableName tableName, final ZooKeeperProtos.DeprecatedTableState.State state) throws KeeperException, IOException { - String znode = ZKUtil.joinZNode(watcher.tableZNode, tableName.getNameAsString()); + String znode = ZKUtil.joinZNode(watcher.znodePaths.tableZNode, tableName.getNameAsString()); if (ZKUtil.checkExists(watcher, znode) == -1) { ZKUtil.createAndFailSilent(watcher, znode); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/mapreduce/TestMobSweepMapper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/mapreduce/TestMobSweepMapper.java index cec8a74..44131ba 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/mapreduce/TestMobSweepMapper.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/mapreduce/TestMobSweepMapper.java @@ -79,7 +79,7 @@ public class TestMobSweepMapper { ZooKeeperWatcher zkw = new ZooKeeperWatcher(configuration, "1", new DummyMobAbortable()); TableName tn = TableName.valueOf("testSweepMapper"); TableName lockName = MobUtils.getTableLockName(tn); - String znode = ZKUtil.joinZNode(zkw.tableLockZNode, lockName.getNameAsString()); + String znode = ZKUtil.joinZNode(zkw.znodePaths.tableLockZNode, lockName.getNameAsString()); configuration.set(SweepJob.SWEEP_JOB_ID, "1"); configuration.set(SweepJob.SWEEP_JOB_TABLE_NODE, znode); ServerName serverName = SweepJob.getCurrentServerName(configuration); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/mapreduce/TestMobSweepReducer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/mapreduce/TestMobSweepReducer.java index 8cef4d9..abe0d3a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/mapreduce/TestMobSweepReducer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/mapreduce/TestMobSweepReducer.java @@ -162,7 +162,7 @@ public class TestMobSweepReducer { ZooKeeperWatcher zkw = new ZooKeeperWatcher(configuration, "1", new DummyMobAbortable()); TableName lockName = MobUtils.getTableLockName(tn); - String znode = ZKUtil.joinZNode(zkw.tableLockZNode, lockName.getNameAsString()); + String znode = ZKUtil.joinZNode(zkw.znodePaths.tableLockZNode, lockName.getNameAsString()); configuration.set(SweepJob.SWEEP_JOB_ID, "1"); configuration.set(SweepJob.SWEEP_JOB_TABLE_NODE, znode); ServerName serverName = SweepJob.getCurrentServerName(configuration); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMasterAddressTracker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMasterAddressTracker.java index 487bb25..b72beb1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMasterAddressTracker.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMasterAddressTracker.java @@ -82,7 +82,7 @@ public class TestMasterAddressTracker { throws Exception { ZooKeeperWatcher zk = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(), name.getMethodName(), null); - ZKUtil.createAndFailSilent(zk, zk.baseZNode); + ZKUtil.createAndFailSilent(zk, zk.znodePaths.baseZNode); // Should not have a master yet MasterAddressTracker addressTracker = new MasterAddressTracker(zk, null); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerHostname.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerHostname.java index c48fbec..63e33d4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerHostname.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerHostname.java @@ -88,7 +88,7 @@ public class TestRegionServerHostname { TEST_UTIL.startMiniCluster(NUM_MASTERS, NUM_RS); try { ZooKeeperWatcher zkw = TEST_UTIL.getZooKeeperWatcher(); - List servers = ZKUtil.listChildrenNoWatch(zkw, zkw.rsZNode); + List servers = ZKUtil.listChildrenNoWatch(zkw, zkw.znodePaths.rsZNode); // there would be NUM_RS+1 children - one for the master assertTrue(servers.size() == NUM_RS+1); for (String server : servers) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitLogWorker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitLogWorker.java index 96ec698..dd4ccf2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitLogWorker.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitLogWorker.java @@ -188,16 +188,16 @@ public class TestSplitLogWorker { zkw = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(), "split-log-worker-tests", null); ds = new DummyServer(zkw, conf); - ZKUtil.deleteChildrenRecursively(zkw, zkw.baseZNode); - ZKUtil.createAndFailSilent(zkw, zkw.baseZNode); - assertThat(ZKUtil.checkExists(zkw, zkw.baseZNode), not (is(-1))); - LOG.debug(zkw.baseZNode + " created"); - ZKUtil.createAndFailSilent(zkw, zkw.splitLogZNode); - assertThat(ZKUtil.checkExists(zkw, zkw.splitLogZNode), not (is(-1))); + ZKUtil.deleteChildrenRecursively(zkw, zkw.znodePaths.baseZNode); + ZKUtil.createAndFailSilent(zkw, zkw.znodePaths.baseZNode); + assertThat(ZKUtil.checkExists(zkw, zkw.znodePaths.baseZNode), not (is(-1))); + LOG.debug(zkw.znodePaths.baseZNode + " created"); + ZKUtil.createAndFailSilent(zkw, zkw.znodePaths.splitLogZNode); + assertThat(ZKUtil.checkExists(zkw, zkw.znodePaths.splitLogZNode), not (is(-1))); - LOG.debug(zkw.splitLogZNode + " created"); - ZKUtil.createAndFailSilent(zkw, zkw.rsZNode); - assertThat(ZKUtil.checkExists(zkw, zkw.rsZNode), not (is(-1))); + LOG.debug(zkw.znodePaths.splitLogZNode + " created"); + ZKUtil.createAndFailSilent(zkw, zkw.znodePaths.rsZNode); + assertThat(ZKUtil.checkExists(zkw, zkw.znodePaths.rsZNode), not (is(-1))); SplitLogCounters.resetCounters(); executorService = new ExecutorService("TestSplitLogWorker"); @@ -416,7 +416,7 @@ public class TestSplitLogWorker { waitForCounter(SplitLogCounters.tot_wkr_preempt_task, 1, 2, WAIT_TIME); waitForCounter(SplitLogCounters.tot_wkr_task_acquired_rescan, 0, 1, WAIT_TIME); - List nodes = ZKUtil.listChildrenNoWatch(zkw, zkw.splitLogZNode); + List nodes = ZKUtil.listChildrenNoWatch(zkw, zkw.znodePaths.splitLogZNode); LOG.debug(nodes); int num = 0; for (String node : nodes) { @@ -424,7 +424,7 @@ public class TestSplitLogWorker { if (node.startsWith("RESCAN")) { String name = ZKSplitLog.getEncodedNodeName(zkw, node); String fn = ZKSplitLog.getFileName(name); - byte [] data = ZKUtil.getData(zkw, ZKUtil.joinZNode(zkw.splitLogZNode, fn)); + byte [] data = ZKUtil.getData(zkw, ZKUtil.joinZNode(zkw.znodePaths.splitLogZNode, fn)); slt = SplitLogTask.parseFrom(data); assertTrue(slt.toString(), slt.isDone(SRV)); } @@ -480,9 +480,9 @@ public class TestSplitLogWorker { RegionServerServices mockedRS = getRegionServer(RS); // create two RS nodes - String rsPath = ZKUtil.joinZNode(zkw.rsZNode, RS.getServerName()); + String rsPath = ZKUtil.joinZNode(zkw.znodePaths.rsZNode, RS.getServerName()); zkw.getRecoverableZooKeeper().create(rsPath, null, Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL); - rsPath = ZKUtil.joinZNode(zkw.rsZNode, RS2.getServerName()); + rsPath = ZKUtil.joinZNode(zkw.znodePaths.rsZNode, RS2.getServerName()); zkw.getRecoverableZooKeeper().create(rsPath, null, Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL); for (int i = 0; i < maxTasks; i++) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateHBaseImpl.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateHBaseImpl.java index 35c4e24..3685d6d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateHBaseImpl.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateHBaseImpl.java @@ -87,7 +87,7 @@ public class TestReplicationStateHBaseImpl { utility.startMiniCluster(); zkw = HBaseTestingUtility.getZooKeeperWatcher(utility); String replicationZNodeName = conf.get("zookeeper.znode.replication", "replication"); - replicationZNode = ZKUtil.joinZNode(zkw.baseZNode, replicationZNodeName); + replicationZNode = ZKUtil.joinZNode(zkw.znodePaths.baseZNode, replicationZNodeName); } @Before diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java index a357a1f..fc35b54 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java @@ -68,7 +68,7 @@ public class TestReplicationStateZKImpl extends TestReplicationStateBasic { conf.setBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY, true); zkw = HBaseTestingUtility.getZooKeeperWatcher(utility); String replicationZNodeName = conf.get("zookeeper.znode.replication", "replication"); - replicationZNode = ZKUtil.joinZNode(zkw.baseZNode, replicationZNodeName); + replicationZNode = ZKUtil.joinZNode(zkw.znodePaths.baseZNode, replicationZNodeName); KEY_ONE = initPeerClusterState("/hbase1"); KEY_TWO = initPeerClusterState("/hbase2"); } @@ -79,7 +79,7 @@ public class TestReplicationStateZKImpl extends TestReplicationStateBasic { Configuration testConf = new Configuration(conf); testConf.set(HConstants.ZOOKEEPER_ZNODE_PARENT, baseZKNode); ZooKeeperWatcher zkw1 = new ZooKeeperWatcher(testConf, "test1", null); - String fakeRs = ZKUtil.joinZNode(zkw1.rsZNode, "hostname1.example.org:1234"); + String fakeRs = ZKUtil.joinZNode(zkw1.znodePaths.rsZNode, "hostname1.example.org:1234"); ZKUtil.createWithParents(zkw1, fakeRs); ZKClusterId.setClusterId(zkw1, new ClusterId()); return ZKConfig.getZooKeeperClusterKey(testConf); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java index 3b19660..7033c3d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java @@ -81,13 +81,13 @@ public class TestReplicationTrackerZKImpl { utility.startMiniZKCluster(); conf = utility.getConfiguration(); ZooKeeperWatcher zk = HBaseTestingUtility.getZooKeeperWatcher(utility); - ZKUtil.createWithParents(zk, zk.rsZNode); + ZKUtil.createWithParents(zk, zk.znodePaths.rsZNode); } @Before public void setUp() throws Exception { zkw = HBaseTestingUtility.getZooKeeperWatcher(utility); - String fakeRs1 = ZKUtil.joinZNode(zkw.rsZNode, "hostname1.example.org:1234"); + String fakeRs1 = ZKUtil.joinZNode(zkw.znodePaths.rsZNode, "hostname1.example.org:1234"); try { ZKClusterId.setClusterId(zkw, new ClusterId()); rp = ReplicationFactory.getReplicationPeers(zkw, conf, zkw); @@ -115,29 +115,29 @@ public class TestReplicationTrackerZKImpl { assertEquals(0, rt.getListOfRegionServers().size()); // 1 region server - ZKUtil.createWithParents(zkw, ZKUtil.joinZNode(zkw.rsZNode, "hostname1.example.org:1234")); + ZKUtil.createWithParents(zkw, ZKUtil.joinZNode(zkw.znodePaths.rsZNode, "hostname1.example.org:1234")); assertEquals(1, rt.getListOfRegionServers().size()); // 2 region servers - ZKUtil.createWithParents(zkw, ZKUtil.joinZNode(zkw.rsZNode, "hostname2.example.org:1234")); + ZKUtil.createWithParents(zkw, ZKUtil.joinZNode(zkw.znodePaths.rsZNode, "hostname2.example.org:1234")); assertEquals(2, rt.getListOfRegionServers().size()); // 1 region server - ZKUtil.deleteNode(zkw, ZKUtil.joinZNode(zkw.rsZNode, "hostname2.example.org:1234")); + ZKUtil.deleteNode(zkw, ZKUtil.joinZNode(zkw.znodePaths.rsZNode, "hostname2.example.org:1234")); assertEquals(1, rt.getListOfRegionServers().size()); // 0 region server - ZKUtil.deleteNode(zkw, ZKUtil.joinZNode(zkw.rsZNode, "hostname1.example.org:1234")); + ZKUtil.deleteNode(zkw, ZKUtil.joinZNode(zkw.znodePaths.rsZNode, "hostname1.example.org:1234")); assertEquals(0, rt.getListOfRegionServers().size()); } @Test(timeout = 30000) public void testRegionServerRemovedEvent() throws Exception { - ZKUtil.createAndWatch(zkw, ZKUtil.joinZNode(zkw.rsZNode, "hostname2.example.org:1234"), + ZKUtil.createAndWatch(zkw, ZKUtil.joinZNode(zkw.znodePaths.rsZNode, "hostname2.example.org:1234"), HConstants.EMPTY_BYTE_ARRAY); rt.registerListener(new DummyReplicationListener()); // delete one - ZKUtil.deleteNode(zkw, ZKUtil.joinZNode(zkw.rsZNode, "hostname2.example.org:1234")); + ZKUtil.deleteNode(zkw, ZKUtil.joinZNode(zkw.znodePaths.rsZNode, "hostname2.example.org:1234")); // wait for event while (rsRemovedCount.get() < 1) { Thread.sleep(5); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKMulti.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKMulti.java index 43d7139..6350af8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKMulti.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKMulti.java @@ -88,7 +88,7 @@ public class TestZKMulti { ZKUtil.multiOrSequential(zkw, new LinkedList(), false); // single create - String path = ZKUtil.joinZNode(zkw.baseZNode, "testSimpleMulti"); + String path = ZKUtil.joinZNode(zkw.znodePaths.baseZNode, "testSimpleMulti"); LinkedList singleCreate = new LinkedList(); singleCreate.add(ZKUtilOp.createAndFailSilent(path, new byte[0])); ZKUtil.multiOrSequential(zkw, singleCreate, false); @@ -110,12 +110,12 @@ public class TestZKMulti { @Test (timeout=60000) public void testComplexMulti() throws Exception { - String path1 = ZKUtil.joinZNode(zkw.baseZNode, "testComplexMulti1"); - String path2 = ZKUtil.joinZNode(zkw.baseZNode, "testComplexMulti2"); - String path3 = ZKUtil.joinZNode(zkw.baseZNode, "testComplexMulti3"); - String path4 = ZKUtil.joinZNode(zkw.baseZNode, "testComplexMulti4"); - String path5 = ZKUtil.joinZNode(zkw.baseZNode, "testComplexMulti5"); - String path6 = ZKUtil.joinZNode(zkw.baseZNode, "testComplexMulti6"); + String path1 = ZKUtil.joinZNode(zkw.znodePaths.baseZNode, "testComplexMulti1"); + String path2 = ZKUtil.joinZNode(zkw.znodePaths.baseZNode, "testComplexMulti2"); + String path3 = ZKUtil.joinZNode(zkw.znodePaths.baseZNode, "testComplexMulti3"); + String path4 = ZKUtil.joinZNode(zkw.znodePaths.baseZNode, "testComplexMulti4"); + String path5 = ZKUtil.joinZNode(zkw.znodePaths.baseZNode, "testComplexMulti5"); + String path6 = ZKUtil.joinZNode(zkw.znodePaths.baseZNode, "testComplexMulti6"); // create 4 nodes that we'll setData on or delete later LinkedList create4Nodes = new LinkedList(); create4Nodes.add(ZKUtilOp.createAndFailSilent(path1, Bytes.toBytes(path1))); @@ -154,7 +154,7 @@ public class TestZKMulti { public void testSingleFailure() throws Exception { // try to delete a node that doesn't exist boolean caughtNoNode = false; - String path = ZKUtil.joinZNode(zkw.baseZNode, "testSingleFailureZ"); + String path = ZKUtil.joinZNode(zkw.znodePaths.baseZNode, "testSingleFailureZ"); LinkedList ops = new LinkedList(); ops.add(ZKUtilOp.deleteNodeFailSilent(path)); try { @@ -191,9 +191,9 @@ public class TestZKMulti { @Test (timeout=60000) public void testSingleFailureInMulti() throws Exception { // try a multi where all but one operation succeeds - String pathA = ZKUtil.joinZNode(zkw.baseZNode, "testSingleFailureInMultiA"); - String pathB = ZKUtil.joinZNode(zkw.baseZNode, "testSingleFailureInMultiB"); - String pathC = ZKUtil.joinZNode(zkw.baseZNode, "testSingleFailureInMultiC"); + String pathA = ZKUtil.joinZNode(zkw.znodePaths.baseZNode, "testSingleFailureInMultiA"); + String pathB = ZKUtil.joinZNode(zkw.znodePaths.baseZNode, "testSingleFailureInMultiB"); + String pathC = ZKUtil.joinZNode(zkw.znodePaths.baseZNode, "testSingleFailureInMultiC"); LinkedList ops = new LinkedList(); ops.add(ZKUtilOp.createAndFailSilent(pathA, Bytes.toBytes(pathA))); ops.add(ZKUtilOp.createAndFailSilent(pathB, Bytes.toBytes(pathB))); @@ -213,17 +213,17 @@ public class TestZKMulti { @Test (timeout=60000) public void testMultiFailure() throws Exception { - String pathX = ZKUtil.joinZNode(zkw.baseZNode, "testMultiFailureX"); - String pathY = ZKUtil.joinZNode(zkw.baseZNode, "testMultiFailureY"); - String pathZ = ZKUtil.joinZNode(zkw.baseZNode, "testMultiFailureZ"); + String pathX = ZKUtil.joinZNode(zkw.znodePaths.baseZNode, "testMultiFailureX"); + String pathY = ZKUtil.joinZNode(zkw.znodePaths.baseZNode, "testMultiFailureY"); + String pathZ = ZKUtil.joinZNode(zkw.znodePaths.baseZNode, "testMultiFailureZ"); // create X that we will use to fail create later LinkedList ops = new LinkedList(); ops.add(ZKUtilOp.createAndFailSilent(pathX, Bytes.toBytes(pathX))); ZKUtil.multiOrSequential(zkw, ops, false); // fail one of each create ,setData, delete - String pathV = ZKUtil.joinZNode(zkw.baseZNode, "testMultiFailureV"); - String pathW = ZKUtil.joinZNode(zkw.baseZNode, "testMultiFailureW"); + String pathV = ZKUtil.joinZNode(zkw.znodePaths.baseZNode, "testMultiFailureV"); + String pathW = ZKUtil.joinZNode(zkw.znodePaths.baseZNode, "testMultiFailureW"); ops = new LinkedList(); ops.add(ZKUtilOp.createAndFailSilent(pathX, Bytes.toBytes(pathX))); // fail -- already exists ops.add(ZKUtilOp.setData(pathY, Bytes.toBytes(pathY))); // fail -- doesn't exist @@ -267,10 +267,10 @@ public class TestZKMulti { @Test (timeout=60000) public void testRunSequentialOnMultiFailure() throws Exception { - String path1 = ZKUtil.joinZNode(zkw.baseZNode, "runSequential1"); - String path2 = ZKUtil.joinZNode(zkw.baseZNode, "runSequential2"); - String path3 = ZKUtil.joinZNode(zkw.baseZNode, "runSequential3"); - String path4 = ZKUtil.joinZNode(zkw.baseZNode, "runSequential4"); + String path1 = ZKUtil.joinZNode(zkw.znodePaths.baseZNode, "runSequential1"); + String path2 = ZKUtil.joinZNode(zkw.znodePaths.baseZNode, "runSequential2"); + String path3 = ZKUtil.joinZNode(zkw.znodePaths.baseZNode, "runSequential3"); + String path4 = ZKUtil.joinZNode(zkw.znodePaths.baseZNode, "runSequential4"); // create some nodes that we will use later LinkedList ops = new LinkedList(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperNodeTracker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperNodeTracker.java index 010c1c9..ba1fce6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperNodeTracker.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperNodeTracker.java @@ -97,10 +97,10 @@ public class TestZooKeeperNodeTracker { Abortable abortable = new StubAbortable(); ZooKeeperWatcher zk = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(), "testNodeTracker", abortable); - ZKUtil.createAndFailSilent(zk, zk.baseZNode); + ZKUtil.createAndFailSilent(zk, zk.znodePaths.baseZNode); final String node = - ZKUtil.joinZNode(zk.baseZNode, new Long(rand.nextLong()).toString()); + ZKUtil.joinZNode(zk.znodePaths.baseZNode, new Long(rand.nextLong()).toString()); final byte [] dataOne = Bytes.toBytes("dataOne"); final byte [] dataTwo = Bytes.toBytes("dataTwo"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/lock/TestZKInterProcessReadWriteLock.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/lock/TestZKInterProcessReadWriteLock.java index c304842..675afc0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/lock/TestZKInterProcessReadWriteLock.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/lock/TestZKInterProcessReadWriteLock.java @@ -80,7 +80,7 @@ public class TestZKInterProcessReadWriteLock { TEST_UTIL.startMiniZKCluster(); conf.setInt(HConstants.ZK_SESSION_TIMEOUT, 1000); ZooKeeperWatcher zkw = getZooKeeperWatcher("setup"); - ZKUtil.createWithParents(zkw, zkw.tableLockZNode); + ZKUtil.createWithParents(zkw, zkw.znodePaths.tableLockZNode); } @AfterClass @@ -226,7 +226,7 @@ public class TestZKInterProcessReadWriteLock { } }; ZooKeeperWatcher zkWatcher = getZooKeeperWatcher(testName); - String znode = ZKUtil.joinZNode(zkWatcher.tableLockZNode, testName); + String znode = ZKUtil.joinZNode(zkWatcher.znodePaths.tableLockZNode, testName); return new ZKInterProcessReadWriteLock(zkWatcher, znode, handler); } @@ -334,7 +334,7 @@ public class TestZKInterProcessReadWriteLock { ZooKeeperWatcher zkWatcher1 = new ZooKeeperWatcher(conf, "testMultipleClients-1", null); ZooKeeperWatcher zkWatcher2 = new ZooKeeperWatcher(conf, "testMultipleClients-2", null); - String znode = ZKUtil.joinZNode(zkWatcher1.tableLockZNode, testName); + String znode = ZKUtil.joinZNode(zkWatcher1.znodePaths.tableLockZNode, testName); ZKInterProcessReadWriteLock clientLock1 = new ZKInterProcessReadWriteLock(zkWatcher1, znode, null); diff --git a/pom.xml b/pom.xml index c148b19..7042ace 100644 --- a/pom.xml +++ b/pom.xml @@ -1227,6 +1227,7 @@ 6.18 2.10.3 1.5.2.1 + 2.11.0 /usr /etc/hbase @@ -1765,28 +1766,39 @@ disruptor ${disruptor.version} - + net.spy spymemcached ${spy.version} true - - - org.bouncycastle - bcprov-jdk16 - ${bouncycastle.version} - test - - - org.apache.kerby - kerb-client - ${kerby.version} - - - org.apache.kerby - kerb-simplekdc - ${kerby.version} - + + + org.bouncycastle + bcprov-jdk16 + ${bouncycastle.version} + test + + + org.apache.kerby + kerb-client + ${kerby.version} + + + org.apache.kerby + kerb-simplekdc + ${kerby.version} + + + org.apache.curator + curator-framework + ${curator.version} + + + com.google.guava + guava + + + -- 2.7.4